You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@tez.apache.org by ss...@apache.org on 2013/09/25 09:31:08 UTC

[01/50] [abbrv] git commit: TEZ-439. Fix task attempt commit flow (bikas)

Updated Branches:
  refs/heads/master 92f458b9a -> f1cbb1c60


TEZ-439. Fix task attempt commit flow (bikas)


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

Branch: refs/heads/master
Commit: 6ca59ac727a0db2d1b033cca269325f49ece6ccd
Parents: bd76ffc
Author: Bikas Saha <bi...@apache.org>
Authored: Fri Sep 20 10:31:28 2013 -0700
Committer: Bikas Saha <bi...@apache.org>
Committed: Fri Sep 20 10:31:28 2013 -0700

----------------------------------------------------------------------
 .../apache/hadoop/mapred/YarnTezDagChild.java   |   6 -
 .../dag/api/oldrecords/TaskAttemptState.java    |   1 -
 .../dag/app/TaskAttemptListenerImpTezDag.java   |  30 -----
 .../org/apache/tez/dag/app/dag/TaskAttempt.java |   1 +
 .../dag/app/dag/TaskAttemptStateInternal.java   |   1 -
 .../tez/dag/app/dag/event/TaskEventType.java    |   1 -
 .../tez/dag/app/dag/impl/TaskAttemptImpl.java   |  50 ++------
 .../apache/tez/dag/app/dag/impl/TaskImpl.java   | 114 +++++++++++--------
 .../tez/dag/app/dag/impl/TestTaskAttempt.java   |  62 ----------
 .../tez/dag/app/dag/impl/TestTaskImpl.java      |  63 +++++++---
 .../tez/engine/newapi/TezProcessorContext.java  |   7 --
 .../tez/common/TezTaskUmbilicalProtocol.java    |   3 -
 .../newapi/impl/TezProcessorContextImpl.java    |   5 -
 .../tez/engine/newapi/impl/TezUmbilical.java    |   3 -
 .../apache/hadoop/mapred/LocalJobRunnerTez.java |   7 --
 .../tez/mapreduce/newprocessor/MRTask.java      |  32 ++----
 .../apache/tez/mapreduce/processor/MRTask.java  |  18 ---
 .../tez/mapreduce/TestUmbilicalProtocol.java    |   7 --
 18 files changed, 141 insertions(+), 270 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/6ca59ac7/tez-dag/src/main/java/org/apache/hadoop/mapred/YarnTezDagChild.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/hadoop/mapred/YarnTezDagChild.java b/tez-dag/src/main/java/org/apache/hadoop/mapred/YarnTezDagChild.java
index 6e4e418..5034262 100644
--- a/tez-dag/src/main/java/org/apache/hadoop/mapred/YarnTezDagChild.java
+++ b/tez-dag/src/main/java/org/apache/hadoop/mapred/YarnTezDagChild.java
@@ -333,12 +333,6 @@ public class YarnTezDagChild {
           throws IOException {
         return umbilical.canCommit(taskAttemptID);
       }
-
-      @Override
-      public void commitPending(TezTaskAttemptID taskAttemptID)
-          throws IOException, InterruptedException {
-        umbilical.commitPending(taskAttemptID);
-      }
     };
 
     // report non-pid to application master

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/6ca59ac7/tez-dag/src/main/java/org/apache/tez/dag/api/oldrecords/TaskAttemptState.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/api/oldrecords/TaskAttemptState.java b/tez-dag/src/main/java/org/apache/tez/dag/api/oldrecords/TaskAttemptState.java
index 068913b..926835a 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/api/oldrecords/TaskAttemptState.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/api/oldrecords/TaskAttemptState.java
@@ -22,7 +22,6 @@ public enum TaskAttemptState {
   NEW, 
   STARTING, 
   RUNNING, 
-  COMMIT_PENDING,  
   SUCCEEDED,
   FAILED,
   KILLED

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/6ca59ac7/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java b/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java
index 8c29fd9..2be9c5f 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java
@@ -44,9 +44,7 @@ import org.apache.tez.common.records.ProceedToCompletionResponse;
 import org.apache.tez.dag.api.TezConfiguration;
 import org.apache.tez.dag.app.dag.DAG;
 import org.apache.tez.dag.app.dag.Task;
-import org.apache.tez.dag.app.dag.event.TaskAttemptEvent;
 import org.apache.tez.dag.app.dag.event.TaskAttemptEventOutputConsumable;
-import org.apache.tez.dag.app.dag.event.TaskAttemptEventType;
 import org.apache.tez.dag.app.dag.event.TaskAttemptEventStartedRemotely;
 import org.apache.tez.dag.app.dag.event.VertexEventRouteEvent;
 import org.apache.tez.dag.app.rm.container.AMContainerImpl;
@@ -346,34 +344,6 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
   */
 
   /**
-   * TaskAttempt is reporting that it is in commit_pending and it is waiting for
-   * the commit Response
-   *
-   * <br/>
-   * Commit it a two-phased protocol. First the attempt informs the
-   * ApplicationMaster that it is
-   * {@link #commitPending(TaskAttemptID, TaskStatus)}. Then it repeatedly polls
-   * the ApplicationMaster whether it {@link #canCommit(TaskAttemptID)} This is
-   * a legacy from the centralized commit protocol handling by the JobTracker.
-   */
-  @Override
-  public void commitPending(TezTaskAttemptID taskAttemptId)
-      throws IOException, InterruptedException {
-    LOG.info("Commit-pending state update from " + taskAttemptId.toString());
-    // An attempt is asking if it can commit its output. This can be decided
-    // only by the task which is managing the multiple attempts. So redirect the
-    // request there.
-    taskHeartbeatHandler.progressing(taskAttemptId);
-    pingContainerHeartbeatHandler(taskAttemptId);
-    //Ignorable TaskStatus? - since a task will send a LastStatusUpdate
-    context.getEventHandler().handle(
-        new TaskAttemptEvent(
-            taskAttemptId,
-            TaskAttemptEventType.TA_COMMIT_PENDING)
-        );
-  }
-
-  /**
    * Child checking whether it can commit.
    *
    * <br/>

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/6ca59ac7/tez-dag/src/main/java/org/apache/tez/dag/app/dag/TaskAttempt.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/TaskAttempt.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/TaskAttempt.java
index ae70022..0cc9163 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/TaskAttempt.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/TaskAttempt.java
@@ -70,6 +70,7 @@ public interface TaskAttempt {
   TezCounters getCounters();
   float getProgress();
   TaskAttemptState getState();
+  TaskAttemptState getStateNoLock();
 
   /** 
    * Has attempt reached the final state or not.

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/6ca59ac7/tez-dag/src/main/java/org/apache/tez/dag/app/dag/TaskAttemptStateInternal.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/TaskAttemptStateInternal.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/TaskAttemptStateInternal.java
index 9ad5460..a49c2a3 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/TaskAttemptStateInternal.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/TaskAttemptStateInternal.java
@@ -30,7 +30,6 @@ public enum TaskAttemptStateInternal {
   START_WAIT,
   RUNNING,
   OUTPUT_CONSUMABLE, 
-  COMMIT_PENDING,
   KILL_IN_PROGRESS, 
   FAIL_IN_PROGRESS,
   KILLED,

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/6ca59ac7/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskEventType.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskEventType.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskEventType.java
index d0ad8a0..a0b99a9 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskEventType.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskEventType.java
@@ -38,7 +38,6 @@ public enum TaskEventType {
   //Producer:TaskAttempt
   T_ATTEMPT_LAUNCHED,
   T_ATTEMPT_OUTPUT_CONSUMABLE,
-  T_ATTEMPT_COMMIT_PENDING,
   T_ATTEMPT_FAILED,
   T_ATTEMPT_SUCCEEDED,
   T_ATTEMPT_KILLED

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/6ca59ac7/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 f171afe..1ae9dcd 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
@@ -176,7 +176,6 @@ public class TaskAttemptImpl implements TaskAttempt,
         .addTransition(TaskAttemptStateInternal.RUNNING, TaskAttemptStateInternal.RUNNING, TaskAttemptEventType.TA_STATUS_UPDATE, STATUS_UPDATER)
         .addTransition(TaskAttemptStateInternal.RUNNING, TaskAttemptStateInternal.RUNNING, TaskAttemptEventType.TA_DIAGNOSTICS_UPDATE, DIAGNOSTIC_INFORMATION_UPDATE_TRANSITION)
         .addTransition(TaskAttemptStateInternal.RUNNING, TaskAttemptStateInternal.OUTPUT_CONSUMABLE, TaskAttemptEventType.TA_OUTPUT_CONSUMABLE, new OutputConsumableTransition()) //Optional, may not come in for all tasks.
-        .addTransition(TaskAttemptStateInternal.RUNNING, TaskAttemptStateInternal.COMMIT_PENDING, TaskAttemptEventType.TA_COMMIT_PENDING, new CommitPendingTransition())
         .addTransition(TaskAttemptStateInternal.RUNNING, TaskAttemptStateInternal.SUCCEEDED, TaskAttemptEventType.TA_DONE, new SucceededTransition())
         .addTransition(TaskAttemptStateInternal.RUNNING, TaskAttemptStateInternal.FAIL_IN_PROGRESS, TaskAttemptEventType.TA_FAILED, new TerminatedWhileRunningTransition(FAILED_HELPER))
         .addTransition(TaskAttemptStateInternal.RUNNING, TaskAttemptStateInternal.FAIL_IN_PROGRESS, TaskAttemptEventType.TA_TIMED_OUT, new TerminatedWhileRunningTransition(FAILED_HELPER))
@@ -185,11 +184,11 @@ public class TaskAttemptImpl implements TaskAttempt,
         .addTransition(TaskAttemptStateInternal.RUNNING, TaskAttemptStateInternal.KILL_IN_PROGRESS, TaskAttemptEventType.TA_NODE_FAILED, new TerminatedWhileRunningTransition(KILLED_HELPER))
         .addTransition(TaskAttemptStateInternal.RUNNING, TaskAttemptStateInternal.FAIL_IN_PROGRESS, TaskAttemptEventType.TA_CONTAINER_TERMINATING, new TerminatedWhileRunningTransition(FAILED_HELPER))
         .addTransition(TaskAttemptStateInternal.RUNNING, TaskAttemptStateInternal.FAILED, TaskAttemptEventType.TA_CONTAINER_TERMINATED, new ContainerCompletedWhileRunningTransition())
+        .addTransition(TaskAttemptStateInternal.RUNNING, TaskAttemptStateInternal.FAIL_IN_PROGRESS, TaskAttemptEventType.TA_TOO_MANY_FETCH_FAILURES, new TerminatedWhileRunningTransition(FAILED_HELPER))
 
         .addTransition(TaskAttemptStateInternal.OUTPUT_CONSUMABLE, TaskAttemptStateInternal.OUTPUT_CONSUMABLE, TaskAttemptEventType.TA_STATUS_UPDATE, STATUS_UPDATER)
         .addTransition(TaskAttemptStateInternal.OUTPUT_CONSUMABLE, TaskAttemptStateInternal.OUTPUT_CONSUMABLE, TaskAttemptEventType.TA_DIAGNOSTICS_UPDATE, DIAGNOSTIC_INFORMATION_UPDATE_TRANSITION)
         .addTransition(TaskAttemptStateInternal.OUTPUT_CONSUMABLE, TaskAttemptStateInternal.OUTPUT_CONSUMABLE, TaskAttemptEventType.TA_OUTPUT_CONSUMABLE) // Stuck RPC. The client retries in a loop.
-        .addTransition(TaskAttemptStateInternal.OUTPUT_CONSUMABLE, TaskAttemptStateInternal.COMMIT_PENDING, TaskAttemptEventType.TA_COMMIT_PENDING, new CommitPendingAtOutputConsumableTransition())
         .addTransition(TaskAttemptStateInternal.OUTPUT_CONSUMABLE, TaskAttemptStateInternal.SUCCEEDED, TaskAttemptEventType.TA_DONE, new SucceededTransition())
         .addTransition(TaskAttemptStateInternal.OUTPUT_CONSUMABLE, TaskAttemptStateInternal.FAIL_IN_PROGRESS, TaskAttemptEventType.TA_FAILED,  new TerminatedWhileRunningTransition(FAILED_HELPER))
         .addTransition(TaskAttemptStateInternal.OUTPUT_CONSUMABLE, TaskAttemptStateInternal.FAIL_IN_PROGRESS, TaskAttemptEventType.TA_TIMED_OUT,  new TerminatedWhileRunningTransition(FAILED_HELPER))
@@ -201,19 +200,6 @@ public class TaskAttemptImpl implements TaskAttempt,
         .addTransition(TaskAttemptStateInternal.OUTPUT_CONSUMABLE, TaskAttemptStateInternal.FAILED, TaskAttemptEventType.TA_CONTAINER_TERMINATED, new ContainerCompletedBeforeRunningTransition())
         .addTransition(TaskAttemptStateInternal.OUTPUT_CONSUMABLE, TaskAttemptStateInternal.FAIL_IN_PROGRESS, TaskAttemptEventType.TA_TOO_MANY_FETCH_FAILURES, new TerminatedWhileRunningTransition(FAILED_HELPER))
 
-        .addTransition(TaskAttemptStateInternal.COMMIT_PENDING, TaskAttemptStateInternal.COMMIT_PENDING, TaskAttemptEventType.TA_STATUS_UPDATE, STATUS_UPDATER)
-        .addTransition(TaskAttemptStateInternal.COMMIT_PENDING, TaskAttemptStateInternal.COMMIT_PENDING, TaskAttemptEventType.TA_DIAGNOSTICS_UPDATE, DIAGNOSTIC_INFORMATION_UPDATE_TRANSITION)
-        .addTransition(TaskAttemptStateInternal.COMMIT_PENDING, TaskAttemptStateInternal.COMMIT_PENDING, TaskAttemptEventType.TA_COMMIT_PENDING)
-        .addTransition(TaskAttemptStateInternal.COMMIT_PENDING, TaskAttemptStateInternal.SUCCEEDED, TaskAttemptEventType.TA_DONE, new SucceededTransition())
-        .addTransition(TaskAttemptStateInternal.COMMIT_PENDING, TaskAttemptStateInternal.FAIL_IN_PROGRESS, TaskAttemptEventType.TA_FAILED, new TerminatedWhileRunningTransition(FAILED_HELPER))
-        .addTransition(TaskAttemptStateInternal.COMMIT_PENDING, TaskAttemptStateInternal.FAIL_IN_PROGRESS, TaskAttemptEventType.TA_TIMED_OUT, new TerminatedWhileRunningTransition(FAILED_HELPER))
-        .addTransition(TaskAttemptStateInternal.COMMIT_PENDING, TaskAttemptStateInternal.FAIL_IN_PROGRESS, TaskAttemptEventType.TA_FAIL_REQUEST, new TerminatedWhileRunningTransition(FAILED_HELPER))
-        .addTransition(TaskAttemptStateInternal.COMMIT_PENDING, TaskAttemptStateInternal.KILL_IN_PROGRESS, TaskAttemptEventType.TA_KILL_REQUEST, new TerminatedWhileRunningTransition(KILLED_HELPER))
-        .addTransition(TaskAttemptStateInternal.COMMIT_PENDING, TaskAttemptStateInternal.KILL_IN_PROGRESS, TaskAttemptEventType.TA_NODE_FAILED, new TerminatedWhileRunningTransition(KILLED_HELPER))
-        .addTransition(TaskAttemptStateInternal.COMMIT_PENDING, TaskAttemptStateInternal.FAIL_IN_PROGRESS, TaskAttemptEventType.TA_CONTAINER_TERMINATING, new TerminatedWhileRunningTransition(FAILED_HELPER))
-        .addTransition(TaskAttemptStateInternal.COMMIT_PENDING, TaskAttemptStateInternal.FAILED, TaskAttemptEventType.TA_CONTAINER_TERMINATED, new ContainerCompletedBeforeRunningTransition())
-        .addTransition(TaskAttemptStateInternal.COMMIT_PENDING, TaskAttemptStateInternal.FAIL_IN_PROGRESS, TaskAttemptEventType.TA_TOO_MANY_FETCH_FAILURES, new TerminatedWhileRunningTransition(FAILED_HELPER))
-
         .addTransition(TaskAttemptStateInternal.KILL_IN_PROGRESS, TaskAttemptStateInternal.KILLED, TaskAttemptEventType.TA_CONTAINER_TERMINATED, new ContainerCompletedWhileTerminating())
         .addTransition(TaskAttemptStateInternal.KILL_IN_PROGRESS, TaskAttemptStateInternal.KILL_IN_PROGRESS, TaskAttemptEventType.TA_DIAGNOSTICS_UPDATE, DIAGNOSTIC_INFORMATION_UPDATE_TRANSITION)
         .addTransition(TaskAttemptStateInternal.KILL_IN_PROGRESS, TaskAttemptStateInternal.KILL_IN_PROGRESS, EnumSet.of(TaskAttemptEventType.TA_STARTED_REMOTELY, TaskAttemptEventType.TA_STATUS_UPDATE, TaskAttemptEventType.TA_OUTPUT_CONSUMABLE, TaskAttemptEventType.TA_COMMIT_PENDING, TaskAttemptEventType.TA_DONE, TaskAttemptEventType.TA_FAILED, TaskAttemptEventType.TA_TIMED_OUT, TaskAttemptEventType.TA_FAIL_REQUEST, TaskAttemptEventType.TA_KILL_REQUEST, TaskAttemptEventType.TA_NODE_FAILED, TaskAttemptEventType.TA_CONTAINER_TERMINATING, TaskAttemptEventType.TA_TOO_MANY_FETCH_FAILURES))
@@ -369,13 +355,18 @@ public class TaskAttemptImpl implements TaskAttempt,
   public TaskAttemptState getState() {
     readLock.lock();
     try {
-      return getExternalState(stateMachine.getCurrentState());
+      return getStateNoLock();
     } finally {
       readLock.unlock();
     }
   }
 
   @Override
+  public TaskAttemptState getStateNoLock() {
+    return getExternalState(stateMachine.getCurrentState());
+  }
+
+  @Override
   public boolean isFinished() {
     readLock.lock();
     try {
@@ -530,10 +521,8 @@ public class TaskAttemptImpl implements TaskAttempt,
     case START_WAIT:
       return TaskAttemptState.STARTING;
     case RUNNING:
-      return TaskAttemptState.RUNNING;
-    case COMMIT_PENDING:
     case OUTPUT_CONSUMABLE:
-      return TaskAttemptState.COMMIT_PENDING;
+      return TaskAttemptState.RUNNING;
     case FAILED:
     case FAIL_IN_PROGRESS:
       return TaskAttemptState.FAILED;
@@ -1076,15 +1065,6 @@ public class TaskAttemptImpl implements TaskAttempt,
     }
   }
 
-  protected static class CommitPendingTransition implements
-      SingleArcTransition<TaskAttemptImpl, TaskAttemptEvent> {
-    @Override
-    public void transition(TaskAttemptImpl ta, TaskAttemptEvent event) {
-      ta.sendEvent(new TaskEventTAUpdate(ta.attemptId,
-          TaskEventType.T_ATTEMPT_COMMIT_PENDING));
-    }
-  }
-
   protected static class SucceededTransition implements
       SingleArcTransition<TaskAttemptImpl, TaskAttemptEvent> {
     @Override
@@ -1139,18 +1119,8 @@ public class TaskAttemptImpl implements TaskAttempt,
     public void transition(TaskAttemptImpl ta, TaskAttemptEvent event) {
       super.transition(ta, event);
       ta.sendTaskAttemptCleanupEvent();
-    }
-  }
-
-  protected static class CommitPendingAtOutputConsumableTransition extends
-      CommitPendingTransition {
-
-    @Override
-    public void transition(TaskAttemptImpl ta, TaskAttemptEvent event) {
-      // TODO Figure out the interaction between OUTPUT_CONSUMABLE AND
-      // COMMIT_PENDING, Ideally both should not exist for the same task.
-      super.transition(ta, event);
-      LOG.info("Received a commit pending while in the OutputConsumable state");
+      TaskAttemptEventContainerTerminated tEvent = (TaskAttemptEventContainerTerminated) event;
+      ta.addDiagnosticInfo(tEvent.getDiagnosticInfo());
     }
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/6ca59ac7/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 ff9ded7..92a1859 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
@@ -167,9 +167,6 @@ public class TaskImpl implements Task, EventHandler<TaskEvent> {
         TaskEventType.T_ATTEMPT_OUTPUT_CONSUMABLE,
         new AttemptProcessingCompleteTransition())
     .addTransition(TaskStateInternal.RUNNING, TaskStateInternal.RUNNING,
-        TaskEventType.T_ATTEMPT_COMMIT_PENDING,
-        new AttemptCommitPendingTransition())
-    .addTransition(TaskStateInternal.RUNNING, TaskStateInternal.RUNNING,
         TaskEventType.T_ADD_SPEC_ATTEMPT, new RedundantScheduleTransition())
     .addTransition(TaskStateInternal.RUNNING, TaskStateInternal.SUCCEEDED,
         TaskEventType.T_ATTEMPT_SUCCEEDED,
@@ -203,7 +200,6 @@ public class TaskImpl implements Task, EventHandler<TaskEvent> {
             TaskEventType.T_TERMINATE,
             TaskEventType.T_ATTEMPT_LAUNCHED,
             TaskEventType.T_ATTEMPT_OUTPUT_CONSUMABLE,
-            TaskEventType.T_ATTEMPT_COMMIT_PENDING,
             TaskEventType.T_ATTEMPT_FAILED,
             TaskEventType.T_ATTEMPT_SUCCEEDED,
             TaskEventType.T_ADD_SPEC_ATTEMPT))
@@ -584,17 +580,50 @@ public class TaskImpl implements Task, EventHandler<TaskEvent> {
 
   @Override
   public boolean canCommit(TezTaskAttemptID taskAttemptID) {
-    readLock.lock();
-    boolean canCommit = false;
+    writeLock.lock();
     try {
-      if (commitAttempt != null) {
-        canCommit = taskAttemptID.equals(commitAttempt);
-        LOG.info("Result of canCommit for " + taskAttemptID + ":" + canCommit);
+      if (getState() != TaskState.RUNNING) {
+        LOG.info("Task not running. Issuing kill to bad commit attempt " + taskAttemptID);
+        eventHandler.handle(new TaskAttemptEventKillRequest(taskAttemptID
+            , "Task not running. Bad attempt."));
+        return false;
       }
+      if (commitAttempt == null) {
+        TaskAttempt ta = getAttempt(taskAttemptID);
+        if (ta == null) {
+          throw new TezUncheckedException("Unknown task for commit: " + taskAttemptID);
+        }
+        // Its ok to get a non-locked state snapshot since we handle changes of 
+        // state in the task attempt. Dont want to deadlock here.
+        TaskAttemptState taState = ta.getStateNoLock();
+        if (taState == TaskAttemptState.RUNNING) {
+          commitAttempt = taskAttemptID;
+          LOG.info(taskAttemptID + " given a go for committing the task output.");
+          return true;
+        } else {
+          LOG.info(taskAttemptID + " with state: " + taState + 
+              " given a no-go for commit because its not running.");
+          return false;
+        }
+      } else {
+        if (commitAttempt.equals(taskAttemptID)) {
+          LOG.info(taskAttemptID + " given a go for committing the task output.");
+          return true;
+        }
+        // Don't think this can be a pluggable decision, so simply raise an
+        // event for the TaskAttempt to delete its output.
+        // Wait for commit attempt to succeed. Dont kill this. If commit
+        // attempt fails then choose a different committer. When commit attempt 
+        // succeeds then this and others will be killed
+        LOG.info(commitAttempt
+            + " is current committer. Commit waiting for:  "
+            + taskAttemptID);
+        return false;
+      }
+    
     } finally {
-      readLock.unlock();
+      writeLock.unlock();
     }
-    return canCommit;
   }
 
   // TODO remove hacky name lookup
@@ -887,50 +916,34 @@ public class TaskImpl implements Task, EventHandler<TaskEvent> {
     }
   }
 
-  private static class AttemptCommitPendingTransition implements
-      SingleArcTransition<TaskImpl, TaskEvent> {
-    @Override
-    public void transition(TaskImpl task, TaskEvent event) {
-      TaskEventTAUpdate ev = (TaskEventTAUpdate) event;
-      // The nextAttemptNumber is commit pending, decide on set the
-      // commitAttempt
-      TezTaskAttemptID attemptID = ev.getTaskAttemptID();
-      if (task.commitAttempt == null) {
-        // TODO: validate attemptID
-        task.commitAttempt = attemptID;
-        LOG.info(attemptID + " given a go for committing the task output.");
-      } else {
-        // Don't think this can be a pluggable decision, so simply raise an
-        // event for the TaskAttempt to delete its output.
-        // TODO . Wait for commit attempt to succeed. Dont kill this. If commit
-        // attempt fails then choose a different committer.
-        LOG.info(task.commitAttempt
-            + " already given a go for committing the task output, so killing "
-            + attemptID);
-        task.eventHandler.handle(new TaskAttemptEventKillRequest(attemptID,
-            "Output being committed by alternate attemptId."));
-      }
-    }
-  }
-
   private static class AttemptSucceededTransition
       implements SingleArcTransition<TaskImpl, TaskEvent> {
     @Override
     public void transition(TaskImpl task, TaskEvent event) {
-      task.handleTaskAttemptCompletion(
-          ((TaskEventTAUpdate) event).getTaskAttemptID(),
+      TezTaskAttemptID successTaId = ((TaskEventTAUpdate) event).getTaskAttemptID();
+      
+      if (task.commitAttempt != null && 
+          !task.commitAttempt.equals(successTaId)) {
+        // The succeeded attempt is not the one that was selected to commit
+        // This is impossible and has to be a bug
+        throw new TezUncheckedException("TA: " + successTaId 
+            + " succeeded but TA: " + task.commitAttempt 
+            + " was expected to commit and succeed");
+      }
+      
+      task.handleTaskAttemptCompletion(successTaId, 
           TezDependentTaskCompletionEvent.Status.SUCCEEDED);
       task.finishedAttempts++;
       --task.numberUncompletedAttempts;
-      task.successfulAttempt = ((TaskEventTAUpdate) event).getTaskAttemptID();
+      task.successfulAttempt = successTaId;
       task.eventHandler.handle(new VertexEventTaskCompleted(
           task.taskId, TaskState.SUCCEEDED));
       LOG.info("Task succeeded with attempt " + task.successfulAttempt);
-      // issue kill to all other attempts
       if (task.historyTaskStartGenerated) {
         task.logJobHistoryTaskFinishedEvent();
       }
 
+      // issue kill to all other attempts
       for (TaskAttempt attempt : task.attempts.values()) {
         if (attempt.getID() != task.successfulAttempt &&
             // This is okay because it can only talk us out of sending a
@@ -954,12 +967,18 @@ public class TaskImpl implements Task, EventHandler<TaskEvent> {
       SingleArcTransition<TaskImpl, TaskEvent> {
     @Override
     public void transition(TaskImpl task, TaskEvent event) {
+      TaskEventTAUpdate castEvent = (TaskEventTAUpdate) event;
+      if (task.commitAttempt !=null && 
+          castEvent.getTaskAttemptID().equals(task.commitAttempt)) {
+        task.commitAttempt = null;
+      }
       task.handleTaskAttemptCompletion(
-          ((TaskEventTAUpdate) event).getTaskAttemptID(),
+          castEvent.getTaskAttemptID(),
           TezDependentTaskCompletionEvent.Status.KILLED);
       task.finishedAttempts++;
-      --task.numberUncompletedAttempts;
-      if (task.successfulAttempt == null) {
+      // we don't need a new event if we already have a spare
+      if (--task.numberUncompletedAttempts == 0
+          && task.successfulAttempt == null) {
         task.addAndScheduleAttempt();
       }
     }
@@ -1001,7 +1020,8 @@ public class TaskImpl implements Task, EventHandler<TaskEvent> {
     public TaskStateInternal transition(TaskImpl task, TaskEvent event) {
       task.failedAttempts++;
       TaskEventTAUpdate castEvent = (TaskEventTAUpdate) event;
-      if (castEvent.getTaskAttemptID().equals(task.commitAttempt)) {
+      if (task.commitAttempt != null && 
+          castEvent.getTaskAttemptID().equals(task.commitAttempt)) {
         task.commitAttempt = null;
       }
       if (castEvent.getTaskAttemptID().equals(task.outputConsumableAttempt)) {
@@ -1143,6 +1163,10 @@ public class TaskImpl implements Task, EventHandler<TaskEvent> {
   }
 
   private void killUnfinishedAttempt(TaskAttempt attempt, String logMsg) {
+    if (commitAttempt != null && commitAttempt.equals(attempt)) {
+      LOG.info("Removing commit attempt: " + commitAttempt);
+      commitAttempt = null;
+    }
     if (attempt != null && !attempt.isFinished()) {
       eventHandler.handle(new TaskAttemptEventKillRequest(attempt.getID(),
           logMsg));

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/6ca59ac7/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 daac3c7..b5e283b 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
@@ -493,68 +493,6 @@ public class TestTaskAttempt {
   }
 
   @Test
-  // Ensure ContainerTerminated is handled correctly by the TaskAttempt
-  public void testContainerTerminatedWhileCommitting() throws Exception {
-    ApplicationId appId = ApplicationId.newInstance(1, 2);
-    ApplicationAttemptId appAttemptId = ApplicationAttemptId.newInstance(
-        appId, 0);
-    TezDAGID dagID = new TezDAGID(appId, 1);
-    TezVertexID vertexID = new TezVertexID(dagID, 1);
-    TezTaskID taskID = new TezTaskID(vertexID, 1);
-    TezTaskAttemptID taskAttemptID = new TezTaskAttemptID(taskID, 0);
-
-    MockEventHandler eventHandler = new MockEventHandler();
-    TaskAttemptListener taListener = mock(TaskAttemptListener.class);
-    when(taListener.getAddress()).thenReturn(
-        new InetSocketAddress("localhost", 0));
-
-    Configuration taskConf = new Configuration();
-    taskConf.setClass("fs.file.impl", StubbedFS.class, FileSystem.class);
-    taskConf.setBoolean("fs.file.impl.disable.cache", true);
-
-    TaskLocationHint locationHint = new TaskLocationHint(
-        new HashSet<String>(Arrays.asList(new String[] {"127.0.0.1"})), null);
-    Resource resource = Resource.newInstance(1024, 1);
-
-    NodeId nid = NodeId.newInstance("127.0.0.1", 0);
-    ContainerId contId = ContainerId.newInstance(appAttemptId, 3);
-    Container container = mock(Container.class);
-    when(container.getId()).thenReturn(contId);
-    when(container.getNodeId()).thenReturn(nid);
-    when(container.getNodeHttpAddress()).thenReturn("localhost:0");
-
-    AppContext appCtx = mock(AppContext.class);
-    AMContainerMap containers = new AMContainerMap(
-        mock(ContainerHeartbeatHandler.class), mock(TaskAttemptListener.class),
-        appCtx);
-    containers.addContainerIfNew(container);
-
-    doReturn(new ClusterInfo()).when(appCtx).getClusterInfo();
-    doReturn(containers).when(appCtx).getAllContainers();
-
-    TaskAttemptImpl taImpl = new MockTaskAttemptImpl(taskID, 1, eventHandler,
-        taListener, taskConf, new SystemClock(),
-        mock(TaskHeartbeatHandler.class), appCtx, locationHint, false,
-        resource, createFakeContainerContext());
-
-    taImpl.handle(new TaskAttemptEventSchedule(taskAttemptID, null));
-    // At state STARTING.
-    taImpl.handle(new TaskAttemptEventStartedRemotely(taskAttemptID, contId,
-        null));
-    assertEquals("Task attempt is not in running state", taImpl.getState(),
-        TaskAttemptState.RUNNING);
-    taImpl.handle(new TaskAttemptEvent(taskAttemptID,
-        TaskAttemptEventType.TA_COMMIT_PENDING));
-    assertEquals("Task attempt is not in commit pending state",
-        taImpl.getState(), TaskAttemptState.COMMIT_PENDING);
-    taImpl.handle(new TaskAttemptEventContainerTerminated(taskAttemptID, null));
-    assertFalse(
-        "InternalError occurred trying to handle TA_CONTAINER_TERMINATED",
-        eventHandler.internalError);
-    // TODO Verify diagnostics
-  }
-
-  @Test
   // Ensure ContainerTerminating and ContainerTerminated is handled correctly by
   // the TaskAttempt
   public void testContainerTerminatedAfterSuccess() throws Exception {

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/6ca59ac7/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 ad36b7b..be3915d 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
@@ -146,12 +146,6 @@ public class TestTaskImpl {
     assertTaskRunningState();
   }
 
-  private void commitTaskAttempt(TezTaskAttemptID attemptId) {
-    mockTask.handle(new TaskEventTAUpdate(attemptId,
-        TaskEventType.T_ATTEMPT_COMMIT_PENDING));
-    assertTaskRunningState();
-  }
-
   private void updateAttemptProgress(MockTaskAttemptImpl attempt, float p) {
     attempt.setProgress(p);
   }
@@ -315,9 +309,9 @@ public class TestTaskImpl {
     TezTaskID taskId = getNewTaskID();
     scheduleTaskAttempt(taskId);
     launchTaskAttempt(mockTask.getLastAttempt().getID());
-    updateAttemptState(mockTask.getLastAttempt(),
-        TaskAttemptState.COMMIT_PENDING);
-    commitTaskAttempt(mockTask.getLastAttempt().getID());
+    updateAttemptState(mockTask.getLastAttempt(), TaskAttemptState.RUNNING);
+    assertTrue("First attempt should commit",
+        mockTask.canCommit(mockTask.getLastAttempt().getID()));
 
     // During the task attempt commit there is an exception which causes
     // the attempt to fail
@@ -325,15 +319,54 @@ public class TestTaskImpl {
     failRunningTaskAttempt(mockTask.getLastAttempt().getID());
 
     assertEquals(2, mockTask.getAttemptList().size());
+    
+    assertFalse("First attempt should not commit",
+        mockTask.canCommit(mockTask.getAttemptList().get(0).getID()));
+    updateAttemptState(mockTask.getLastAttempt(), TaskAttemptState.RUNNING);
+    assertTrue("Second attempt should commit",
+        mockTask.canCommit(mockTask.getLastAttempt().getID()));
+
     updateAttemptState(mockTask.getLastAttempt(), TaskAttemptState.SUCCEEDED);
-    commitTaskAttempt(mockTask.getLastAttempt().getID());
     mockTask.handle(new TaskEventTAUpdate(mockTask.getLastAttempt().getID(),
         TaskEventType.T_ATTEMPT_SUCCEEDED));
 
+    assertTaskSucceededState();
+  }
+
+
+  @Test
+  public void testChangeCommitTaskAttempt() {
+    TezTaskID taskId = getNewTaskID();
+    scheduleTaskAttempt(taskId);
+    launchTaskAttempt(mockTask.getLastAttempt().getID());
+    updateAttemptState(mockTask.getLastAttempt(), TaskAttemptState.RUNNING);
+    
+    // Add a speculative task attempt that succeeds
+    mockTask.handle(new TaskEventTAUpdate(mockTask.getLastAttempt().getID(),
+        TaskEventType.T_ADD_SPEC_ATTEMPT));
+    launchTaskAttempt(mockTask.getLastAttempt().getID());
+    updateAttemptState(mockTask.getLastAttempt(), TaskAttemptState.RUNNING);
+    
+    assertTrue("Second attempt should commit",
+        mockTask.canCommit(mockTask.getAttemptList().get(1).getID()));
     assertFalse("First attempt should not commit",
         mockTask.canCommit(mockTask.getAttemptList().get(0).getID()));
-    assertTrue("Second attempt should commit",
-        mockTask.canCommit(mockTask.getLastAttempt().getID()));
+
+    // During the task attempt commit there is an exception which causes
+    // the second attempt to fail
+    updateAttemptState(mockTask.getLastAttempt(), TaskAttemptState.FAILED);
+    failRunningTaskAttempt(mockTask.getLastAttempt().getID());
+
+    assertEquals(2, mockTask.getAttemptList().size());
+    
+    assertFalse("Second attempt should not commit",
+        mockTask.canCommit(mockTask.getAttemptList().get(1).getID()));
+    assertTrue("First attempt should commit",
+        mockTask.canCommit(mockTask.getAttemptList().get(0).getID()));
+
+    updateAttemptState(mockTask.getAttemptList().get(0), TaskAttemptState.SUCCEEDED);
+    mockTask.handle(new TaskEventTAUpdate(mockTask.getAttemptList().get(0).getID(),
+        TaskEventType.T_ATTEMPT_SUCCEEDED));
 
     assertTaskSucceededState();
   }
@@ -349,7 +382,6 @@ public class TestTaskImpl {
     mockTask.handle(new TaskEventTAUpdate(mockTask.getLastAttempt().getID(),
         TaskEventType.T_ADD_SPEC_ATTEMPT));
     launchTaskAttempt(mockTask.getLastAttempt().getID());
-    commitTaskAttempt(mockTask.getLastAttempt().getID());
     mockTask.handle(new TaskEventTAUpdate(mockTask.getLastAttempt().getID(),
         TaskEventType.T_ATTEMPT_SUCCEEDED));
 
@@ -456,6 +488,11 @@ public class TestTaskImpl {
     public TaskAttemptState getState() {
       return state;
     }
+    
+    @Override
+    public TaskAttemptState getStateNoLock() {
+      return state;
+    }
   }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/6ca59ac7/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/TezProcessorContext.java
----------------------------------------------------------------------
diff --git a/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/TezProcessorContext.java b/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/TezProcessorContext.java
index cbe6e34..5b44f23 100644
--- a/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/TezProcessorContext.java
+++ b/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/TezProcessorContext.java
@@ -38,11 +38,4 @@ public interface TezProcessorContext extends TezTaskContext {
    */
   public boolean canCommit() throws IOException;
 
-  /**
-   * Tell the AM that this processor has a pending commit
-   * @throws IOException
-   * @throws InterruptedException
-   */
-  public void commitPending() throws IOException, InterruptedException;
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/6ca59ac7/tez-engine/src/main/java/org/apache/tez/common/TezTaskUmbilicalProtocol.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/common/TezTaskUmbilicalProtocol.java b/tez-engine/src/main/java/org/apache/tez/common/TezTaskUmbilicalProtocol.java
index 0ff424c..28991a8 100644
--- a/tez-engine/src/main/java/org/apache/tez/common/TezTaskUmbilicalProtocol.java
+++ b/tez-engine/src/main/java/org/apache/tez/common/TezTaskUmbilicalProtocol.java
@@ -42,9 +42,6 @@ public interface TezTaskUmbilicalProtocol extends Master {
 
   ContainerTask getTask(ContainerContext containerContext) throws IOException;
 
-  void commitPending(TezTaskAttemptID taskId)
-      throws IOException, InterruptedException;
-
   boolean canCommit(TezTaskAttemptID taskid) throws IOException;
 
   // TODO TEZAM5 Can commitPending and outputReady be collapsed into a single

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/6ca59ac7/tez-engine/src/main/java/org/apache/tez/engine/newapi/impl/TezProcessorContextImpl.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/newapi/impl/TezProcessorContextImpl.java b/tez-engine/src/main/java/org/apache/tez/engine/newapi/impl/TezProcessorContextImpl.java
index fd3cdf0..d710f7a 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/newapi/impl/TezProcessorContextImpl.java
+++ b/tez-engine/src/main/java/org/apache/tez/engine/newapi/impl/TezProcessorContextImpl.java
@@ -83,9 +83,4 @@ public class TezProcessorContextImpl extends TezTaskContextImpl
     return tezUmbilical.canCommit(this.taskAttemptID);
   }
 
-  @Override
-  public void commitPending() throws IOException, InterruptedException {
-    tezUmbilical.commitPending(this.taskAttemptID);
-  }
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/6ca59ac7/tez-engine/src/main/java/org/apache/tez/engine/newapi/impl/TezUmbilical.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/newapi/impl/TezUmbilical.java b/tez-engine/src/main/java/org/apache/tez/engine/newapi/impl/TezUmbilical.java
index 43f5edc..5889622 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/newapi/impl/TezUmbilical.java
+++ b/tez-engine/src/main/java/org/apache/tez/engine/newapi/impl/TezUmbilical.java
@@ -33,7 +33,4 @@ public interface TezUmbilical {
 
   public boolean canCommit(TezTaskAttemptID taskAttemptID) throws IOException;
 
-  public void commitPending(TezTaskAttemptID taskAttemptID)
-      throws IOException, InterruptedException;
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/6ca59ac7/tez-mapreduce/src/main/java/org/apache/hadoop/mapred/LocalJobRunnerTez.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/hadoop/mapred/LocalJobRunnerTez.java b/tez-mapreduce/src/main/java/org/apache/hadoop/mapred/LocalJobRunnerTez.java
index 4ad1026..1362396 100644
--- a/tez-mapreduce/src/main/java/org/apache/hadoop/mapred/LocalJobRunnerTez.java
+++ b/tez-mapreduce/src/main/java/org/apache/hadoop/mapred/LocalJobRunnerTez.java
@@ -606,13 +606,6 @@ public class LocalJobRunnerTez implements ClientProtocol {
       return null;
     }
 
-    @Override
-    public void commitPending(TezTaskAttemptID taskId)
-        throws IOException, InterruptedException {
-      // TODO Auto-generated method stub
-      // TODO TODONEWTEZ
-    }
-
   }
 
   public LocalJobRunnerTez(Configuration conf) throws IOException {

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/6ca59ac7/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/newprocessor/MRTask.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/newprocessor/MRTask.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/newprocessor/MRTask.java
index 2db823d..d71dba0 100644
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/newprocessor/MRTask.java
+++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/newprocessor/MRTask.java
@@ -464,26 +464,6 @@ public abstract class MRTask {
     if (output instanceof SimpleOutput) {
       SimpleOutput sOut = (SimpleOutput)output;
       if (sOut.isCommitRequired()) {
-        processorContext.commitPending();
-        // TODO NEWTEZ TEZ-439
-  //      int retries = MAX_RETRIES;
-  //      setState(TezTaskStatus.State.COMMIT_PENDING);
-  //      //say the task tracker that task is commit pending
-  //      // TODO TEZAM2 - Why is the commitRequired check missing ?
-  //      while (true) {
-  //        try {
-  //          umbilical.commitPending(taskAttemptId, status);
-  //          break;
-  //        } catch (InterruptedException ie) {
-  //          // ignore
-  //        } catch (IOException ie) {
-  //          LOG.warn("Failure sending commit pending: " +
-  //              StringUtils.stringifyException(ie));
-  //          if (--retries == 0) {
-  //            System.exit(67);
-  //          }
-  //        }
-  //      }
         //wait for commit approval and commit
         // TODO EVENTUALLY - Commit is not required for map tasks.
         // skip a couple of RPCs before exiting.
@@ -517,14 +497,24 @@ public abstract class MRTask {
   }
 
   private void commit(SimpleOutput output) throws IOException {
-    while (!processorContext.canCommit()) {
+    int retries = 3;
+    while (true) {
       // This will loop till the AM asks for the task to be killed. As
       // against, the AM sending a signal to the task to kill itself
       // gracefully.
       try {
+        if (processorContext.canCommit()) {
+          break;
+        }
         Thread.sleep(1000);
       } catch(InterruptedException ie) {
         //ignore
+      } catch (IOException ie) {
+        LOG.warn("Failure sending canCommit: "
+            + StringUtils.stringifyException(ie));
+        if (--retries == 0) {
+          throw ie;
+        }
       }
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/6ca59ac7/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/MRTask.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/MRTask.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/MRTask.java
index a6b5470..0653cc8 100644
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/MRTask.java
+++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/MRTask.java
@@ -400,24 +400,6 @@ public abstract class MRTask extends RunningTaskContext {
     TezTaskUmbilicalProtocol umbilical = getUmbilical();
     // TODO TEZ Interaciton between Commit and OutputReady. Merge ?
     if (isCommitRequired()) {
-      int retries = MAX_RETRIES;
-      setState(TezTaskStatus.State.COMMIT_PENDING);
-      // say the task tracker that task is commit pending
-      // TODO TEZAM2 - Why is the commitRequired check missing ?
-      while (true) {
-        try {
-          umbilical.commitPending(taskAttemptId);
-          break;
-        } catch (InterruptedException ie) {
-          // ignore
-        } catch (IOException ie) {
-          LOG.warn("Failure sending commit pending: " +
-              StringUtils.stringifyException(ie));
-          if (--retries == 0) {
-            System.exit(67);
-          }
-        }
-      }
       //wait for commit approval and commit
       // TODO EVENTUALLY - Commit is not required for map tasks. skip a couple of RPCs before exiting.
       commit(umbilical, reporter, committer);

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/6ca59ac7/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/TestUmbilicalProtocol.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/TestUmbilicalProtocol.java b/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/TestUmbilicalProtocol.java
index 07f3a2c..e5cc902 100644
--- a/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/TestUmbilicalProtocol.java
+++ b/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/TestUmbilicalProtocol.java
@@ -79,13 +79,6 @@ public class TestUmbilicalProtocol implements TezTaskUmbilicalProtocol {
     return null;
   }
 
-
-  @Override
-  public void commitPending(TezTaskAttemptID taskId)
-      throws IOException, InterruptedException {
-    LOG.info("Got 'commit-pending' from " + taskId);
-  }
-
   @Override
   public boolean canCommit(TezTaskAttemptID taskid) throws IOException {
     LOG.info("Got 'can-commit' from " + taskid);


[11/50] [abbrv] Rename *.new* packages back to what they should be, remove dead code from the old packages - mapreduce module - tez-engine module (part of TEZ-398). (sseth)

Posted by ss...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/TestUmbilicalProtocol.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/TestUmbilicalProtocol.java b/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/TestUmbilicalProtocol.java
index e5cc902..404dd8c 100644
--- a/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/TestUmbilicalProtocol.java
+++ b/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/TestUmbilicalProtocol.java
@@ -28,8 +28,8 @@ import org.apache.tez.common.ContainerTask;
 import org.apache.tez.common.TezTaskUmbilicalProtocol;
 import org.apache.tez.common.records.ProceedToCompletionResponse;
 import org.apache.tez.dag.records.TezTaskAttemptID;
-import org.apache.tez.engine.newapi.impl.TezHeartbeatRequest;
-import org.apache.tez.engine.newapi.impl.TezHeartbeatResponse;
+import org.apache.tez.engine.api.impl.TezHeartbeatRequest;
+import org.apache.tez.engine.api.impl.TezHeartbeatResponse;
 import org.apache.tez.engine.records.OutputContext;
 import org.apache.tez.engine.records.TezTaskDependencyCompletionEventsUpdate;
 

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/processor/MapUtils.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/processor/MapUtils.java b/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/processor/MapUtils.java
index 2bc327c..85e6653 100644
--- a/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/processor/MapUtils.java
+++ b/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/processor/MapUtils.java
@@ -44,14 +44,13 @@ import org.apache.hadoop.mapreduce.split.JobSplit;
 import org.apache.hadoop.mapreduce.split.JobSplit.SplitMetaInfo;
 import org.apache.hadoop.mapreduce.split.SplitMetaInfoReaderTez;
 import org.apache.hadoop.util.DiskChecker.DiskErrorException;
-import org.apache.tez.common.InputSpec;
-import org.apache.tez.common.OutputSpec;
-import org.apache.tez.common.TezEngineTaskContext;
 import org.apache.tez.common.TezJobConfig;
-import org.apache.tez.common.TezTaskUmbilicalProtocol;
 import org.apache.tez.dag.api.ProcessorDescriptor;
-import org.apache.tez.engine.api.Task;
-import org.apache.tez.engine.runtime.RuntimeUtils;
+import org.apache.tez.engine.api.impl.InputSpec;
+import org.apache.tez.engine.api.impl.OutputSpec;
+import org.apache.tez.engine.api.impl.TaskSpec;
+import org.apache.tez.engine.api.impl.TezUmbilical;
+import org.apache.tez.engine.newruntime.LogicalIOProcessorRuntimeTask;
 import org.apache.tez.mapreduce.TezTestUtils;
 import org.apache.tez.mapreduce.hadoop.MRJobConfig;
 import org.apache.tez.mapreduce.processor.map.MapProcessor;
@@ -174,9 +173,9 @@ public class MapUtils {
     outMeta.close();
   }
 
-  public static Task runMapProcessor(FileSystem fs, Path workDir,
+  public static LogicalIOProcessorRuntimeTask runMapProcessor(FileSystem fs, Path workDir,
       JobConf jobConf, int mapId, Path mapInput,
-      TezTaskUmbilicalProtocol umbilical,
+      TezUmbilical umbilical,
       String vertexName, List<InputSpec> inputSpecs,
       List<OutputSpec> outputSpecs) throws Exception {
     jobConf.setInputFormat(SequenceFileInputFormat.class);
@@ -185,14 +184,24 @@ public class MapUtils {
     ProcessorDescriptor mapProcessorDesc = new ProcessorDescriptor(
         MapProcessor.class.getName());
     writeSplitFiles(fs, jobConf, split);
-    TezEngineTaskContext taskContext = new TezEngineTaskContext(
-        TezTestUtils.getMockTaskAttemptId(0, 0, mapId, 0), "testuser",
-        "testJob", vertexName, mapProcessorDesc,
-        inputSpecs, outputSpecs);
-
-    Task t = RuntimeUtils.createRuntimeTask(taskContext);
-    t.initialize(jobConf, null, umbilical);
-    t.getProcessor().process(t.getInputs(), t.getOutputs());
-    return t;
+
+    TaskSpec taskSpec = new TaskSpec(
+        TezTestUtils.getMockTaskAttemptId(0, 0, mapId, 0),
+        "testuser",
+        vertexName,
+        mapProcessorDesc,
+        inputSpecs,
+        outputSpecs);
+    
+    // TODO NEWTEZ Fix umbilical access
+    LogicalIOProcessorRuntimeTask task = new LogicalIOProcessorRuntimeTask(
+        taskSpec,
+        1,
+        jobConf,
+        umbilical,
+        null);
+    task.initialize();
+    task.run();
+    return task;
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/processor/map/TestMapProcessor.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/processor/map/TestMapProcessor.java b/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/processor/map/TestMapProcessor.java
index 84f1f81..2ecce8b 100644
--- a/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/processor/map/TestMapProcessor.java
+++ b/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/processor/map/TestMapProcessor.java
@@ -31,24 +31,24 @@ import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.tez.common.Constants;
-import org.apache.tez.common.InputSpec;
-import org.apache.tez.common.OutputSpec;
 import org.apache.tez.common.TezJobConfig;
-import org.apache.tez.engine.api.Task;
+import org.apache.tez.dag.api.InputDescriptor;
+import org.apache.tez.dag.api.OutputDescriptor;
+import org.apache.tez.engine.api.impl.InputSpec;
+import org.apache.tez.engine.api.impl.OutputSpec;
+import org.apache.tez.engine.api.impl.TezUmbilical;
+import org.apache.tez.engine.common.InputAttemptIdentifier;
 import org.apache.tez.engine.common.sort.impl.IFile;
 import org.apache.tez.engine.common.task.local.output.TezLocalTaskOutputFiles;
 import org.apache.tez.engine.common.task.local.output.TezTaskOutput;
-import org.apache.tez.engine.lib.oldoutput.OldLocalOnFileSorterOutput;
-import org.apache.tez.mapreduce.TestUmbilicalProtocol;
+import org.apache.tez.engine.lib.output.LocalOnFileSorterOutput;
+import org.apache.tez.engine.newruntime.LogicalIOProcessorRuntimeTask;
 import org.apache.tez.mapreduce.hadoop.MRJobConfig;
 import org.apache.tez.mapreduce.hadoop.MultiStageMRConfToTezTranslator;
 import org.apache.tez.mapreduce.hadoop.MultiStageMRConfigUtil;
-import org.apache.tez.mapreduce.hadoop.mapreduce.TezNullOutputCommitter;
 import org.apache.tez.mapreduce.input.SimpleInputLegacy;
-import org.apache.tez.mapreduce.processor.MRTask;
 import org.apache.tez.mapreduce.processor.MapUtils;
 import org.junit.After;
-import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 
@@ -75,7 +75,7 @@ public class TestMapProcessor {
   
 
 
-  TezTaskOutput mapOutputs = new TezLocalTaskOutputFiles();
+  
 
   public void setUpJobConf(JobConf job) {
     job.set(TezJobConfig.LOCAL_DIRS, workDir.toString());
@@ -97,8 +97,7 @@ public class TestMapProcessor {
     String vertexName = MultiStageMRConfigUtil.getInitialMapVertexName();
     JobConf jobConf = new JobConf(defaultConf);
     setUpJobConf(jobConf);
-    TezTaskOutput mapOutputs = new TezLocalTaskOutputFiles();
-    mapOutputs.setConf(jobConf);
+    TezTaskOutput mapOutputs = new TezLocalTaskOutputFiles(jobConf, "TODONEWTEZ_uniqueId");
 
     Configuration conf = MultiStageMRConfToTezTranslator.convertMRToLinearTez(jobConf);
     conf.setInt(MRJobConfig.APPLICATION_ATTEMPT_ID, 0);
@@ -111,20 +110,22 @@ public class TestMapProcessor {
     job.set(TezJobConfig.TASK_LOCAL_RESOURCE_DIR, new Path(workDir,
         "localized-resources").toUri().toString());
     
+    InputSpec mapInputSpec = new InputSpec("NullSrcVertex", new InputDescriptor(SimpleInputLegacy.class.getName()), 1);
+    OutputSpec mapOutputSpec = new OutputSpec("NullDestVertex", new OutputDescriptor(LocalOnFileSorterOutput.class.getName()), 1);
 
-    Task t = MapUtils.runMapProcessor(localFs, workDir, job, 0,
-        new Path(workDir, "map0"), new TestUmbilicalProtocol(), vertexName,
-        Collections.singletonList(new InputSpec("NullVertex", 0,
-            SimpleInputLegacy.class.getName())),
-        Collections.singletonList(new OutputSpec("FakeVertex", 1,
-            OldLocalOnFileSorterOutput.class.getName())));
+    // TODO NEWTEZ FIXME TezUmbilical handling
+    LogicalIOProcessorRuntimeTask t = MapUtils.runMapProcessor(localFs, workDir, job, 0,
+        new Path(workDir, "map0"), (TezUmbilical) null, vertexName,
+        Collections.singletonList(mapInputSpec),
+        Collections.singletonList(mapOutputSpec));
 
-    MRTask mrTask = (MRTask)t.getProcessor();
-    Assert.assertEquals(TezNullOutputCommitter.class.getName(), mrTask
-        .getCommitter().getClass().getName());
-    t.close();
+    // TODO NEWTEZ FIXME OutputCommitter verification
+//    MRTask mrTask = (MRTask)t.getProcessor();
+//    Assert.assertEquals(TezNullOutputCommitter.class.getName(), mrTask
+//        .getCommitter().getClass().getName());
+//    t.close();
 
-    Path mapOutputFile = mapOutputs.getInputFile(0);
+    Path mapOutputFile = mapOutputs.getInputFile(new InputAttemptIdentifier(0, 0));
     LOG.info("mapOutputFile = " + mapOutputFile);
     IFile.Reader reader =
         new IFile.Reader(job, localFs, mapOutputFile, null, null);

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/processor/reduce/TestReduceProcessor.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/processor/reduce/TestReduceProcessor.java b/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/processor/reduce/TestReduceProcessor.java
index 8bcd353..1d35f9b 100644
--- a/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/processor/reduce/TestReduceProcessor.java
+++ b/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/processor/reduce/TestReduceProcessor.java
@@ -32,18 +32,19 @@ import org.apache.hadoop.mapred.FileOutputFormat;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.SequenceFileOutputFormat;
 import org.apache.tez.common.Constants;
-import org.apache.tez.common.InputSpec;
-import org.apache.tez.common.OutputSpec;
-import org.apache.tez.common.TezEngineTaskContext;
 import org.apache.tez.common.TezJobConfig;
+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.engine.api.Task;
+import org.apache.tez.engine.api.impl.InputSpec;
+import org.apache.tez.engine.api.impl.OutputSpec;
+import org.apache.tez.engine.api.impl.TaskSpec;
+import org.apache.tez.engine.api.impl.TezUmbilical;
 import org.apache.tez.engine.common.task.local.output.TezLocalTaskOutputFiles;
 import org.apache.tez.engine.common.task.local.output.TezTaskOutput;
-import org.apache.tez.engine.lib.oldinput.LocalMergedInput;
-import org.apache.tez.engine.lib.oldoutput.OldLocalOnFileSorterOutput;
-import org.apache.tez.engine.runtime.RuntimeUtils;
-import org.apache.tez.mapreduce.TestUmbilicalProtocol;
+import org.apache.tez.engine.lib.input.LocalMergedInput;
+import org.apache.tez.engine.lib.output.LocalOnFileSorterOutput;
+import org.apache.tez.engine.newruntime.LogicalIOProcessorRuntimeTask;
 import org.apache.tez.mapreduce.TezTestUtils;
 import org.apache.tez.mapreduce.hadoop.IDConverter;
 import org.apache.tez.mapreduce.hadoop.MRJobConfig;
@@ -51,7 +52,6 @@ import org.apache.tez.mapreduce.hadoop.MultiStageMRConfToTezTranslator;
 import org.apache.tez.mapreduce.hadoop.MultiStageMRConfigUtil;
 import org.apache.tez.mapreduce.input.SimpleInputLegacy;
 import org.apache.tez.mapreduce.output.SimpleOutput;
-import org.apache.tez.mapreduce.processor.MRTask;
 import org.apache.tez.mapreduce.processor.MapUtils;
 import org.apache.tez.mapreduce.processor.reduce.ReduceProcessor;
 import org.junit.After;
@@ -104,8 +104,7 @@ public class TestReduceProcessor {
     String reduceVertexName = MultiStageMRConfigUtil.getFinalReduceVertexName();
     JobConf jobConf = new JobConf(defaultConf);
     setUpJobConf(jobConf);
-    TezTaskOutput mapOutputs = new TezLocalTaskOutputFiles();
-    mapOutputs.setConf(jobConf);
+    TezTaskOutput mapOutputs = new TezLocalTaskOutputFiles(jobConf, "TODONEWTEZ_uniqueId");
     
     Configuration conf = MultiStageMRConfToTezTranslator.convertMRToLinearTez(jobConf);
     conf.setInt(MRJobConfig.APPLICATION_ATTEMPT_ID, 0);
@@ -117,14 +116,16 @@ public class TestReduceProcessor {
     mapConf.set(TezJobConfig.TASK_LOCAL_RESOURCE_DIR, new Path(workDir,
         "localized-resources").toUri().toString());
     
-    
+    InputSpec mapInputSpec = new InputSpec("NullSrcVertex", new InputDescriptor(
+        SimpleInputLegacy.class.getName()), 0);
+    OutputSpec mapOutputSpec = new OutputSpec("NullDestVertex", new OutputDescriptor(
+        LocalOnFileSorterOutput.class.getName()), 1);
     // Run a map
+    // TODO NEWTEZ FIX Umbilical creation
     MapUtils.runMapProcessor(localFs, workDir, mapConf, 0,
-        new Path(workDir, "map0"), new TestUmbilicalProtocol(), mapVertexName,
-        Collections.singletonList(new InputSpec("NullVertex", 0,
-            SimpleInputLegacy.class.getName())),
-        Collections.singletonList(new OutputSpec("FakeVertex", 1,
-            OldLocalOnFileSorterOutput.class.getName())));
+        new Path(workDir, "map0"), (TezUmbilical) null, mapVertexName,
+        Collections.singletonList(mapInputSpec),
+        Collections.singletonList(mapOutputSpec));
 
     LOG.info("Starting reduce...");
     
@@ -138,28 +139,43 @@ public class TestReduceProcessor {
     FileOutputFormat.setOutputPath(reduceConf, new Path(workDir, "output"));
     ProcessorDescriptor reduceProcessorDesc = new ProcessorDescriptor(
         ReduceProcessor.class.getName());
+    
+    InputSpec reduceInputSpec = new InputSpec(mapVertexName, new InputDescriptor(LocalMergedInput.class.getName()), 1);
+    OutputSpec reduceOutputSpec = new OutputSpec("NullDestinationVertex", new OutputDescriptor(SimpleOutput.class.getName()), 1);
+    
     // Now run a reduce
-    TezEngineTaskContext taskContext = new TezEngineTaskContext(
-        TezTestUtils.getMockTaskAttemptId(0, 1, 0, 0), "testUser",
-        "testJob", reduceVertexName, reduceProcessorDesc,
-        Collections.singletonList(new InputSpec(mapVertexName, 1,
-            LocalMergedInput.class.getName())),
-        Collections.singletonList(new OutputSpec("", 1,
-                SimpleOutput.class.getName())));
+    TaskSpec taskSpec = new TaskSpec(
+        TezTestUtils.getMockTaskAttemptId(0, 1, 0, 0),
+        "testUser",
+        reduceVertexName,
+        reduceProcessorDesc,
+        Collections.singletonList(reduceInputSpec),
+        Collections.singletonList(reduceOutputSpec));
+    
+    // TODO NEWTEZ FIXME Umbilical and jobToken
+    LogicalIOProcessorRuntimeTask task = new LogicalIOProcessorRuntimeTask(
+        taskSpec,
+        1,
+        reduceConf,
+        (TezUmbilical) null,
+        null);
     
-    Task t = RuntimeUtils.createRuntimeTask(taskContext);
-    t.initialize(reduceConf, null, new TestUmbilicalProtocol());
-    t.run();
-    MRTask mrTask = (MRTask)t.getProcessor();
+    task.initialize();
+    task.run();
+    
+//    MRTask mrTask = (MRTask)t.getProcessor();
 //    TODO NEWTEZ Verify the partitioner has been created
 //    Assert.assertNull(mrTask.getPartitioner());
-    t.close();
+    task.close();
     
     // Can this be done via some utility class ? MapOutputFile derivative, or
     // instantiating the OutputCommitter
+    
+
+    // TODO NEWTEZ FIXME uniqueId generation and event generation (mockTaskId will not work here)
     Path reduceOutputDir = new Path(new Path(workDir, "output"),
         "_temporary/0/" + IDConverter
-            .toMRTaskId(taskContext.getTaskAttemptId().getTaskID()));
+            .toMRTaskId(TezTestUtils.getMockTaskId(0, 1, 0)));
     Path reduceOutputFile = new Path(reduceOutputDir, "part-00000");
 
     SequenceFile.Reader reader = new SequenceFile.Reader(localFs,


[46/50] [abbrv] Rename tez-engine-api to tez-runtime-api and tez-engine is split into 2: - tez-engine-library for user-visible Input/Output/Processor implementations - tez-engine-internals for framework internals

Posted by ss...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-api/src/main/java/org/apache/tez/runtime/api/LogicalInput.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/runtime/api/LogicalInput.java b/tez-api/src/main/java/org/apache/tez/runtime/api/LogicalInput.java
new file mode 100644
index 0000000..3382d12
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/runtime/api/LogicalInput.java
@@ -0,0 +1,37 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.runtime.api;
+
+/**
+ * An @link {@link Input} which handles all incoming physical connections on an
+ * edge. A {@link LogicalIOProcessor} sees a single Logical Input per incoming
+ * edge.
+ */
+public interface LogicalInput extends Input {
+
+  /**
+   * Sets the number of physical inputs that this <code>LogicalInput</code> will
+   * receive. This will be called by the Tez framework before initializing the
+   * <code>LogicalInput</code>
+   * 
+   * @param numInputs
+   *          the number of physical inputs.
+   */
+  public void setNumPhysicalInputs(int numInputs);
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-api/src/main/java/org/apache/tez/runtime/api/LogicalOutput.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/runtime/api/LogicalOutput.java b/tez-api/src/main/java/org/apache/tez/runtime/api/LogicalOutput.java
new file mode 100644
index 0000000..475eaef
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/runtime/api/LogicalOutput.java
@@ -0,0 +1,36 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.runtime.api;
+
+/**
+ * An @link {@link Output} which handles all outgoing physical connections on an
+ * edge. A {@link LogicalIOProcessor} sees a single Logical Output per outgoing
+ * edge.
+ */
+public interface LogicalOutput extends Output {
+  /**
+   * Sets the number of physical ouputs that this <code>LogicalOutput</code>
+   * will receive. This will be called by the Tez framework before initializing
+   * the <code>LogicalOutput</code>
+   * 
+   * @param numOutputs
+   *          the number of physical outputs
+   */
+  public void setNumPhysicalOutputs(int numOutputs);
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-api/src/main/java/org/apache/tez/runtime/api/Output.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/runtime/api/Output.java b/tez-api/src/main/java/org/apache/tez/runtime/api/Output.java
new file mode 100644
index 0000000..22f8d66
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/runtime/api/Output.java
@@ -0,0 +1,71 @@
+/**
+ * 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.api;
+
+import java.util.List;
+
+/**
+ * Represents an Output through which a TezProcessor writes information on an
+ * edge. </p>
+ *
+ * <code>Output</code> implementations must have a 0 argument public constructor
+ * for Tez to construct the <code>Output</code>. Tez will take care of
+ * initializing and closing the Input after a {@link Processor} completes. </p>
+ */
+public interface Output {
+
+  /**
+   * Initializes the <code>Output</code>
+   *
+   * @param outputContext
+   *          the {@link TezOutputContext}
+   * @return
+   * @throws Exception
+   *           if an error occurs
+   */
+  public List<Event> initialize(TezOutputContext outputContext)
+      throws Exception;
+
+  /**
+   * Gets an instance of the {@link Writer} in an <code>Output</code>
+   *
+   * @return
+   * @throws Exception
+   *           if an error occurs
+   */
+  public Writer getWriter() throws Exception;
+
+  /**
+   * Handles user and system generated {@link Events}s, which typically carry
+   * information such as a downstream vertex being ready to consume input.
+   *
+   * @param outputEvents
+   *          the list of {@link Event}s
+   */
+  public void handleEvents(List<Event> outputEvents);
+
+  /**
+   * Closes the <code>Output</code>
+   *
+   * @return
+   * @throws Exception
+   *           if an error occurs
+   */
+  public List<Event> close() throws Exception;
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-api/src/main/java/org/apache/tez/runtime/api/Processor.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/runtime/api/Processor.java b/tez-api/src/main/java/org/apache/tez/runtime/api/Processor.java
new file mode 100644
index 0000000..17c2d05
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/runtime/api/Processor.java
@@ -0,0 +1,55 @@
+/**
+ * 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.api;
+
+import java.io.IOException;
+import java.util.List;
+
+/**
+ * {@link Processor} represents the <em>Tez</em> entity responsible for
+ * consuming {@link Input} and producing {@link Output}.
+ */
+public interface Processor {
+
+  /**
+   * Initializes the <code>Processor</code>
+   *
+   * @param processorContext
+   * @throws IOException
+   *           if an error occurs
+   */
+  public void initialize(TezProcessorContext processorContext)
+      throws Exception;
+
+  /**
+   * Handles user and system generated {@link Events}s.
+   *
+   * @param processorEvents
+   *          the list of {@link Event}s
+   */
+  public void handleEvents(List<Event> processorEvents);
+
+  /**
+   * Closes the <code>Processor</code>
+   *
+   * @throws IOException
+   *           if an error occurs
+   */
+  public void close() throws Exception;
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-api/src/main/java/org/apache/tez/runtime/api/Reader.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/runtime/api/Reader.java b/tez-api/src/main/java/org/apache/tez/runtime/api/Reader.java
new file mode 100644
index 0000000..dd006bc
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/runtime/api/Reader.java
@@ -0,0 +1,26 @@
+/**
+ * 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.api;
+
+/**
+ * A <code>Reader</code> represents the data being read in an {@link Input}
+ */
+public interface Reader {
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-api/src/main/java/org/apache/tez/runtime/api/TezInputContext.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/runtime/api/TezInputContext.java b/tez-api/src/main/java/org/apache/tez/runtime/api/TezInputContext.java
new file mode 100644
index 0000000..b07e92c
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/runtime/api/TezInputContext.java
@@ -0,0 +1,32 @@
+/**
+ * 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.api;
+
+/**
+ * Context handle for the Input to initialize itself.
+ */
+public interface TezInputContext extends TezTaskContext {
+
+  /**
+   * Get the Vertex Name of the Source that generated data for this Input
+   * @return Name of the Source Vertex
+   */
+  public String getSourceVertexName();
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-api/src/main/java/org/apache/tez/runtime/api/TezOutputContext.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/runtime/api/TezOutputContext.java b/tez-api/src/main/java/org/apache/tez/runtime/api/TezOutputContext.java
new file mode 100644
index 0000000..fda30ca
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/runtime/api/TezOutputContext.java
@@ -0,0 +1,33 @@
+/**
+ * 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.api;
+
+/**
+ * Context handle for the Output to initialize itself.
+ */
+public interface TezOutputContext extends TezTaskContext {
+
+  /**
+   * Get the Vertex Name of the Destination that is the recipient of this
+   * Output's data
+   * @return Name of the Destination Vertex
+   */
+  public String getDestinationVertexName();
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-api/src/main/java/org/apache/tez/runtime/api/TezProcessorContext.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/runtime/api/TezProcessorContext.java b/tez-api/src/main/java/org/apache/tez/runtime/api/TezProcessorContext.java
new file mode 100644
index 0000000..001461b
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/runtime/api/TezProcessorContext.java
@@ -0,0 +1,41 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.runtime.api;
+
+import java.io.IOException;
+
+/**
+ * Context handle for the Processor to initialize itself.
+ */
+public interface TezProcessorContext extends TezTaskContext {
+
+  /**
+   * Set the overall progress of this Task Attempt
+   * @param progress Progress in the range from [0.0 - 1.0f]
+   */
+  public void setProgress(float progress);
+
+  /**
+   * Check whether this attempt can commit its output
+   * @return true if commit allowed
+   * @throws IOException
+   */
+  public boolean canCommit() throws IOException;
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-api/src/main/java/org/apache/tez/runtime/api/TezTaskContext.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/runtime/api/TezTaskContext.java b/tez-api/src/main/java/org/apache/tez/runtime/api/TezTaskContext.java
new file mode 100644
index 0000000..ef652b6
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/runtime/api/TezTaskContext.java
@@ -0,0 +1,130 @@
+/**
+ * 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.api;
+
+import java.nio.ByteBuffer;
+import java.util.List;
+
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.tez.common.counters.TezCounters;
+
+/**
+ * Base interface for Context classes used to initialize the Input, Output
+ * and Processor instances.
+ */
+public interface TezTaskContext {
+
+  // TODO NEWTEZ
+  // Scale the maximum events we fetch per RPC call to mitigate OOM issues
+  // on the ApplicationMaster when a thundering herd of reducers fetch events
+  // This should not be necessary after HADOOP-8942
+
+  /**
+   * Get the {@link ApplicationId} for the running app
+   * @return the {@link ApplicationId}
+   */
+  public ApplicationId getApplicationId();
+
+  /**
+   * Get the current DAG Attempt Number
+   * @return DAG Attempt Number
+   */
+  public int getDAGAttemptNumber();
+
+  /**
+   * Get the index of this Task
+   * @return Task Index
+   */
+  public int getTaskIndex();
+
+  /**
+   * Get the current Task Attempt Number
+   * @return Task Attempt Number
+   */
+  public int getTaskAttemptNumber();
+
+  /**
+   * Get the name of the DAG
+   * @return the DAG name
+   */
+  public String getDAGName();
+
+  /**
+   * Get the name of the Vertex in which the task is running
+   * @return Vertex Name
+   */
+  public String getTaskVertexName();
+
+  public TezCounters getCounters();
+
+  /**
+   * Send Events to the AM and/or dependent Vertices
+   * @param events Events to be sent
+   */
+  public void sendEvents(List<Event> events);
+
+  /**
+   * Get the User Payload for the Input/Output/Processor
+   * @return User Payload
+   */
+  public byte[] getUserPayload();
+
+  /**
+   * Get the work diectories for the Input/Output/Processor
+   * @return an array of work dirs
+   */
+  public String[] getWorkDirs();
+
+  /**
+   * Returns an identifier which is unique to the specific Input, Processor or
+   * Output
+   *
+   * @return
+   */
+  public String getUniqueIdentifier();
+
+  /**
+   * Report a fatal error to the framework. This will cause the entire task to
+   * fail and should not be used for reporting temporary or recoverable errors
+   *
+   * @param exception an exception representing the error
+   */
+  public void fatalError(Throwable exception, String message);
+
+  /**
+   * Returns meta-data for the specified service. As an example, when the MR
+   * ShuffleHandler is used - this would return the jobToken serialized as bytes
+   *
+   * @param serviceName
+   *          the name of the service for which meta-data is required
+   * @return a ByteBuffer representing the meta-data
+   */
+  public ByteBuffer getServiceConsumerMetaData(String serviceName);
+
+  /**
+   * Return Provider meta-data for the specified service As an example, when the
+   * MR ShuffleHandler is used - this would return the shuffle port serialized
+   * as bytes
+   *
+   * @param serviceName
+   *          the name of the service for which provider meta-data is required
+   * @return a ByteBuffer representing the meta-data
+   */
+  public ByteBuffer getServiceProviderMetaData(String serviceName);
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-api/src/main/java/org/apache/tez/runtime/api/Writer.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/runtime/api/Writer.java b/tez-api/src/main/java/org/apache/tez/runtime/api/Writer.java
new file mode 100644
index 0000000..9604e59
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/runtime/api/Writer.java
@@ -0,0 +1,26 @@
+/**
+ * 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.api;
+
+/**
+ * A <code>Writer</code> represents the data being written by an {@link Output}
+ */
+public interface Writer {
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-api/src/main/java/org/apache/tez/runtime/api/events/DataMovementEvent.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/runtime/api/events/DataMovementEvent.java b/tez-api/src/main/java/org/apache/tez/runtime/api/events/DataMovementEvent.java
new file mode 100644
index 0000000..b384676
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/runtime/api/events/DataMovementEvent.java
@@ -0,0 +1,109 @@
+/**
+ * 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.api.events;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.tez.runtime.api.Event;
+
+/**
+ * Event used by user code to send information between tasks. An output can
+ * generate an Event of this type to sending information regarding output data
+ * ( such as URI for file-based output data, port info in case of
+ * streaming-based data transfers ) to the Input on the destination vertex.
+ */
+public final class DataMovementEvent extends Event {
+
+  /**
+   * Index(i) of the i-th (physical) Input or Output that generated an Event.
+   * For a Processor-generated event, this is ignored.
+   */
+  private final int sourceIndex;
+
+  /**
+   * Index(i) of the i-th (physical) Input or Output that is meant to receive
+   * this Event. For a Processor event, this is ignored.
+   */
+  private int targetIndex;
+
+  /**
+   * User Payload for this Event
+   */
+  private final byte[] userPayload;
+
+  /**
+   * Version number to indicate what attempt generated this Event
+   */
+  private int version;
+
+  /**
+   * User Event constructor
+   * @param sourceIndex Index to identify the physical edge of the input/output
+   * that generated the event
+   * @param userPayload User Payload of the User Event
+   */
+  public DataMovementEvent(int sourceIndex,
+      byte[] userPayload) {
+    this.userPayload = userPayload;
+    this.sourceIndex = sourceIndex;
+  }
+
+  @Private
+  public DataMovementEvent(int sourceIndex,
+      int targetIndex,
+      byte[] userPayload) {
+    this.userPayload = userPayload;
+    this.sourceIndex = sourceIndex;
+    this.targetIndex = targetIndex;
+  }
+
+  /**
+   * Constructor for Processor-generated User Events
+   * @param userPayload
+   */
+  public DataMovementEvent(byte[] userPayload) {
+    this(-1, userPayload);
+  }
+
+  public byte[] getUserPayload() {
+    return userPayload;
+  }
+
+  public int getSourceIndex() {
+    return sourceIndex;
+  }
+
+  public int getTargetIndex() {
+    return targetIndex;
+  }
+
+  @Private
+  public void setTargetIndex(int targetIndex) {
+    this.targetIndex = targetIndex;
+  }
+
+  public int getVersion() {
+    return version;
+  }
+
+  @Private
+  public void setVersion(int version) {
+    this.version = version;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-api/src/main/java/org/apache/tez/runtime/api/events/InputFailedEvent.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/runtime/api/events/InputFailedEvent.java b/tez-api/src/main/java/org/apache/tez/runtime/api/events/InputFailedEvent.java
new file mode 100644
index 0000000..dedc8d9
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/runtime/api/events/InputFailedEvent.java
@@ -0,0 +1,89 @@
+/**
+ * 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.api.events;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.tez.runtime.api.Event;
+
+/**
+ * Event sent from the AM to an Input to indicate that one of it's sources has
+ * failed - effectively the input is no longer available from the particular
+ * source.
+ * Users are not expected to send this event.
+ */
+public class InputFailedEvent extends Event{
+
+  /**
+   * Index(i) of the i-th (physical) Input or Output that generated the data.
+   * For a Processor-generated event, this is ignored.
+   */
+  private final int sourceIndex;
+
+  /**
+   * Index(i) of the i-th (physical) Input or Output that is meant to receive
+   * this Event. For a Processor event, this is ignored.
+   */
+  private int targetIndex;
+
+  /**
+   * Version number to indicate what attempt generated this Event
+   */
+  private int version;
+
+  /**
+   * User Event constructor
+   * @param sourceIndex Index to identify the physical edge of the input/output
+   * that generated the event
+   * @param userPayload User Payload of the User Event
+   */
+  public InputFailedEvent(int sourceIndex) {
+    this.sourceIndex = sourceIndex;
+  }
+
+  @Private
+  public InputFailedEvent(int sourceIndex,
+      int targetIndex,
+      int version) {
+    this.sourceIndex = sourceIndex;
+    this.targetIndex = targetIndex;
+    this.version = version;
+  }
+
+  public int getSourceIndex() {
+    return sourceIndex;
+  }
+
+  public int getTargetIndex() {
+    return targetIndex;
+  }
+
+  @Private
+  public void setTargetIndex(int targetIndex) {
+    this.targetIndex = targetIndex;
+  }
+
+  public int getVersion() {
+    return version;
+  }
+
+  @Private
+  public void setVersion(int version) {
+    this.version = version;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-api/src/main/java/org/apache/tez/runtime/api/events/InputInformationEvent.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/runtime/api/events/InputInformationEvent.java b/tez-api/src/main/java/org/apache/tez/runtime/api/events/InputInformationEvent.java
new file mode 100644
index 0000000..0322b75
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/runtime/api/events/InputInformationEvent.java
@@ -0,0 +1,41 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.runtime.api.events;
+
+import org.apache.tez.runtime.api.Event;
+
+/**
+ * Event used to send user specific data from the user 
+ * code in the AM to the task input
+ */
+public class InputInformationEvent extends Event {
+
+  /**
+   * User Payload for this Event
+   */
+  private final byte[] userPayload;
+  public InputInformationEvent(byte[] userPayload) {
+    this.userPayload = userPayload;
+  }
+
+  public byte[] getUserPayload() {
+    return userPayload;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-api/src/main/java/org/apache/tez/runtime/api/events/InputReadErrorEvent.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/runtime/api/events/InputReadErrorEvent.java b/tez-api/src/main/java/org/apache/tez/runtime/api/events/InputReadErrorEvent.java
new file mode 100644
index 0000000..612a1e3
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/runtime/api/events/InputReadErrorEvent.java
@@ -0,0 +1,65 @@
+/**
+ * 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.api.events;
+
+import org.apache.tez.runtime.api.Event;
+
+/**
+ * Event generated by an Input to indicate error when trying to retrieve data.
+ * This is not necessarily a fatal event - it's an indication to the AM to retry
+ * source data generation.
+ */
+public final class InputReadErrorEvent extends Event {
+
+  /**
+   * Diagnostics/trace of the error that occurred on the Input's edge.
+   */
+  private final String diagnostics;
+
+  /**
+   * Index of the physical edge on which the error occurred.
+   */
+  private final int index;
+
+  /**
+   * Version of the data on which the error occurred.
+   */
+  private final int version;
+
+  public InputReadErrorEvent(String diagnostics, int index,
+      int version) {
+    super();
+    this.diagnostics = diagnostics;
+    this.index = index;
+    this.version = version;
+  }
+
+  public String getDiagnostics() {
+    return diagnostics;
+  }
+
+  public int getIndex() {
+    return index;
+  }
+
+  public int getVersion() {
+    return version;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-api/src/main/java/org/apache/tez/runtime/common/objectregistry/ObjectLifeCycle.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/runtime/common/objectregistry/ObjectLifeCycle.java b/tez-api/src/main/java/org/apache/tez/runtime/common/objectregistry/ObjectLifeCycle.java
new file mode 100644
index 0000000..9dae043
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/runtime/common/objectregistry/ObjectLifeCycle.java
@@ -0,0 +1,37 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.runtime.common.objectregistry;
+
+/**
+ * Defines the valid lifecycle and scope of Objects stored in ObjectRegistry.
+ * Objects are guaranteed to not be valid outside of their defined life-cycle
+ * period. Objects are not guaranteed to be retained through the defined period
+ * as they may be evicted for various reasons.
+ */
+public enum ObjectLifeCycle {
+  /** Objects are valid for the lifetime of the Tez JVM/Session
+   */
+  SESSION,
+  /** Objects are valid for the lifetime of the DAG.
+   */
+  DAG,
+  /** Objects are valid for the lifetime of the Vertex.
+   */
+  VERTEX,
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-api/src/main/java/org/apache/tez/runtime/common/objectregistry/ObjectRegistry.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/runtime/common/objectregistry/ObjectRegistry.java b/tez-api/src/main/java/org/apache/tez/runtime/common/objectregistry/ObjectRegistry.java
new file mode 100644
index 0000000..03f0424
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/runtime/common/objectregistry/ObjectRegistry.java
@@ -0,0 +1,56 @@
+/**
+ * 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.common.objectregistry;
+
+/**
+ * Preliminary version of a simple shared object cache to re-use
+ * objects across multiple tasks within the same container/JVM.
+ */
+public interface ObjectRegistry {
+
+  /**
+   * Insert or update object into the registry. This will remove an object
+   * associated with the same key with a different life-cycle as there is only
+   * one instance of an Object stored for a given key irrespective of the
+   * life-cycle attached to the Object.
+   * @param lifeCycle What life-cycle is the Object valid for
+   * @param key Key to identify the Object
+   * @param value Object to be inserted
+   * @return Previous Object associated with the key attached if present
+   * else null. Could return the same object if the object was associated with
+   * the same key for a different life-cycle.
+   */
+  public Object add(ObjectLifeCycle lifeCycle, String key, Object value);
+
+  /**
+   * Return the object associated with the provided key
+   * @param key Key to find object
+   * @return Object if found else null
+   */
+  public Object get(String key);
+
+  /**
+   * Delete the object associated with the provided key
+   * @param lifeCycle What life-cycle is the Object valid for
+   * @param key Key to find object
+   * @return True if an object was found and removed
+   */
+  public boolean delete(String key);
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-api/src/main/java/org/apache/tez/runtime/common/objectregistry/ObjectRegistryFactory.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/runtime/common/objectregistry/ObjectRegistryFactory.java b/tez-api/src/main/java/org/apache/tez/runtime/common/objectregistry/ObjectRegistryFactory.java
new file mode 100644
index 0000000..3e409cd
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/runtime/common/objectregistry/ObjectRegistryFactory.java
@@ -0,0 +1,32 @@
+/**
+ * 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.common.objectregistry;
+
+import com.google.inject.Inject;
+
+public class ObjectRegistryFactory {
+
+  @Inject
+  private static ObjectRegistry objectRegistry;
+
+  public static ObjectRegistry getObjectRegistry() {
+    return objectRegistry;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-api/src/main/proto/Events.proto
----------------------------------------------------------------------
diff --git a/tez-api/src/main/proto/Events.proto b/tez-api/src/main/proto/Events.proto
index 21cacf6..b91125d 100644
--- a/tez-api/src/main/proto/Events.proto
+++ b/tez-api/src/main/proto/Events.proto
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-option java_package = "org.apache.tez.engine.api.events";
+option java_package = "org.apache.tez.runtime.api.events";
 option java_outer_classname = "EventProtos";
 option java_generate_equals_and_hash = true;
 

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-dag/pom.xml
----------------------------------------------------------------------
diff --git a/tez-dag/pom.xml b/tez-dag/pom.xml
index bc6aeef..9bf3fe4 100644
--- a/tez-dag/pom.xml
+++ b/tez-dag/pom.xml
@@ -41,7 +41,7 @@
     </dependency>
     <dependency>
       <groupId>org.apache.tez</groupId>
-      <artifactId>tez-engine</artifactId>
+      <artifactId>tez-runtime-internals</artifactId>
     </dependency>
     <dependency>
       <groupId>org.apache.hadoop</groupId>

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-dag/src/main/java/org/apache/hadoop/mapred/YarnTezDagChild.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/hadoop/mapred/YarnTezDagChild.java b/tez-dag/src/main/java/org/apache/hadoop/mapred/YarnTezDagChild.java
index 1967462..4938d9e 100644
--- a/tez-dag/src/main/java/org/apache/hadoop/mapred/YarnTezDagChild.java
+++ b/tez-dag/src/main/java/org/apache/hadoop/mapred/YarnTezDagChild.java
@@ -67,26 +67,26 @@ import org.apache.tez.dag.api.TezConfiguration;
 import org.apache.tez.dag.api.TezException;
 import org.apache.tez.dag.records.TezTaskAttemptID;
 import org.apache.tez.dag.records.TezVertexID;
-import org.apache.tez.engine.api.events.TaskAttemptCompletedEvent;
-import org.apache.tez.engine.api.events.TaskAttemptFailedEvent;
-import org.apache.tez.engine.api.events.TaskStatusUpdateEvent;
-import org.apache.tez.engine.api.impl.EventMetaData;
-import org.apache.tez.engine.api.impl.InputSpec;
-import org.apache.tez.engine.api.impl.OutputSpec;
-import org.apache.tez.engine.api.impl.TaskSpec;
-import org.apache.tez.engine.api.impl.TezEvent;
-import org.apache.tez.engine.api.impl.TezHeartbeatRequest;
-import org.apache.tez.engine.api.impl.TezHeartbeatResponse;
-import org.apache.tez.engine.api.impl.TezUmbilical;
-import org.apache.tez.engine.api.impl.EventMetaData.EventProducerConsumerType;
-import org.apache.tez.engine.common.objectregistry.ObjectLifeCycle;
-import org.apache.tez.engine.common.objectregistry.ObjectRegistryImpl;
-import org.apache.tez.engine.common.objectregistry.ObjectRegistryModule;
-import org.apache.tez.engine.common.security.JobTokenIdentifier;
-import org.apache.tez.engine.common.security.TokenCache;
-import org.apache.tez.engine.newruntime.LogicalIOProcessorRuntimeTask;
 import org.apache.tez.mapreduce.input.MRInputLegacy;
 import org.apache.tez.mapreduce.output.MROutput;
+import org.apache.tez.runtime.LogicalIOProcessorRuntimeTask;
+import org.apache.tez.runtime.api.events.TaskAttemptCompletedEvent;
+import org.apache.tez.runtime.api.events.TaskAttemptFailedEvent;
+import org.apache.tez.runtime.api.events.TaskStatusUpdateEvent;
+import org.apache.tez.runtime.api.impl.EventMetaData;
+import org.apache.tez.runtime.api.impl.InputSpec;
+import org.apache.tez.runtime.api.impl.OutputSpec;
+import org.apache.tez.runtime.api.impl.TaskSpec;
+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.apache.tez.runtime.api.impl.TezUmbilical;
+import org.apache.tez.runtime.api.impl.EventMetaData.EventProducerConsumerType;
+import org.apache.tez.runtime.common.objectregistry.ObjectLifeCycle;
+import org.apache.tez.runtime.common.objectregistry.ObjectRegistryImpl;
+import org.apache.tez.runtime.common.objectregistry.ObjectRegistryModule;
+import org.apache.tez.runtime.library.common.security.JobTokenIdentifier;
+import org.apache.tez.runtime.library.common.security.TokenCache;
 
 import com.google.inject.Guice;
 import com.google.inject.Injector;

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/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 56c89b2..67fe763 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
@@ -108,7 +108,7 @@ import org.apache.tez.dag.history.HistoryEventHandler;
 import org.apache.tez.dag.history.avro.HistoryEventType;
 import org.apache.tez.dag.history.events.AMStartedEvent;
 import org.apache.tez.dag.records.TezDAGID;
-import org.apache.tez.engine.common.security.JobTokenSecretManager;
+import org.apache.tez.runtime.library.common.security.JobTokenSecretManager;
 
 /**
  * The Map-Reduce Application Master.

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java b/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java
index 7a143a5..582d274 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java
@@ -51,10 +51,10 @@ import org.apache.tez.dag.app.rm.container.AMContainerTask;
 import org.apache.tez.dag.app.security.authorize.MRAMPolicyProvider;
 import org.apache.tez.dag.records.TezTaskAttemptID;
 import org.apache.tez.dag.records.TezVertexID;
-import org.apache.tez.engine.api.impl.TezEvent;
-import org.apache.tez.engine.api.impl.TezHeartbeatRequest;
-import org.apache.tez.engine.api.impl.TezHeartbeatResponse;
-import org.apache.tez.engine.common.security.JobTokenSecretManager;
+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.apache.tez.runtime.library.common.security.JobTokenSecretManager;
 
 @SuppressWarnings("unchecked")
 public class TaskAttemptListenerImpTezDag extends AbstractService implements

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-dag/src/main/java/org/apache/tez/dag/app/dag/EdgeManager.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/EdgeManager.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/EdgeManager.java
index 2779faf..674d18e 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/EdgeManager.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/EdgeManager.java
@@ -20,9 +20,9 @@ package org.apache.tez.dag.app.dag;
 
 import java.util.List;
 
-import org.apache.tez.engine.api.events.DataMovementEvent;
-import org.apache.tez.engine.api.events.InputFailedEvent;
-import org.apache.tez.engine.api.events.InputReadErrorEvent;
+import org.apache.tez.runtime.api.events.DataMovementEvent;
+import org.apache.tez.runtime.api.events.InputFailedEvent;
+import org.apache.tez.runtime.api.events.InputReadErrorEvent;
 
 public abstract class EdgeManager {
   

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-dag/src/main/java/org/apache/tez/dag/app/dag/Task.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/Task.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/Task.java
index 088a195..293e4c5 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/Task.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/Task.java
@@ -26,7 +26,7 @@ import org.apache.tez.dag.api.oldrecords.TaskReport;
 import org.apache.tez.dag.api.oldrecords.TaskState;
 import org.apache.tez.dag.records.TezTaskAttemptID;
 import org.apache.tez.dag.records.TezTaskID;
-import org.apache.tez.engine.api.impl.TezEvent;
+import org.apache.tez.runtime.api.impl.TezEvent;
 
 /**
  * Read only view of Task.

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-dag/src/main/java/org/apache/tez/dag/app/dag/Vertex.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/Vertex.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/Vertex.java
index 42ff8de..76964a3 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/Vertex.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/Vertex.java
@@ -32,9 +32,9 @@ import org.apache.tez.dag.app.dag.impl.Edge;
 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.engine.api.impl.InputSpec;
-import org.apache.tez.engine.api.impl.OutputSpec;
-import org.apache.tez.engine.records.TezDependentTaskCompletionEvent;
+import org.apache.tez.runtime.api.impl.InputSpec;
+import org.apache.tez.runtime.api.impl.OutputSpec;
+import org.apache.tez.runtime.records.TezDependentTaskCompletionEvent;
 
 
 /**

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-dag/src/main/java/org/apache/tez/dag/app/dag/VertexScheduler.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/VertexScheduler.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/VertexScheduler.java
index 4c79712..4a1a7a6 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/VertexScheduler.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/VertexScheduler.java
@@ -20,7 +20,7 @@ package org.apache.tez.dag.app.dag;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.tez.dag.records.TezTaskAttemptID;
-import org.apache.tez.engine.records.TezDependentTaskCompletionEvent;
+import org.apache.tez.runtime.records.TezDependentTaskCompletionEvent;
 
 // Rename to VertexManager TEZ-364 and move to DAG API. Make abstract class.
 public interface VertexScheduler {

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskAttemptEventStatusUpdate.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskAttemptEventStatusUpdate.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskAttemptEventStatusUpdate.java
index 0b8db76..30aefde 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskAttemptEventStatusUpdate.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskAttemptEventStatusUpdate.java
@@ -25,7 +25,7 @@ import org.apache.tez.common.counters.DAGCounter;
 import org.apache.tez.common.counters.TezCounters;
 import org.apache.tez.dag.api.oldrecords.TaskAttemptState;
 import org.apache.tez.dag.records.TezTaskAttemptID;
-import org.apache.tez.engine.api.events.TaskStatusUpdateEvent;
+import org.apache.tez.runtime.api.events.TaskStatusUpdateEvent;
 
 public class TaskAttemptEventStatusUpdate extends TaskAttemptEvent {
   

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskEventAddTezEvent.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskEventAddTezEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskEventAddTezEvent.java
index 4154bd0..1fbe609 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskEventAddTezEvent.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskEventAddTezEvent.java
@@ -19,7 +19,7 @@
 package org.apache.tez.dag.app.dag.event;
 
 import org.apache.tez.dag.records.TezTaskID;
-import org.apache.tez.engine.api.impl.TezEvent;
+import org.apache.tez.runtime.api.impl.TezEvent;
 
 public class TaskEventAddTezEvent extends TaskEvent {
 

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexEventRouteEvent.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexEventRouteEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexEventRouteEvent.java
index 37478cb..a872ae2 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexEventRouteEvent.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexEventRouteEvent.java
@@ -21,7 +21,7 @@ package org.apache.tez.dag.app.dag.event;
 import java.util.List;
 
 import org.apache.tez.dag.records.TezVertexID;
-import org.apache.tez.engine.api.impl.TezEvent;
+import org.apache.tez.runtime.api.impl.TezEvent;
 
 public class VertexEventRouteEvent extends VertexEvent {
   

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexEventSourceTaskAttemptCompleted.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexEventSourceTaskAttemptCompleted.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexEventSourceTaskAttemptCompleted.java
index 27d9a02..388beba 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexEventSourceTaskAttemptCompleted.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexEventSourceTaskAttemptCompleted.java
@@ -19,7 +19,7 @@
 package org.apache.tez.dag.app.dag.event;
 
 import org.apache.tez.dag.records.TezVertexID;
-import org.apache.tez.engine.records.TezDependentTaskCompletionEvent;
+import org.apache.tez.runtime.records.TezDependentTaskCompletionEvent;
 
 public class VertexEventSourceTaskAttemptCompleted extends VertexEvent {
 

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexEventTaskAttemptCompleted.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexEventTaskAttemptCompleted.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexEventTaskAttemptCompleted.java
index ef6de96..c9d3f7d 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexEventTaskAttemptCompleted.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexEventTaskAttemptCompleted.java
@@ -18,7 +18,7 @@
 
 package org.apache.tez.dag.app.dag.event;
 
-import org.apache.tez.engine.records.TezDependentTaskCompletionEvent;
+import org.apache.tez.runtime.records.TezDependentTaskCompletionEvent;
 
 public class VertexEventTaskAttemptCompleted extends VertexEvent {
 

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/BroadcastEdgeManager.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/BroadcastEdgeManager.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/BroadcastEdgeManager.java
index b05a6f1..55a2c86 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/BroadcastEdgeManager.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/BroadcastEdgeManager.java
@@ -22,9 +22,9 @@ import java.util.List;
 
 import org.apache.tez.dag.app.dag.EdgeManager;
 import org.apache.tez.dag.app.dag.Vertex;
-import org.apache.tez.engine.api.events.DataMovementEvent;
-import org.apache.tez.engine.api.events.InputFailedEvent;
-import org.apache.tez.engine.api.events.InputReadErrorEvent;
+import org.apache.tez.runtime.api.events.DataMovementEvent;
+import org.apache.tez.runtime.api.events.InputFailedEvent;
+import org.apache.tez.runtime.api.events.InputReadErrorEvent;
 
 public class BroadcastEdgeManager extends EdgeManager {
 

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/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 d30d178..ee12221 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
@@ -88,10 +88,10 @@ import org.apache.tez.dag.history.events.DAGStartedEvent;
 import org.apache.tez.dag.records.TezDAGID;
 import org.apache.tez.dag.records.TezVertexID;
 import org.apache.tez.dag.utils.TezBuilderUtils;
-import org.apache.tez.engine.common.security.JobTokenIdentifier;
-import org.apache.tez.engine.common.security.JobTokenSecretManager;
-import org.apache.tez.engine.common.security.TokenCache;
 import org.apache.tez.mapreduce.hadoop.MRJobConfig;
+import org.apache.tez.runtime.library.common.security.JobTokenIdentifier;
+import org.apache.tez.runtime.library.common.security.JobTokenSecretManager;
+import org.apache.tez.runtime.library.common.security.TokenCache;
 
 import com.google.common.annotations.VisibleForTesting;
 

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/Edge.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/Edge.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/Edge.java
index 060a112..aaca662 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/Edge.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/Edge.java
@@ -31,14 +31,14 @@ import org.apache.tez.dag.app.dag.Vertex;
 import org.apache.tez.dag.app.dag.event.TaskEventAddTezEvent;
 import org.apache.tez.dag.records.TezTaskAttemptID;
 import org.apache.tez.dag.records.TezTaskID;
-import org.apache.tez.engine.api.events.DataMovementEvent;
-import org.apache.tez.engine.api.events.InputFailedEvent;
-import org.apache.tez.engine.api.events.InputReadErrorEvent;
-import org.apache.tez.engine.api.impl.EventMetaData;
-import org.apache.tez.engine.api.impl.InputSpec;
-import org.apache.tez.engine.api.impl.OutputSpec;
-import org.apache.tez.engine.api.impl.TezEvent;
-import org.apache.tez.engine.api.impl.EventMetaData.EventProducerConsumerType;
+import org.apache.tez.runtime.api.events.DataMovementEvent;
+import org.apache.tez.runtime.api.events.InputFailedEvent;
+import org.apache.tez.runtime.api.events.InputReadErrorEvent;
+import org.apache.tez.runtime.api.impl.EventMetaData;
+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 org.apache.tez.runtime.api.impl.EventMetaData.EventProducerConsumerType;
 
 public class Edge {
 

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/ImmediateStartVertexScheduler.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/ImmediateStartVertexScheduler.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/ImmediateStartVertexScheduler.java
index 8ee7e55..a4e5f3b 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/ImmediateStartVertexScheduler.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/ImmediateStartVertexScheduler.java
@@ -22,7 +22,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.tez.dag.app.dag.Vertex;
 import org.apache.tez.dag.app.dag.VertexScheduler;
 import org.apache.tez.dag.records.TezTaskAttemptID;
-import org.apache.tez.engine.records.TezDependentTaskCompletionEvent;
+import org.apache.tez.runtime.records.TezDependentTaskCompletionEvent;
 
 /**
  * Starts all tasks immediately on vertex start

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/OneToOneEdgeManager.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/OneToOneEdgeManager.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/OneToOneEdgeManager.java
index a916ad2..29abfac 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/OneToOneEdgeManager.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/OneToOneEdgeManager.java
@@ -22,9 +22,9 @@ import java.util.List;
 
 import org.apache.tez.dag.app.dag.EdgeManager;
 import org.apache.tez.dag.app.dag.Vertex;
-import org.apache.tez.engine.api.events.DataMovementEvent;
-import org.apache.tez.engine.api.events.InputFailedEvent;
-import org.apache.tez.engine.api.events.InputReadErrorEvent;
+import org.apache.tez.runtime.api.events.DataMovementEvent;
+import org.apache.tez.runtime.api.events.InputFailedEvent;
+import org.apache.tez.runtime.api.events.InputReadErrorEvent;
 
 public class OneToOneEdgeManager extends EdgeManager {
 

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/ScatterGatherEdgeManager.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/ScatterGatherEdgeManager.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/ScatterGatherEdgeManager.java
index 1d4df5b..3d1d289 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/ScatterGatherEdgeManager.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/ScatterGatherEdgeManager.java
@@ -22,9 +22,9 @@ import java.util.List;
 
 import org.apache.tez.dag.app.dag.EdgeManager;
 import org.apache.tez.dag.app.dag.Vertex;
-import org.apache.tez.engine.api.events.DataMovementEvent;
-import org.apache.tez.engine.api.events.InputFailedEvent;
-import org.apache.tez.engine.api.events.InputReadErrorEvent;
+import org.apache.tez.runtime.api.events.DataMovementEvent;
+import org.apache.tez.runtime.api.events.InputFailedEvent;
+import org.apache.tez.runtime.api.events.InputReadErrorEvent;
 
 public class ScatterGatherEdgeManager extends EdgeManager {
 

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/ShuffleVertexManager.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/ShuffleVertexManager.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/ShuffleVertexManager.java
index a0ed329..a4dd555 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/ShuffleVertexManager.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/ShuffleVertexManager.java
@@ -39,11 +39,11 @@ import org.apache.tez.dag.app.dag.VertexScheduler;
 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.engine.api.events.DataMovementEvent;
-import org.apache.tez.engine.api.events.InputFailedEvent;
-import org.apache.tez.engine.api.events.InputReadErrorEvent;
-import org.apache.tez.engine.records.TezDependentTaskCompletionEvent;
 import org.apache.tez.mapreduce.hadoop.MRHelpers;
+import org.apache.tez.runtime.api.events.DataMovementEvent;
+import org.apache.tez.runtime.api.events.InputFailedEvent;
+import org.apache.tez.runtime.api.events.InputReadErrorEvent;
+import org.apache.tez.runtime.records.TezDependentTaskCompletionEvent;
 
 /**
  * Starts scheduling tasks when number of completed source tasks crosses 
@@ -264,14 +264,14 @@ public class ShuffleVertexManager implements VertexScheduler {
       List<byte[]> taskConfs = new ArrayList<byte[]>(finalTaskParallelism);
       try {
         Configuration taskConf = new Configuration(false);
-        taskConf.setInt(TezJobConfig.TEZ_ENGINE_SHUFFLE_PARTITION_RANGE,
+        taskConf.setInt(TezJobConfig.TEZ_RUNTIME_SHUFFLE_PARTITION_RANGE,
             basePartitionRange);
         // create event user payload to inform the task
         for (int i = 0; i < numShufflersWithBaseRange; ++i) {
           taskConfs.add(MRHelpers.createUserPayloadFromConf(taskConf));
         }
         if(finalTaskParallelism > numShufflersWithBaseRange) {
-          taskConf.setInt(TezJobConfig.TEZ_ENGINE_SHUFFLE_PARTITION_RANGE,
+          taskConf.setInt(TezJobConfig.TEZ_RUNTIME_SHUFFLE_PARTITION_RANGE,
               remainderRangeForLastShuffler);
           taskConfs.add(MRHelpers.createUserPayloadFromConf(taskConf));
         }

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/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 f2b2776..b79f856 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
@@ -89,8 +89,8 @@ import org.apache.tez.dag.records.TezTaskAttemptID;
 import org.apache.tez.dag.records.TezTaskID;
 import org.apache.tez.dag.records.TezVertexID;
 import org.apache.tez.dag.utils.TezBuilderUtils;
-import org.apache.tez.engine.api.events.TaskStatusUpdateEvent;
-import org.apache.tez.engine.api.impl.TaskSpec;
+import org.apache.tez.runtime.api.events.TaskStatusUpdateEvent;
+import org.apache.tez.runtime.api.impl.TaskSpec;
 
 import com.google.common.annotations.VisibleForTesting;
 

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/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 13fa915..c1a9415 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
@@ -75,8 +75,8 @@ import org.apache.tez.dag.history.events.TaskStartedEvent;
 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.engine.api.impl.TezEvent;
-import org.apache.tez.engine.records.TezDependentTaskCompletionEvent;
+import org.apache.tez.runtime.api.impl.TezEvent;
+import org.apache.tez.runtime.records.TezDependentTaskCompletionEvent;
 
 import com.google.common.annotations.VisibleForTesting;
 

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/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 74005b7..520473d 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
@@ -106,15 +106,15 @@ 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.engine.api.events.DataMovementEvent;
-import org.apache.tez.engine.api.events.InputFailedEvent;
-import org.apache.tez.engine.api.events.TaskAttemptFailedEvent;
-import org.apache.tez.engine.api.events.TaskStatusUpdateEvent;
-import org.apache.tez.engine.api.impl.EventMetaData;
-import org.apache.tez.engine.api.impl.InputSpec;
-import org.apache.tez.engine.api.impl.OutputSpec;
-import org.apache.tez.engine.api.impl.TezEvent;
-import org.apache.tez.engine.records.TezDependentTaskCompletionEvent;
+import org.apache.tez.runtime.api.events.DataMovementEvent;
+import org.apache.tez.runtime.api.events.InputFailedEvent;
+import org.apache.tez.runtime.api.events.TaskAttemptFailedEvent;
+import org.apache.tez.runtime.api.events.TaskStatusUpdateEvent;
+import org.apache.tez.runtime.api.impl.EventMetaData;
+import org.apache.tez.runtime.api.impl.InputSpec;
+import org.apache.tez.runtime.api.impl.OutputSpec;
+import org.apache.tez.runtime.api.impl.TezEvent;
+import org.apache.tez.runtime.records.TezDependentTaskCompletionEvent;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.HashMultiset;

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-dag/src/main/java/org/apache/tez/dag/app/rm/AMSchedulerEventTALaunchRequest.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/AMSchedulerEventTALaunchRequest.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/AMSchedulerEventTALaunchRequest.java
index 14edd96..7a0b6f7 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/AMSchedulerEventTALaunchRequest.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/AMSchedulerEventTALaunchRequest.java
@@ -22,7 +22,7 @@ import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.tez.dag.app.ContainerContext;
 import org.apache.tez.dag.app.dag.TaskAttempt;
 import org.apache.tez.dag.records.TezTaskAttemptID;
-import org.apache.tez.engine.api.impl.TaskSpec;
+import org.apache.tez.runtime.api.impl.TaskSpec;
 
 public class AMSchedulerEventTALaunchRequest extends AMSchedulerEvent {
 

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerEventAssignTA.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerEventAssignTA.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerEventAssignTA.java
index 76e80f5..4adc302 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerEventAssignTA.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerEventAssignTA.java
@@ -19,7 +19,7 @@ package org.apache.tez.dag.app.rm.container;
 
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.tez.dag.records.TezTaskAttemptID;
-import org.apache.tez.engine.api.impl.TaskSpec;
+import org.apache.tez.runtime.api.impl.TaskSpec;
 
 public class AMContainerEventAssignTA extends AMContainerEvent {
 

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerHelpers.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerHelpers.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerHelpers.java
index 1855dbd..11e8aeb 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerHelpers.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerHelpers.java
@@ -44,9 +44,9 @@ import org.apache.hadoop.yarn.util.ConverterUtils;
 import org.apache.tez.dag.api.TezUncheckedException;
 import org.apache.tez.dag.app.AppContext;
 import org.apache.tez.dag.app.TaskAttemptListener;
-import org.apache.tez.dag.utils.TezEngineChildJVM;
-import org.apache.tez.engine.common.security.TokenCache;
-import org.apache.tez.engine.common.shuffle.server.ShuffleHandler;
+import org.apache.tez.dag.utils.TezRuntimeChildJVM;
+import org.apache.tez.runtime.library.common.security.TokenCache;
+import org.apache.tez.runtime.library.common.shuffle.server.ShuffleHandler;
 
 import com.google.common.annotations.VisibleForTesting;
 
@@ -163,7 +163,7 @@ public class AMContainerHelpers {
     myEnv.putAll(vertexEnv);
 
     // Set up the launch command
-    List<String> commands = TezEngineChildJVM.getVMCommand(
+    List<String> commands = TezRuntimeChildJVM.getVMCommand(
         taskAttemptListener.getAddress(), containerId.toString(),
         appContext.getApplicationID().toString(),
         appContext.getApplicationAttemptId().getAttemptId(),

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerImpl.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerImpl.java
index 94dd580..4731a24 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerImpl.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerImpl.java
@@ -54,7 +54,7 @@ import org.apache.tez.dag.app.rm.NMCommunicatorLaunchRequestEvent;
 import org.apache.tez.dag.app.rm.NMCommunicatorStopRequestEvent;
 import org.apache.tez.dag.records.TezTaskAttemptID;
 //import org.apache.tez.dag.app.dag.event.TaskAttemptEventDiagnosticsUpdate;
-import org.apache.tez.engine.api.impl.TaskSpec;
+import org.apache.tez.runtime.api.impl.TaskSpec;
 
 @SuppressWarnings("rawtypes")
 public class AMContainerImpl implements AMContainer {

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerTask.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerTask.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerTask.java
index c0ef524..2015505 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerTask.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerTask.java
@@ -18,7 +18,7 @@
 
 package org.apache.tez.dag.app.rm.container;
 
-import org.apache.tez.engine.api.impl.TaskSpec;
+import org.apache.tez.runtime.api.impl.TaskSpec;
 
 public class AMContainerTask {
   private final boolean shouldDie;

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-dag/src/main/java/org/apache/tez/dag/utils/TezEngineChildJVM.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/utils/TezEngineChildJVM.java b/tez-dag/src/main/java/org/apache/tez/dag/utils/TezEngineChildJVM.java
deleted file mode 100644
index 8919698..0000000
--- a/tez-dag/src/main/java/org/apache/tez/dag/utils/TezEngineChildJVM.java
+++ /dev/null
@@ -1,122 +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.utils;
-
-import java.net.InetSocketAddress;
-import java.util.List;
-import java.util.Vector;
-
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.mapred.YarnTezDagChild;
-import org.apache.hadoop.yarn.api.ApplicationConstants;
-import org.apache.hadoop.yarn.api.ApplicationConstants.Environment;
-import org.apache.hadoop.yarn.conf.YarnConfiguration;
-
-public class TezEngineChildJVM {
-
-    // FIXME
-  public static enum LogName {
-    /** Log on the stdout of the task. */
-    STDOUT ("stdout"),
-
-    /** Log on the stderr of the task. */
-    STDERR ("stderr"),
-
-    /** Log on the map-reduce system logs of the task. */
-    SYSLOG ("syslog"),
-
-    /** The java profiler information. */
-    PROFILE ("profile.out"),
-
-    /** Log the debug script's stdout  */
-    DEBUGOUT ("debugout");
-
-    private String prefix;
-
-    private LogName(String prefix) {
-      this.prefix = prefix;
-    }
-
-    @Override
-    public String toString() {
-      return prefix;
-    }
-  }
-
-  private static String getTaskLogFile(LogName filter) {
-    return ApplicationConstants.LOG_DIR_EXPANSION_VAR + Path.SEPARATOR +
-        filter.toString();
-  }
-
-  public static List<String> getVMCommand(
-      InetSocketAddress taskAttemptListenerAddr,
-      String containerIdentifier,
-      String tokenIdentifier,
-      int applicationAttemptNumber,
-      boolean shouldProfile,
-      String javaOpts) {
-
-    Vector<String> vargs = new Vector<String>(9);
-
-    vargs.add(Environment.JAVA_HOME.$() + "/bin/java");
-
-    //set custom javaOpts
-    vargs.add(javaOpts);
-
-//[Debug Task] Current simplest way to attach debugger to  Tez Child Task
-// Uncomment the following, then launch a regular job
-// Works best on one-box configured with a single container (hence one task at a time).
-//    LOG.error(" !!!!!!!!! Launching Child-Task in debug/suspend mode.  Attach to port 8003 !!!!!!!!");
-//    vargs.add("-Xdebug -Djava.compiler=NONE -Xrunjdwp:transport=dt_socket,address=8003,server=y,suspend=y");
-
-    Path childTmpDir = new Path(Environment.PWD.$(),
-        YarnConfiguration.DEFAULT_CONTAINER_TEMP_DIR);
-    vargs.add("-Djava.io.tmpdir=" + childTmpDir);
-
-    // FIXME Setup the log4j properties
-
-    // Decision to profile needs to be made in the scheduler.
-    if (shouldProfile) {
-      // FIXME add support for profiling
-    }
-
-    // Add main class and its arguments
-    vargs.add(YarnTezDagChild.class.getName());  // main of Child
-
-    // pass TaskAttemptListener's address
-    vargs.add(taskAttemptListenerAddr.getAddress().getHostAddress());
-    vargs.add(Integer.toString(taskAttemptListenerAddr.getPort()));
-    vargs.add(containerIdentifier);
-    vargs.add(tokenIdentifier);
-    vargs.add(Integer.toString(applicationAttemptNumber));
-
-    vargs.add("1>" + getTaskLogFile(LogName.STDOUT));
-    vargs.add("2>" + getTaskLogFile(LogName.STDERR));
-
-    // Final commmand
-    StringBuilder mergedCommand = new StringBuilder();
-    for (CharSequence str : vargs) {
-      mergedCommand.append(str).append(" ");
-    }
-    Vector<String> vargsFinal = new Vector<String>(1);
-    vargsFinal.add(mergedCommand.toString());
-    return vargsFinal;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-dag/src/main/java/org/apache/tez/dag/utils/TezRuntimeChildJVM.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/utils/TezRuntimeChildJVM.java b/tez-dag/src/main/java/org/apache/tez/dag/utils/TezRuntimeChildJVM.java
new file mode 100644
index 0000000..e1219c1
--- /dev/null
+++ b/tez-dag/src/main/java/org/apache/tez/dag/utils/TezRuntimeChildJVM.java
@@ -0,0 +1,122 @@
+/**
+* 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.utils;
+
+import java.net.InetSocketAddress;
+import java.util.List;
+import java.util.Vector;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapred.YarnTezDagChild;
+import org.apache.hadoop.yarn.api.ApplicationConstants;
+import org.apache.hadoop.yarn.api.ApplicationConstants.Environment;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+
+public class TezRuntimeChildJVM {
+
+    // FIXME
+  public static enum LogName {
+    /** Log on the stdout of the task. */
+    STDOUT ("stdout"),
+
+    /** Log on the stderr of the task. */
+    STDERR ("stderr"),
+
+    /** Log on the map-reduce system logs of the task. */
+    SYSLOG ("syslog"),
+
+    /** The java profiler information. */
+    PROFILE ("profile.out"),
+
+    /** Log the debug script's stdout  */
+    DEBUGOUT ("debugout");
+
+    private String prefix;
+
+    private LogName(String prefix) {
+      this.prefix = prefix;
+    }
+
+    @Override
+    public String toString() {
+      return prefix;
+    }
+  }
+
+  private static String getTaskLogFile(LogName filter) {
+    return ApplicationConstants.LOG_DIR_EXPANSION_VAR + Path.SEPARATOR +
+        filter.toString();
+  }
+
+  public static List<String> getVMCommand(
+      InetSocketAddress taskAttemptListenerAddr,
+      String containerIdentifier,
+      String tokenIdentifier,
+      int applicationAttemptNumber,
+      boolean shouldProfile,
+      String javaOpts) {
+
+    Vector<String> vargs = new Vector<String>(9);
+
+    vargs.add(Environment.JAVA_HOME.$() + "/bin/java");
+
+    //set custom javaOpts
+    vargs.add(javaOpts);
+
+//[Debug Task] Current simplest way to attach debugger to  Tez Child Task
+// Uncomment the following, then launch a regular job
+// Works best on one-box configured with a single container (hence one task at a time).
+//    LOG.error(" !!!!!!!!! Launching Child-Task in debug/suspend mode.  Attach to port 8003 !!!!!!!!");
+//    vargs.add("-Xdebug -Djava.compiler=NONE -Xrunjdwp:transport=dt_socket,address=8003,server=y,suspend=y");
+
+    Path childTmpDir = new Path(Environment.PWD.$(),
+        YarnConfiguration.DEFAULT_CONTAINER_TEMP_DIR);
+    vargs.add("-Djava.io.tmpdir=" + childTmpDir);
+
+    // FIXME Setup the log4j properties
+
+    // Decision to profile needs to be made in the scheduler.
+    if (shouldProfile) {
+      // FIXME add support for profiling
+    }
+
+    // Add main class and its arguments
+    vargs.add(YarnTezDagChild.class.getName());  // main of Child
+
+    // pass TaskAttemptListener's address
+    vargs.add(taskAttemptListenerAddr.getAddress().getHostAddress());
+    vargs.add(Integer.toString(taskAttemptListenerAddr.getPort()));
+    vargs.add(containerIdentifier);
+    vargs.add(tokenIdentifier);
+    vargs.add(Integer.toString(applicationAttemptNumber));
+
+    vargs.add("1>" + getTaskLogFile(LogName.STDOUT));
+    vargs.add("2>" + getTaskLogFile(LogName.STDERR));
+
+    // Final commmand
+    StringBuilder mergedCommand = new StringBuilder();
+    for (CharSequence str : vargs) {
+      mergedCommand.append(str).append(" ");
+    }
+    Vector<String> vargsFinal = new Vector<String>(1);
+    vargsFinal.add(mergedCommand.toString());
+    return vargsFinal;
+  }
+
+}


[18/50] [abbrv] TEZ-443. Merge tez-dag-api and tez-engine-api into a single module - tez-api (part of TEZ-398). (sseth)

Posted by ss...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-dag-api/src/main/java/org/apache/tez/dag/api/client/rpc/DAGClientRPCImpl.java
----------------------------------------------------------------------
diff --git a/tez-dag-api/src/main/java/org/apache/tez/dag/api/client/rpc/DAGClientRPCImpl.java b/tez-dag-api/src/main/java/org/apache/tez/dag/api/client/rpc/DAGClientRPCImpl.java
deleted file mode 100644
index dae5625..0000000
--- a/tez-dag-api/src/main/java/org/apache/tez/dag/api/client/rpc/DAGClientRPCImpl.java
+++ /dev/null
@@ -1,291 +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.api.client.rpc;
-
-import java.io.IOException;
-import java.net.InetSocketAddress;
-import java.util.Collections;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.ipc.ProtobufRpcEngine;
-import org.apache.hadoop.ipc.RPC;
-import org.apache.hadoop.yarn.api.records.ApplicationId;
-import org.apache.hadoop.yarn.api.records.ApplicationReport;
-import org.apache.hadoop.yarn.api.records.YarnApplicationState;
-import org.apache.hadoop.yarn.client.api.YarnClient;
-import org.apache.hadoop.yarn.client.api.impl.YarnClientImpl;
-import org.apache.hadoop.yarn.conf.YarnConfiguration;
-import org.apache.hadoop.yarn.exceptions.YarnException;
-import org.apache.tez.dag.api.TezConfiguration;
-import org.apache.tez.dag.api.TezException;
-import org.apache.tez.dag.api.TezUncheckedException;
-import org.apache.tez.dag.api.client.DAGClient;
-import org.apache.tez.dag.api.client.DAGStatus;
-import org.apache.tez.dag.api.client.VertexStatus;
-import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolRPC.GetDAGStatusRequestProto;
-import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolRPC.GetVertexStatusRequestProto;
-import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolRPC.TryKillDAGRequestProto;
-import org.apache.tez.dag.api.records.DAGProtos.DAGStatusProto;
-import org.apache.tez.dag.api.records.DAGProtos.DAGStatusStateProto;
-
-import com.google.protobuf.ServiceException;
-
-public class DAGClientRPCImpl implements DAGClient {
-  private static final Log LOG = LogFactory.getLog(DAGClientRPCImpl.class);
-
-  private final ApplicationId appId;
-  private final String dagId;
-  private final TezConfiguration conf;
-  private ApplicationReport appReport;
-  private YarnClient yarnClient;
-  private DAGClientAMProtocolBlockingPB proxy = null;
-
-  public DAGClientRPCImpl(ApplicationId appId, String dagId,
-      TezConfiguration conf) {
-    this.appId = appId;
-    this.dagId = dagId;
-    this.conf = conf;
-    yarnClient = new YarnClientImpl();
-    yarnClient.init(new YarnConfiguration(conf));
-    yarnClient.start();
-    appReport = null;
-  }
-
-  @Override
-  public ApplicationId getApplicationId() {
-    return appId;
-  }
-
-  @Override
-  public DAGStatus getDAGStatus() throws IOException, TezException {
-    if(createAMProxyIfNeeded()) {
-      try {
-        return getDAGStatusViaAM();
-      } catch (TezException e) {
-        resetProxy(e); // create proxy again
-      }
-    }
-
-    // Later maybe from History
-    return getDAGStatusViaRM();
-  }
-
-  @Override
-  public VertexStatus getVertexStatus(String vertexName)
-                                    throws IOException, TezException {
-    if(createAMProxyIfNeeded()) {
-      try {
-        return getVertexStatusViaAM(vertexName);
-      } catch (TezException e) {
-        resetProxy(e); // create proxy again
-      }
-    }
-
-    // need AM for this. Later maybe from History
-    return null;
-  }
-
-  @Override
-  public void tryKillDAG() throws TezException, IOException {
-    if(LOG.isDebugEnabled()) {
-      LOG.debug("TryKill for app: " + appId + " dag:" + dagId);
-    }
-    if(createAMProxyIfNeeded()) {
-      TryKillDAGRequestProto requestProto =
-          TryKillDAGRequestProto.newBuilder().setDagId(dagId).build();
-      try {
-        proxy.tryKillDAG(null, requestProto);
-      } catch (ServiceException e) {
-        resetProxy(e);
-      }
-    }
-  }
-
-  @Override
-  public void close() throws IOException {
-    if (this.proxy != null) {
-      RPC.stopProxy(this.proxy);
-    }
-    if(yarnClient != null) {
-      yarnClient.stop();
-    }
-  }
-
-  @Override
-  public ApplicationReport getApplicationReport() {
-    return appReport;
-  }
-
-  void resetProxy(Exception e) {
-    if(LOG.isDebugEnabled()) {
-      LOG.debug("Resetting AM proxy for app: " + appId + " dag:" + dagId +
-          " due to exception :", e);
-    }
-    proxy = null;
-  }
-
-  DAGStatus getDAGStatusViaAM() throws IOException, TezException {
-    if(LOG.isDebugEnabled()) {
-      LOG.debug("GetDAGStatus via AM for app: " + appId + " dag:" + dagId);
-    }
-    GetDAGStatusRequestProto requestProto =
-        GetDAGStatusRequestProto.newBuilder().setDagId(dagId).build();
-    try {
-      return new DAGStatus(
-                 proxy.getDAGStatus(null, requestProto).getDagStatus());
-    } catch (ServiceException e) {
-      // TEZ-151 retrieve wrapped TezException
-      throw new TezException(e);
-    }
-  }
-
-
-
-  DAGStatus getDAGStatusViaRM() throws TezException, IOException {
-    if(LOG.isDebugEnabled()) {
-      LOG.debug("GetDAGStatus via AM for app: " + appId + " dag:" + dagId);
-    }
-    ApplicationReport appReport;
-    try {
-      appReport = yarnClient.getApplicationReport(appId);
-    } catch (YarnException e) {
-      throw new TezException(e);
-    }
-
-    if(appReport == null) {
-      throw new TezException("Unknown/Invalid appId: " + appId);
-    }
-
-    DAGStatusProto.Builder builder = DAGStatusProto.newBuilder();
-    DAGStatus dagStatus = new DAGStatus(builder);
-    DAGStatusStateProto dagState = null;
-    switch (appReport.getYarnApplicationState()) {
-    case NEW:
-    case NEW_SAVING:
-    case SUBMITTED:
-    case ACCEPTED:
-      dagState = DAGStatusStateProto.DAG_SUBMITTED;
-      break;
-    case RUNNING:
-      dagState = DAGStatusStateProto.DAG_RUNNING;
-      break;
-    case FAILED:
-      dagState = DAGStatusStateProto.DAG_FAILED;
-      break;
-    case KILLED:
-      dagState = DAGStatusStateProto.DAG_KILLED;
-      break;
-    case FINISHED:
-      switch(appReport.getFinalApplicationStatus()) {
-      case UNDEFINED:
-      case FAILED:
-        dagState = DAGStatusStateProto.DAG_FAILED;
-        break;
-      case KILLED:
-        dagState = DAGStatusStateProto.DAG_KILLED;
-        break;
-      case SUCCEEDED:
-        dagState = DAGStatusStateProto.DAG_SUCCEEDED;
-        break;
-      default:
-        throw new TezUncheckedException("Encountered unknown final application"
-          + " status from YARN"
-          + ", appState=" + appReport.getYarnApplicationState()
-          + ", finalStatus=" + appReport.getFinalApplicationStatus());
-      }
-      break;
-    default:
-      throw new TezUncheckedException("Encountered unknown application state"
-          + " from YARN, appState=" + appReport.getYarnApplicationState());
-    }
-
-    builder.setState(dagState);
-    if(appReport.getDiagnostics() != null) {
-      builder.addAllDiagnostics(Collections.singleton(appReport.getDiagnostics()));
-    }
-
-    return dagStatus;
-  }
-
-  VertexStatus getVertexStatusViaAM(String vertexName) throws TezException {
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("GetVertexStatus via AM for app: " + appId + " dag: " + dagId
-          + " vertex: " + vertexName);
-    }
-    GetVertexStatusRequestProto requestProto =
-        GetVertexStatusRequestProto.newBuilder().
-                        setDagId(dagId).setVertexName(vertexName).build();
-
-    try {
-      return new VertexStatus(
-                 proxy.getVertexStatus(null, requestProto).getVertexStatus());
-    } catch (ServiceException e) {
-      // TEZ-151 retrieve wrapped TezException
-      throw new TezException(e);
-    }
-  }
-
-  ApplicationReport getAppReport() throws IOException, TezException {
-    try {
-      ApplicationReport appReport = yarnClient.getApplicationReport(appId);
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("App: " + appId + " in state: "
-            + appReport.getYarnApplicationState());
-      }
-      return appReport;
-    } catch (YarnException e) {
-      throw new TezException(e);
-    }
-  }
-
-  boolean createAMProxyIfNeeded() throws IOException, TezException {
-    if(proxy != null) {
-      // if proxy exist optimistically use it assuming there is no retry
-      return true;
-    }
-    appReport = getAppReport();
-
-    if(appReport == null) {
-      return false;
-    }
-    YarnApplicationState appState = appReport.getYarnApplicationState();
-    if(appState != YarnApplicationState.RUNNING) {
-      return false;
-    }
-
-    // YARN-808. Cannot ascertain if AM is ready until we connect to it.
-    // workaround check the default string set by YARN
-    if(appReport.getHost() == null || appReport.getHost().equals("N/A") ||
-        appReport.getRpcPort() == 0){
-      // attempt not running
-      return false;
-    }
-
-    InetSocketAddress addr = new InetSocketAddress(appReport.getHost(),
-        appReport.getRpcPort());
-
-    RPC.setProtocolEngine(conf, DAGClientAMProtocolBlockingPB.class,
-        ProtobufRpcEngine.class);
-    proxy = (DAGClientAMProtocolBlockingPB) RPC.getProxy(
-        DAGClientAMProtocolBlockingPB.class, 0, addr, conf);
-    return true;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-dag-api/src/main/proto/DAGApiRecords.proto
----------------------------------------------------------------------
diff --git a/tez-dag-api/src/main/proto/DAGApiRecords.proto b/tez-dag-api/src/main/proto/DAGApiRecords.proto
deleted file mode 100644
index 4385749..0000000
--- a/tez-dag-api/src/main/proto/DAGApiRecords.proto
+++ /dev/null
@@ -1,183 +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.
- */
-
-option java_package = "org.apache.tez.dag.api.records";
-option java_outer_classname = "DAGProtos";
-option java_generate_equals_and_hash = true;
-
-// DAG plan messages
-
-// Many of these types have a dual in the Tez-api.  To reduce confusion, these types have prefix or suffix 
-// of "Plan" to indicate they are to be used in the dag-plan.
-// The big types use a suffix:  JobPlan, VertexPlan, EdgePlan 
-//   --> these get more direct use in the runtime and the naming is natural.
-// The enums and utility types use prefix: PlanVertexType, PlanEdgeConnectionPaatern, etc
-//   --> there is not great naming choice for these that avoids ambiguity, but this one seems acceptable. 
-
-enum PlanVertexType {
-  INPUT = 0;
-  NORMAL = 1;
-  OUTPUT = 2;
-}
-
-enum PlanEdgeDataMovementType {
-  ONE_TO_ONE = 0;
-  BROADCAST = 1;
-  SCATTER_GATHER = 2;
-}
-
-enum PlanEdgeDataSourceType {
-  PERSISTED = 0;
-  PERSISTED_RELIABLE = 1;
-  EPHEMERAL = 2;
-}
-
-enum PlanEdgeSchedulingType {
-  SEQUENTIAL = 0;
-  CONCURRENT = 1;
-}
-
-message PlanKeyValuePair {
-  required string key = 1;
-  required string value = 2;
-}
-
-enum PlanLocalResourceType {
-  FILE = 0;
-  ARCHIVE = 1;
-  PATTERN = 2;
-}
-
-enum PlanLocalResourceVisibility {
-  PUBLIC = 0;
-  PRIVATE = 1;
-  APPLICATION = 2;
-}
-
-message PlanLocalResource {
-  required string name = 1;
-  required string uri = 2;
-  required int64 size = 3;
-  required int64 timeStamp = 4;
-  required PlanLocalResourceType type = 5;
-  required PlanLocalResourceVisibility visibility = 6;
-  optional string pattern = 7; // only used if type=PATTERN
-}
-
-// Each taskLocationHint represents a single split in in the input.
-// It is the list of [{rack,machines}] that host a replica of each particular split.
-// For now it is represented as pair-of-arrays rather than array-of-pairs.
-message PlanTaskLocationHint {
-  repeated string rack = 1;
-  repeated string host = 2;
-}
-
-message PlanTaskConfiguration {
-  required int32 numTasks = 1;
-  required int32 memoryMb = 2;
-  required int32 virtualCores = 3;
-  required string javaOpts = 4;
-  required string taskModule = 5;
-  repeated PlanLocalResource localResource = 6;
-  repeated PlanKeyValuePair environmentSetting = 8;  
-}
-
-message TezEntityDescriptorProto {
-  optional string class_name = 1;
-  optional bytes user_payload = 2;
-}
-
-message VertexPlan {
-  required string name = 1;
-  required PlanVertexType type = 2;
-  optional TezEntityDescriptorProto processor_descriptor = 3;
-  required PlanTaskConfiguration taskConfig = 4;
-  repeated PlanTaskLocationHint taskLocationHint = 7;
-  repeated string inEdgeId = 8;
-  repeated string outEdgeId = 9;
-}
-
-message EdgePlan {
-  required string id = 1;
-  required string inputVertexName = 2;
-  required string outputVertexName = 3;
-  required PlanEdgeDataMovementType dataMovementType = 4;
-  required PlanEdgeDataSourceType dataSourceType = 5;
-  required PlanEdgeSchedulingType schedulingType = 6;
-  optional TezEntityDescriptorProto edge_source = 7;
-  optional TezEntityDescriptorProto edge_destination = 8;
-}
-
-message ConfigurationProto {
-  repeated PlanKeyValuePair confKeyValues = 1;
-}
-
-message DAGPlan {
-  required string name = 1;
-  repeated VertexPlan vertex = 2;
-  repeated EdgePlan edge = 3;
-  optional ConfigurationProto dagKeyValues = 4;
-}
-
-// DAG monitoring messages
-message ProgressProto {
-  optional int32 totalTaskCount = 1;
-  optional int32 succeededTaskCount = 2;
-  optional int32 runningTaskCount = 3;
-  optional int32 failedTaskCount = 4;
-  optional int32 killedTaskCount = 5;
-}
-
-enum VertexStatusStateProto {
-  VERTEX_INITED = 0;
-  VERTEX_RUNNING = 1;
-  VERTEX_SUCCEEDED = 2;
-  VERTEX_KILLED = 3;
-  VERTEX_FAILED = 4;
-  VERTEX_ERROR = 5;
-  VERTEX_TERMINATING = 6;
-}
-
-message VertexStatusProto {
-  optional VertexStatusStateProto state = 1;
-  repeated string diagnostics = 2;
-  optional ProgressProto progress = 3;
-}
-
-enum DAGStatusStateProto {
-  DAG_SUBMITTED = 0;
-  DAG_INITING = 1;
-  DAG_RUNNING = 2;
-  DAG_SUCCEEDED = 3;
-  DAG_KILLED = 4;
-  DAG_FAILED = 5;
-  DAG_ERROR = 6;
-  DAG_TERMINATING = 7;
-}
-
-message StringProgressPairProto {
-  required string key = 1;
-  required ProgressProto progress = 2;
-}
-
-message DAGStatusProto {
-  optional DAGStatusStateProto state = 1;
-  repeated string diagnostics = 2;
-  optional ProgressProto DAGProgress = 3;
-  repeated StringProgressPairProto vertexProgress = 4;  
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-dag-api/src/main/proto/DAGClientAMProtocol.proto
----------------------------------------------------------------------
diff --git a/tez-dag-api/src/main/proto/DAGClientAMProtocol.proto b/tez-dag-api/src/main/proto/DAGClientAMProtocol.proto
deleted file mode 100644
index 6fcd1f8..0000000
--- a/tez-dag-api/src/main/proto/DAGClientAMProtocol.proto
+++ /dev/null
@@ -1,81 +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.
- */
-
-option java_package = "org.apache.tez.dag.api.client.rpc";
-option java_outer_classname = "DAGClientAMProtocolRPC";
-option java_generic_services = true;
-option java_generate_equals_and_hash = true;
-
-//import "DAGClientAMProtocolRecords.proto";
-
-import "DAGApiRecords.proto";
-
-message GetAllDAGsRequestProto {
-}
-
-message GetAllDAGsResponseProto {
-  repeated string dagId = 1;
-}
-
-message GetDAGStatusRequestProto {
-  optional string dagId = 1;
-}
-
-message GetDAGStatusResponseProto {
-  optional DAGStatusProto dagStatus = 1;
-}
-
-message GetVertexStatusRequestProto {
-  optional string dagId = 1;
-  optional string vertexName = 2;
-}
-
-message GetVertexStatusResponseProto {
-  optional VertexStatusProto vertexStatus = 1;
-}
-
-message TryKillDAGRequestProto {
-  optional string dagId = 1;
-}
-
-message TryKillDAGResponseProto {
-  //nothing yet
-}
-
-message SubmitDAGRequestProto {
-  optional DAGPlan d_a_g_plan = 1;
-}
-
-message SubmitDAGResponseProto {
-  optional string dagId = 1;
-}
-
-message ShutdownSessionRequestProto {
-}
-
-message ShutdownSessionResponseProto {
-}
-
-service DAGClientAMProtocol {
-  rpc getAllDAGs (GetAllDAGsRequestProto) returns (GetAllDAGsResponseProto);
-  rpc getDAGStatus (GetDAGStatusRequestProto) returns (GetDAGStatusResponseProto);
-  rpc getVertexStatus (GetVertexStatusRequestProto) returns (GetVertexStatusResponseProto);
-  rpc tryKillDAG (TryKillDAGRequestProto) returns (TryKillDAGResponseProto);
-  rpc submitDAG (SubmitDAGRequestProto) returns (SubmitDAGResponseProto);
-  rpc shutdownSession (ShutdownSessionRequestProto) returns (ShutdownSessionResponseProto);
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-dag-api/src/test/java/org/apache/tez/dag/api/TestDAGPlan.java
----------------------------------------------------------------------
diff --git a/tez-dag-api/src/test/java/org/apache/tez/dag/api/TestDAGPlan.java b/tez-dag-api/src/test/java/org/apache/tez/dag/api/TestDAGPlan.java
deleted file mode 100644
index 53ec357..0000000
--- a/tez-dag-api/src/test/java/org/apache/tez/dag/api/TestDAGPlan.java
+++ /dev/null
@@ -1,155 +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.api;
-
-import static org.junit.Assert.*;
-
-import java.io.File;
-import java.io.FileInputStream;
-import java.io.FileOutputStream;
-import java.io.IOException;
-import java.util.HashMap;
-
-import org.apache.hadoop.yarn.api.records.LocalResource;
-import org.apache.hadoop.yarn.api.records.Resource;
-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.records.DAGProtos.DAGPlan;
-import org.apache.tez.dag.api.records.DAGProtos.EdgePlan;
-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.VertexPlan;
-import org.junit.Assert;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TemporaryFolder;
-
-// based on TestDAGLocationHint
-public class TestDAGPlan {
-  @Rule
-  public TemporaryFolder tempFolder = new TemporaryFolder(); //TODO: doesn't seem to be deleting this folder automatically as expected.
-
-  @Test
-  public void testBasicJobPlanSerde() throws IOException {
-
-    DAGPlan job = DAGPlan.newBuilder()
-       .setName("test")
-       .addVertex(
-           VertexPlan.newBuilder()
-             .setName("vertex1")
-             .setType(PlanVertexType.NORMAL)
-             .addTaskLocationHint(PlanTaskLocationHint.newBuilder().addHost("machineName").addRack("rack1").build())
-             .setTaskConfig(
-                 PlanTaskConfiguration.newBuilder()
-                   .setNumTasks(2)
-                   .setVirtualCores(4)
-                   .setMemoryMb(1024)
-                   .setJavaOpts("")
-                   .setTaskModule("x.y")
-                   .build())
-             .build())
-        .build();
-   File file = tempFolder.newFile("jobPlan");
-   FileOutputStream outStream = null;
-   try {
-     outStream = new FileOutputStream(file);
-     job.writeTo(outStream);
-   }
-   finally {
-     if(outStream != null){
-       outStream.close();
-     }
-   }
-
-   DAGPlan inJob;
-   FileInputStream inputStream;
-   try {
-     inputStream = new FileInputStream(file);
-     inJob = DAGPlan.newBuilder().mergeFrom(inputStream).build();
-   }
-   finally {
-     outStream.close();
-   }
-
-   Assert.assertEquals(job, inJob);
-  }
-
-  @Test
-  public void testUserPayloadSerde() {
-    DAG dag = new DAG("testDag");
-    ProcessorDescriptor pd1 = new ProcessorDescriptor("processor1").
-        setUserPayload("processor1Bytes".getBytes());
-    ProcessorDescriptor pd2 = new ProcessorDescriptor("processor2").
-        setUserPayload("processor2Bytes".getBytes());
-    Vertex v1 = new Vertex("v1", pd1, 10, Resource.newInstance(1024, 1));
-    Vertex v2 = new Vertex("v2", pd2, 1, Resource.newInstance(1024, 1));
-    v1.setJavaOpts("").setTaskEnvironment(new HashMap<String, String>())
-        .setTaskLocalResources(new HashMap<String, LocalResource>());
-    v2.setJavaOpts("").setTaskEnvironment(new HashMap<String, String>())
-        .setTaskLocalResources(new HashMap<String, LocalResource>());
-
-    InputDescriptor inputDescriptor = new InputDescriptor("input").
-        setUserPayload("inputBytes".getBytes());
-    OutputDescriptor outputDescriptor = new OutputDescriptor("output").
-        setUserPayload("outputBytes".getBytes());
-    Edge edge = new Edge(v1, v2, new EdgeProperty(
-        DataMovementType.SCATTER_GATHER, DataSourceType.PERSISTED,
-        SchedulingType.SEQUENTIAL, outputDescriptor, inputDescriptor));
-
-    dag.addVertex(v1).addVertex(v2).addEdge(edge);
-
-    DAGPlan dagProto = dag.createDag(new TezConfiguration());
-
-    assertEquals(2, dagProto.getVertexCount());
-    assertEquals(1, dagProto.getEdgeCount());
-
-    VertexPlan v1Proto = dagProto.getVertex(0);
-    VertexPlan v2Proto = dagProto.getVertex(1);
-    EdgePlan edgeProto = dagProto.getEdge(0);
-
-    assertEquals("processor1Bytes", new String(v1Proto.getProcessorDescriptor()
-        .getUserPayload().toByteArray()));
-    assertEquals("processor1", v1Proto.getProcessorDescriptor().getClassName());
-
-    assertEquals("processor2Bytes", new String(v2Proto.getProcessorDescriptor()
-        .getUserPayload().toByteArray()));
-    assertEquals("processor2", v2Proto.getProcessorDescriptor().getClassName());
-
-    assertEquals("inputBytes", new String(edgeProto.getEdgeDestination()
-        .getUserPayload().toByteArray()));
-    assertEquals("input", edgeProto.getEdgeDestination().getClassName());
-
-    assertEquals("outputBytes", new String(edgeProto.getEdgeSource()
-        .getUserPayload().toByteArray()));
-    assertEquals("output", edgeProto.getEdgeSource().getClassName());
-
-    EdgeProperty edgeProperty = DagTypeConverters
-        .createEdgePropertyMapFromDAGPlan(dagProto.getEdgeList().get(0));
-
-    byte[] ib = edgeProperty.getEdgeDestination().getUserPayload();
-    assertEquals("inputBytes", new String(ib));
-    assertEquals("input", edgeProperty.getEdgeDestination().getClassName());
-
-    byte[] ob = edgeProperty.getEdgeSource().getUserPayload();
-    assertEquals("outputBytes", new String(ob));
-    assertEquals("output", edgeProperty.getEdgeSource().getClassName());
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-dag-api/src/test/java/org/apache/tez/dag/api/TestDAGVerify.java
----------------------------------------------------------------------
diff --git a/tez-dag-api/src/test/java/org/apache/tez/dag/api/TestDAGVerify.java b/tez-dag-api/src/test/java/org/apache/tez/dag/api/TestDAGVerify.java
deleted file mode 100644
index b33f3a6..0000000
--- a/tez-dag-api/src/test/java/org/apache/tez/dag/api/TestDAGVerify.java
+++ /dev/null
@@ -1,417 +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.api;
-
-import org.apache.hadoop.yarn.api.records.Resource;
-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.junit.Assert;
-import org.junit.Test;
-
-public class TestDAGVerify {
-
-  private final String dummyProcessorClassName = TestDAGVerify.class.getName();
-  private final String dummyInputClassName = TestDAGVerify.class.getName();
-  private final String dummyOutputClassName = TestDAGVerify.class.getName();
-  private final int dummyTaskCount = 2;
-  private final Resource dummyTaskResource = Resource.newInstance(1, 1);
-
-  //    v1
-  //    |
-  //    v2
-  @Test
-  public void testVerify1() {
-    Vertex v1 = new Vertex("v1",
-        new ProcessorDescriptor(dummyProcessorClassName),
-        dummyTaskCount, dummyTaskResource);
-    Vertex v2 = new Vertex("v2",
-        new ProcessorDescriptor("MapProcessor"),
-        dummyTaskCount, dummyTaskResource);
-    Edge e1 = new Edge(v1, v2,
-        new EdgeProperty(DataMovementType.SCATTER_GATHER, 
-            DataSourceType.PERSISTED, SchedulingType.SEQUENTIAL, 
-            new OutputDescriptor(dummyOutputClassName),
-            new InputDescriptor(dummyInputClassName)));
-    DAG dag = new DAG("testDag");
-    dag.addVertex(v1);
-    dag.addVertex(v2);
-    dag.addEdge(e1);
-    dag.verify();
-  }
-
-  @Test(expected = IllegalStateException.class)  
-  public void testVerify2() {
-    Vertex v1 = new Vertex("v1",
-        new ProcessorDescriptor(dummyProcessorClassName),
-        dummyTaskCount, dummyTaskResource);
-    Vertex v2 = new Vertex("v2",
-        new ProcessorDescriptor("MapProcessor"),
-        dummyTaskCount, dummyTaskResource);
-    Edge e1 = new Edge(v1, v2,
-        new EdgeProperty(DataMovementType.ONE_TO_ONE, 
-            DataSourceType.PERSISTED, SchedulingType.SEQUENTIAL, 
-            new OutputDescriptor(dummyOutputClassName),
-            new InputDescriptor(dummyInputClassName)));
-    DAG dag = new DAG("testDag");
-    dag.addVertex(v1);
-    dag.addVertex(v2);
-    dag.addEdge(e1);
-    dag.verify();
-  }
-
-  @Test(expected = IllegalStateException.class)  
-  public void testVerify3() {
-    Vertex v1 = new Vertex("v1",
-        new ProcessorDescriptor(dummyProcessorClassName),
-        dummyTaskCount, dummyTaskResource);
-    Vertex v2 = new Vertex("v2",
-        new ProcessorDescriptor("MapProcessor"),
-        dummyTaskCount, dummyTaskResource);
-    Edge e1 = new Edge(v1, v2,
-        new EdgeProperty(DataMovementType.SCATTER_GATHER, 
-            DataSourceType.EPHEMERAL, SchedulingType.SEQUENTIAL, 
-            new OutputDescriptor(dummyOutputClassName),
-            new InputDescriptor(dummyInputClassName)));
-    DAG dag = new DAG("testDag");
-    dag.addVertex(v1);
-    dag.addVertex(v2);
-    dag.addEdge(e1);
-    dag.verify();
-  }
-
-  @Test(expected = IllegalStateException.class)  
-  public void testVerify4() {
-    Vertex v1 = new Vertex("v1",
-        new ProcessorDescriptor(dummyProcessorClassName),
-        dummyTaskCount, dummyTaskResource);
-    Vertex v2 = new Vertex("v2",
-        new ProcessorDescriptor("MapProcessor"),
-        dummyTaskCount, dummyTaskResource);
-    Edge e1 = new Edge(v1, v2,
-        new EdgeProperty(DataMovementType.SCATTER_GATHER, 
-            DataSourceType.EPHEMERAL, SchedulingType.CONCURRENT, 
-            new OutputDescriptor(dummyOutputClassName),
-            new InputDescriptor(dummyInputClassName)));
-    DAG dag = new DAG("testDag");
-    dag.addVertex(v1);
-    dag.addVertex(v2);
-    dag.addEdge(e1);
-    dag.verify();
-  }
-
-  //    v1 <----
-  //      |     ^
-  //       v2   ^
-  //      |  |  ^
-  //    v3    v4
-  @Test
-  public void testCycle1() {
-    IllegalStateException ex=null;
-    Vertex v1 = new Vertex("v1",
-        new ProcessorDescriptor("MapProcessor"),
-        dummyTaskCount, dummyTaskResource);
-    Vertex v2 = new Vertex("v2",
-        new ProcessorDescriptor("MapProcessor"),
-        dummyTaskCount, dummyTaskResource);
-    Vertex v3 = new Vertex("v3",
-        new ProcessorDescriptor("MapProcessor"),
-        dummyTaskCount, dummyTaskResource);
-    Vertex v4 = new Vertex("v4",
-        new ProcessorDescriptor("MapProcessor"),
-        dummyTaskCount, dummyTaskResource);
-    Edge e1 = new Edge(v1, v2,
-        new EdgeProperty(DataMovementType.SCATTER_GATHER, 
-            DataSourceType.PERSISTED, SchedulingType.SEQUENTIAL, 
-            new OutputDescriptor("dummy output class"),
-            new InputDescriptor("dummy input class")));
-    Edge e2 = new Edge(v2, v3,
-        new EdgeProperty(DataMovementType.SCATTER_GATHER, 
-            DataSourceType.PERSISTED, SchedulingType.SEQUENTIAL, 
-            new OutputDescriptor("dummy output class"),
-            new InputDescriptor("dummy input class")));
-    Edge e3 = new Edge(v2, v4,
-        new EdgeProperty(DataMovementType.SCATTER_GATHER, 
-            DataSourceType.PERSISTED, SchedulingType.SEQUENTIAL, 
-            new OutputDescriptor("dummy output class"),
-            new InputDescriptor("dummy input class")));
-    Edge e4 = new Edge(v4, v1,
-        new EdgeProperty(DataMovementType.SCATTER_GATHER, 
-            DataSourceType.PERSISTED, SchedulingType.SEQUENTIAL, 
-            new OutputDescriptor("dummy output class"),
-            new InputDescriptor("dummy input class")));
-    DAG dag = new DAG("testDag");
-    dag.addVertex(v1);
-    dag.addVertex(v2);
-    dag.addVertex(v3);
-    dag.addVertex(v4);
-    dag.addEdge(e1);
-    dag.addEdge(e2);
-    dag.addEdge(e3);
-    dag.addEdge(e4);
-    try{
-      dag.verify();
-    }
-    catch (IllegalStateException e){
-      ex = e;
-    }
-    Assert.assertNotNull(ex);
-    System.out.println(ex.getMessage());
-    Assert.assertTrue(ex.getMessage().startsWith("DAG contains a cycle"));
-  }
-
-  //     v1
-  //      |
-  //    -> v2
-  //    ^  | |
-  //    v3    v4
-  @Test
-  public void testCycle2() {
-    IllegalStateException ex=null;
-    Vertex v1 = new Vertex("v1",
-        new ProcessorDescriptor("MapProcessor"),
-        dummyTaskCount, dummyTaskResource);
-    Vertex v2 = new Vertex("v2",
-        new ProcessorDescriptor("MapProcessor"),
-        dummyTaskCount, dummyTaskResource);
-    Vertex v3 = new Vertex("v3",
-        new ProcessorDescriptor("MapProcessor"),
-        dummyTaskCount, dummyTaskResource);
-    Vertex v4 = new Vertex("v4",
-        new ProcessorDescriptor("MapProcessor"),
-        dummyTaskCount, dummyTaskResource);
-    Edge e1 = new Edge(v1, v2,
-        new EdgeProperty(DataMovementType.SCATTER_GATHER, 
-            DataSourceType.PERSISTED, SchedulingType.SEQUENTIAL, 
-            new OutputDescriptor("dummy output class"),
-            new InputDescriptor("dummy input class")));
-    Edge e2 = new Edge(v2, v3,
-        new EdgeProperty(DataMovementType.SCATTER_GATHER, 
-            DataSourceType.PERSISTED, SchedulingType.SEQUENTIAL, 
-            new OutputDescriptor("dummy output class"),
-            new InputDescriptor("dummy input class")));
-    Edge e3 = new Edge(v2, v4,
-        new EdgeProperty(DataMovementType.SCATTER_GATHER, 
-            DataSourceType.PERSISTED, SchedulingType.SEQUENTIAL, 
-            new OutputDescriptor("dummy output class"),
-            new InputDescriptor("dummy input class")));
-    Edge e4 = new Edge(v3, v2,
-        new EdgeProperty(DataMovementType.SCATTER_GATHER, 
-            DataSourceType.PERSISTED, SchedulingType.SEQUENTIAL, 
-            new OutputDescriptor("dummy output class"),
-            new InputDescriptor("dummy input class")));
-    DAG dag = new DAG("testDag");
-    dag.addVertex(v1);
-    dag.addVertex(v2);
-    dag.addVertex(v3);
-    dag.addVertex(v4);
-    dag.addEdge(e1);
-    dag.addEdge(e2);
-    dag.addEdge(e3);
-    dag.addEdge(e4);
-    try{
-      dag.verify();
-    }
-    catch (IllegalStateException e){
-      ex = e;
-    }
-    Assert.assertNotNull(ex);
-    System.out.println(ex.getMessage());
-    Assert.assertTrue(ex.getMessage().startsWith("DAG contains a cycle"));
-  }
-
-  @Test
-  public void repeatedVertexName() {
-    IllegalStateException ex=null;
-    Vertex v1 = new Vertex("v1",
-        new ProcessorDescriptor("MapProcessor"),
-        dummyTaskCount, dummyTaskResource);
-    Vertex v1repeat = new Vertex("v1",
-        new ProcessorDescriptor("MapProcessor"),
-        dummyTaskCount, dummyTaskResource);
-    DAG dag = new DAG("testDag");
-    dag.addVertex(v1);
-    dag.addVertex(v1repeat);
-    try {
-      dag.verify();
-    }
-    catch (IllegalStateException e){
-      ex = e;
-    }
-    Assert.assertNotNull(ex);
-    System.out.println(ex.getMessage());
-    Assert.assertTrue(ex.getMessage().startsWith("DAG contains multiple vertices with name"));
-  }
-
-  //  v1  v2
-  //   |  |
-  //    v3
-  @Test
-  public void BinaryInputDisallowed() {
-    IllegalStateException ex=null;
-    try {
-      Vertex v1 = new Vertex("v1",
-          new ProcessorDescriptor("MapProcessor"),
-          dummyTaskCount, dummyTaskResource);
-      Vertex v2 = new Vertex("v2",
-          new ProcessorDescriptor("MapProcessor"),
-          dummyTaskCount, dummyTaskResource);
-      Vertex v3 = new Vertex("v3",
-          new ProcessorDescriptor("ReduceProcessor"),
-          dummyTaskCount, dummyTaskResource);
-      Edge e1 = new Edge(v1, v3,
-          new EdgeProperty(DataMovementType.ONE_TO_ONE, 
-              DataSourceType.PERSISTED, SchedulingType.SEQUENTIAL, 
-              new OutputDescriptor("dummy output class"),
-              new InputDescriptor("dummy input class")));
-      Edge e2 = new Edge(v2, v3,
-          new EdgeProperty(DataMovementType.SCATTER_GATHER, 
-              DataSourceType.PERSISTED, SchedulingType.SEQUENTIAL, 
-              new OutputDescriptor("dummy output class"),
-              new InputDescriptor("dummy input class")));
-      DAG dag = new DAG("testDag");
-      dag.addVertex(v1);
-      dag.addVertex(v2);
-      dag.addVertex(v3);
-      dag.addEdge(e1);
-      dag.addEdge(e2);
-      dag.verify();
-    }
-    catch (IllegalStateException e){
-      ex = e;
-    }
-    Assert.assertNotNull(ex);
-    System.out.println(ex.getMessage());
-    Assert.assertTrue(ex.getMessage().startsWith(
-        "Unsupported connection pattern on edge"));
-  }
-
-  //  v1  v2
-  //   |  |
-  //    v3
-  @Test
-  public void BinaryInputAllowed() {
-    Vertex v1 = new Vertex("v1",
-        new ProcessorDescriptor("MapProcessor"),
-        dummyTaskCount, dummyTaskResource);
-    Vertex v2 = new Vertex("v2",
-        new ProcessorDescriptor("MapProcessor"),
-        dummyTaskCount, dummyTaskResource);
-    Vertex v3 = new Vertex("v3",
-        new ProcessorDescriptor("ReduceProcessor"),
-        dummyTaskCount, dummyTaskResource);
-    Edge e1 = new Edge(v1, v3,
-        new EdgeProperty(DataMovementType.SCATTER_GATHER, 
-            DataSourceType.PERSISTED, SchedulingType.SEQUENTIAL,
-            new OutputDescriptor("dummy output class"),
-            new InputDescriptor("dummy input class")));
-    Edge e2 = new Edge(v2, v3,
-        new EdgeProperty(DataMovementType.SCATTER_GATHER, 
-            DataSourceType.PERSISTED, SchedulingType.SEQUENTIAL, 
-            new OutputDescriptor("dummy output class"),
-            new InputDescriptor("dummy input class")));
-    DAG dag = new DAG("testDag");
-    dag.addVertex(v1);
-    dag.addVertex(v2);
-    dag.addVertex(v3);
-    dag.addEdge(e1);
-    dag.addEdge(e2);
-    dag.verify();
-  }
-
-  //   v1
-  //  |  |
-  //  v2  v3
-  @Test
-  public void BinaryOutput() {
-    IllegalStateException ex=null;
-    try {
-      Vertex v1 = new Vertex("v1",
-          new ProcessorDescriptor("MapProcessor"),
-          dummyTaskCount, dummyTaskResource);
-      Vertex v2 = new Vertex("v2",
-          new ProcessorDescriptor("MapProcessor"),
-          dummyTaskCount, dummyTaskResource);
-      Vertex v3 = new Vertex("v3",
-          new ProcessorDescriptor("MapProcessor"),
-          dummyTaskCount, dummyTaskResource);
-      Edge e1 = new Edge(v1, v2,
-          new EdgeProperty(DataMovementType.SCATTER_GATHER, 
-              DataSourceType.PERSISTED, SchedulingType.SEQUENTIAL, 
-              new OutputDescriptor("dummy output class"),
-              new InputDescriptor("dummy input class")));
-      Edge e2 = new Edge(v1, v2,
-          new EdgeProperty(DataMovementType.SCATTER_GATHER, 
-              DataSourceType.PERSISTED, SchedulingType.SEQUENTIAL, 
-              new OutputDescriptor("dummy output class"),
-              new InputDescriptor("dummy input class")));
-      DAG dag = new DAG("testDag");
-      dag.addVertex(v1);
-      dag.addVertex(v2);
-      dag.addVertex(v3);
-      dag.addEdge(e1);
-      dag.addEdge(e2);
-      dag.verify();
-    }
-    catch (IllegalStateException e){
-      ex = e;
-    }
-    Assert.assertNotNull(ex);
-    System.out.println(ex.getMessage());
-    Assert.assertTrue(ex.getMessage().startsWith("Vertex has outDegree>1"));
-  }
-
-  @Test
-  public void testDagWithNoVertices() {
-    IllegalStateException ex=null;
-    try {
-      DAG dag = new DAG("testDag");
-      dag.verify();
-    }
-    catch (IllegalStateException e){
-      ex = e;
-    }
-    Assert.assertNotNull(ex);
-    System.out.println(ex.getMessage());
-    Assert.assertTrue(ex.getMessage()
-        .startsWith("Invalid dag containing 0 vertices"));
-  }
-
-  @SuppressWarnings("unused")
-  @Test
-  public void testInvalidVertexConstruction() {
-    try {
-      Vertex v1 = new Vertex("v1",
-          new ProcessorDescriptor("MapProcessor"),
-          0, dummyTaskResource);
-      Assert.fail("Expected exception for 0 parallelism");
-    } catch (IllegalArgumentException e) {
-      Assert.assertTrue(e.getMessage().startsWith("Parallelism cannot be 0"));
-    }
-    try {
-      Vertex v1 = new Vertex("v1",
-          new ProcessorDescriptor("MapProcessor"),
-          1, null);
-      Assert.fail("Expected exception for 0 parallelism");
-    } catch (IllegalArgumentException e) {
-      Assert.assertTrue(e.getMessage().startsWith("Resource cannot be null"));
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-dag/pom.xml
----------------------------------------------------------------------
diff --git a/tez-dag/pom.xml b/tez-dag/pom.xml
index 875a196..bc6aeef 100644
--- a/tez-dag/pom.xml
+++ b/tez-dag/pom.xml
@@ -29,6 +29,10 @@
   <dependencies>
     <dependency>
       <groupId>org.apache.tez</groupId>
+      <artifactId>tez-api</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.tez</groupId>
       <artifactId>tez-common</artifactId>
     </dependency>
     <dependency>
@@ -55,14 +59,6 @@
       <scope>test</scope>
     </dependency>
     <dependency>
-      <groupId>org.apache.tez</groupId>
-      <artifactId>tez-dag-api</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.tez</groupId>
-      <artifactId>tez-engine-api</artifactId>
-    </dependency>
-    <dependency>
       <groupId>junit</groupId>
       <artifactId>junit</artifactId>
       <scope>test</scope>

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java b/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java
index 36486c9..7a143a5 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java
@@ -44,7 +44,6 @@ import org.apache.tez.common.records.ProceedToCompletionResponse;
 import org.apache.tez.dag.api.TezConfiguration;
 import org.apache.tez.dag.app.dag.DAG;
 import org.apache.tez.dag.app.dag.Task;
-import org.apache.tez.dag.app.dag.event.TaskAttemptEventOutputConsumable;
 import org.apache.tez.dag.app.dag.event.TaskAttemptEventStartedRemotely;
 import org.apache.tez.dag.app.dag.event.VertexEventRouteEvent;
 import org.apache.tez.dag.app.rm.container.AMContainerImpl;
@@ -56,9 +55,6 @@ import org.apache.tez.engine.api.impl.TezEvent;
 import org.apache.tez.engine.api.impl.TezHeartbeatRequest;
 import org.apache.tez.engine.api.impl.TezHeartbeatResponse;
 import org.apache.tez.engine.common.security.JobTokenSecretManager;
-import org.apache.tez.engine.records.OutputContext;
-import org.apache.tez.engine.records.TezDependentTaskCompletionEvent;
-import org.apache.tez.engine.records.TezTaskDependencyCompletionEventsUpdate;
 
 @SuppressWarnings("unchecked")
 public class TaskAttemptListenerImpTezDag extends AbstractService implements
@@ -176,29 +172,6 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
   }
 
   @Override
-  public TezTaskDependencyCompletionEventsUpdate getDependentTasksCompletionEvents(
-      int fromEventIdx, int maxEvents,
-      TezTaskAttemptID taskAttemptID) {
-
-    LOG.info("Dependency Completion Events request from " + taskAttemptID
-        + ". fromEventID " + fromEventIdx + " maxEvents " + maxEvents);
-
-    // TODO: shouldReset is never used. See TT. Ask for Removal.
-    boolean shouldReset = false;
-    TezDependentTaskCompletionEvent[] events =
-        context.getCurrentDAG().
-            getVertex(taskAttemptID.getTaskID().getVertexID()).
-                getTaskAttemptCompletionEvents(taskAttemptID, fromEventIdx, maxEvents);
-
-    taskHeartbeatHandler.progressing(taskAttemptID);
-    pingContainerHeartbeatHandler(taskAttemptID);
-
-    // No filters for now. Only required events stored in a vertex.
-
-    return new TezTaskDependencyCompletionEventsUpdate(events,shouldReset);
-  }
-
-  @Override
   public ContainerTask getTask(ContainerContext containerContext)
       throws IOException {
 
@@ -370,17 +343,6 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
   }
 
   @Override
-  public void outputReady(TezTaskAttemptID taskAttemptId,
-      OutputContext outputContext) throws IOException {
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("AttemptId: " + taskAttemptId + " reported output context: "
-          + outputContext);
-    }
-    context.getEventHandler().handle(
-        new TaskAttemptEventOutputConsumable(taskAttemptId, outputContext));
-  }
-
-  @Override
   public ProceedToCompletionResponse
       proceedToCompletion(TezTaskAttemptID taskAttemptId) throws IOException {
 

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-dag/src/main/java/org/apache/tez/dag/app/dag/EdgeManager.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/EdgeManager.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/EdgeManager.java
index c476966..2779faf 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/EdgeManager.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/EdgeManager.java
@@ -20,9 +20,9 @@ package org.apache.tez.dag.app.dag;
 
 import java.util.List;
 
-import org.apache.tez.engine.newapi.events.DataMovementEvent;
-import org.apache.tez.engine.newapi.events.InputReadErrorEvent;
-import org.apache.tez.engine.newapi.events.InputFailedEvent;
+import org.apache.tez.engine.api.events.DataMovementEvent;
+import org.apache.tez.engine.api.events.InputFailedEvent;
+import org.apache.tez.engine.api.events.InputReadErrorEvent;
 
 public abstract class EdgeManager {
   

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskAttemptEventOutputConsumable.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskAttemptEventOutputConsumable.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskAttemptEventOutputConsumable.java
deleted file mode 100644
index f10209f..0000000
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskAttemptEventOutputConsumable.java
+++ /dev/null
@@ -1,36 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with this
- * work for additional information regarding copyright ownership. The ASF
- * licenses this file to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
- * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
- * License for the specific language governing permissions and limitations under
- * the License.
- */
-
-package org.apache.tez.dag.app.dag.event;
-
-import org.apache.tez.dag.records.TezTaskAttemptID;
-import org.apache.tez.engine.records.OutputContext;
-
-public class TaskAttemptEventOutputConsumable extends TaskAttemptEvent {
-
-  private final OutputContext outputContext;
-
-  public TaskAttemptEventOutputConsumable(TezTaskAttemptID id,
-      OutputContext outputContext) {
-    super(id, TaskAttemptEventType.TA_OUTPUT_CONSUMABLE);
-    this.outputContext = outputContext;
-  }
-
-  public OutputContext getOutputContext() {
-    return this.outputContext;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/BroadcastEdgeManager.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/BroadcastEdgeManager.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/BroadcastEdgeManager.java
index 71f17ac..b05a6f1 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/BroadcastEdgeManager.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/BroadcastEdgeManager.java
@@ -22,9 +22,9 @@ import java.util.List;
 
 import org.apache.tez.dag.app.dag.EdgeManager;
 import org.apache.tez.dag.app.dag.Vertex;
-import org.apache.tez.engine.newapi.events.DataMovementEvent;
-import org.apache.tez.engine.newapi.events.InputReadErrorEvent;
-import org.apache.tez.engine.newapi.events.InputFailedEvent;
+import org.apache.tez.engine.api.events.DataMovementEvent;
+import org.apache.tez.engine.api.events.InputFailedEvent;
+import org.apache.tez.engine.api.events.InputReadErrorEvent;
 
 public class BroadcastEdgeManager extends EdgeManager {
 

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/Edge.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/Edge.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/Edge.java
index 3605857..060a112 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/Edge.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/Edge.java
@@ -31,14 +31,14 @@ import org.apache.tez.dag.app.dag.Vertex;
 import org.apache.tez.dag.app.dag.event.TaskEventAddTezEvent;
 import org.apache.tez.dag.records.TezTaskAttemptID;
 import org.apache.tez.dag.records.TezTaskID;
+import org.apache.tez.engine.api.events.DataMovementEvent;
+import org.apache.tez.engine.api.events.InputFailedEvent;
+import org.apache.tez.engine.api.events.InputReadErrorEvent;
 import org.apache.tez.engine.api.impl.EventMetaData;
 import org.apache.tez.engine.api.impl.InputSpec;
 import org.apache.tez.engine.api.impl.OutputSpec;
 import org.apache.tez.engine.api.impl.TezEvent;
 import org.apache.tez.engine.api.impl.EventMetaData.EventProducerConsumerType;
-import org.apache.tez.engine.newapi.events.DataMovementEvent;
-import org.apache.tez.engine.newapi.events.InputFailedEvent;
-import org.apache.tez.engine.newapi.events.InputReadErrorEvent;
 
 public class Edge {
 

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/OneToOneEdgeManager.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/OneToOneEdgeManager.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/OneToOneEdgeManager.java
index 7c4743e..a916ad2 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/OneToOneEdgeManager.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/OneToOneEdgeManager.java
@@ -22,9 +22,9 @@ import java.util.List;
 
 import org.apache.tez.dag.app.dag.EdgeManager;
 import org.apache.tez.dag.app.dag.Vertex;
-import org.apache.tez.engine.newapi.events.DataMovementEvent;
-import org.apache.tez.engine.newapi.events.InputReadErrorEvent;
-import org.apache.tez.engine.newapi.events.InputFailedEvent;
+import org.apache.tez.engine.api.events.DataMovementEvent;
+import org.apache.tez.engine.api.events.InputFailedEvent;
+import org.apache.tez.engine.api.events.InputReadErrorEvent;
 
 public class OneToOneEdgeManager extends EdgeManager {
 

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/ScatterGatherEdgeManager.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/ScatterGatherEdgeManager.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/ScatterGatherEdgeManager.java
index 380b6b6..1d4df5b 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/ScatterGatherEdgeManager.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/ScatterGatherEdgeManager.java
@@ -22,9 +22,9 @@ import java.util.List;
 
 import org.apache.tez.dag.app.dag.EdgeManager;
 import org.apache.tez.dag.app.dag.Vertex;
-import org.apache.tez.engine.newapi.events.DataMovementEvent;
-import org.apache.tez.engine.newapi.events.InputReadErrorEvent;
-import org.apache.tez.engine.newapi.events.InputFailedEvent;
+import org.apache.tez.engine.api.events.DataMovementEvent;
+import org.apache.tez.engine.api.events.InputFailedEvent;
+import org.apache.tez.engine.api.events.InputReadErrorEvent;
 
 public class ScatterGatherEdgeManager extends EdgeManager {
 

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/ShuffleVertexManager.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/ShuffleVertexManager.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/ShuffleVertexManager.java
index b854a43..a0ed329 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/ShuffleVertexManager.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/ShuffleVertexManager.java
@@ -39,9 +39,9 @@ import org.apache.tez.dag.app.dag.VertexScheduler;
 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.engine.newapi.events.DataMovementEvent;
-import org.apache.tez.engine.newapi.events.InputFailedEvent;
-import org.apache.tez.engine.newapi.events.InputReadErrorEvent;
+import org.apache.tez.engine.api.events.DataMovementEvent;
+import org.apache.tez.engine.api.events.InputFailedEvent;
+import org.apache.tez.engine.api.events.InputReadErrorEvent;
 import org.apache.tez.engine.records.TezDependentTaskCompletionEvent;
 import org.apache.tez.mapreduce.hadoop.MRHelpers;
 

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/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 1ec1225..74005b7 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
@@ -106,14 +106,14 @@ 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.engine.api.events.DataMovementEvent;
+import org.apache.tez.engine.api.events.InputFailedEvent;
 import org.apache.tez.engine.api.events.TaskAttemptFailedEvent;
 import org.apache.tez.engine.api.events.TaskStatusUpdateEvent;
 import org.apache.tez.engine.api.impl.EventMetaData;
 import org.apache.tez.engine.api.impl.InputSpec;
 import org.apache.tez.engine.api.impl.OutputSpec;
 import org.apache.tez.engine.api.impl.TezEvent;
-import org.apache.tez.engine.newapi.events.DataMovementEvent;
-import org.apache.tez.engine.newapi.events.InputFailedEvent;
 import org.apache.tez.engine.records.TezDependentTaskCompletionEvent;
 
 import com.google.common.annotations.VisibleForTesting;

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-dag/src/main/java/org/apache/tez/engine/records/TezDependentTaskCompletionEvent.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/engine/records/TezDependentTaskCompletionEvent.java b/tez-dag/src/main/java/org/apache/tez/engine/records/TezDependentTaskCompletionEvent.java
new file mode 100644
index 0000000..7a4dd13
--- /dev/null
+++ b/tez-dag/src/main/java/org/apache/tez/engine/records/TezDependentTaskCompletionEvent.java
@@ -0,0 +1,228 @@
+/**
+ * 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.engine.records;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableUtils;
+import org.apache.tez.dag.records.TezTaskAttemptID;
+
+/**
+ * This is used to track task completion events on 
+ * job tracker. 
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+// TODO TEZAM3 This needs to be more generic. Maybe some kind of a serialized
+// blob - which can be interpretted by the Input plugin.
+public class TezDependentTaskCompletionEvent implements Writable {
+  @InterfaceAudience.Public
+  @InterfaceStability.Evolving
+  // TODO EVENTUALLY - Remove TIPFAILED state ?
+  static public enum Status {FAILED, KILLED, SUCCEEDED, OBSOLETE, TIPFAILED};
+    
+  private int eventId;
+  private int taskRunTime; // using int since runtime is the time difference
+  private TezTaskAttemptID taskAttemptId;
+  private long dataSize;
+  Status status;
+  byte[] userPayload;
+  // TODO TEZAM2 Get rid of the isMap field. Job specific type information can be determined from TaskAttemptId.getTaskType
+//  boolean isMap = false;
+  public static final TezDependentTaskCompletionEvent[] EMPTY_ARRAY = 
+    new TezDependentTaskCompletionEvent[0];
+
+  public TezDependentTaskCompletionEvent() {
+    taskAttemptId = new TezTaskAttemptID();
+  }
+  
+  /**
+   * Constructor. eventId should be created externally and incremented
+   * per event for each job. 
+   * @param eventId event id, event id should be unique and assigned in
+   *  incrementally, starting from 0. 
+   * @param taskAttemptId task id
+   * @param status task's status 
+   * @param taskTrackerHttp task tracker's host:port for http. 
+   */
+  public TezDependentTaskCompletionEvent(int eventId, 
+                             TezTaskAttemptID taskAttemptId,
+//                             boolean isMap,
+                             Status status, 
+                             int runTime,
+                             long dataSize){
+      
+    this.taskAttemptId = taskAttemptId;
+//    this.isMap = isMap;
+    this.eventId = eventId; 
+    this.status =status; 
+    this.taskRunTime = runTime;
+    this.dataSize = dataSize;
+  }
+  
+  public TezDependentTaskCompletionEvent clone() {
+    TezDependentTaskCompletionEvent clone = new TezDependentTaskCompletionEvent(
+        this.eventId, this.taskAttemptId, this.status, 
+        this.taskRunTime, this.dataSize);
+    
+    return clone;
+  }
+  
+  /**
+   * Returns event Id. 
+   * @return event id
+   */
+  public int getEventId() {
+    return eventId;
+  }
+
+  /**
+   * Returns task id. 
+   * @return task id
+   */
+  public TezTaskAttemptID getTaskAttemptID() {
+    return taskAttemptId;
+  }
+  
+  /**
+   * Returns enum Status.SUCESS or Status.FAILURE.
+   * @return task tracker status
+   */
+  public Status getStatus() {
+    return status;
+  }
+  
+  /**
+   * Returns time (in millisec) the task took to complete. 
+   */
+  public int getTaskRunTime() {
+    return taskRunTime;
+  }
+  
+  /**
+   * Return size of output produced by the task
+   */
+  public long getDataSize() {
+    return dataSize;
+  }
+  
+  /**
+   * @return user payload. Maybe null
+   */
+  public byte[] getUserPayload() {
+    return userPayload;
+  }
+
+  /**
+   * Set the task completion time
+   * @param taskCompletionTime time (in millisec) the task took to complete
+   */
+  protected void setTaskRunTime(int taskCompletionTime) {
+    this.taskRunTime = taskCompletionTime;
+  }
+
+  /**
+   * set event Id. should be assigned incrementally starting from 0. 
+   * @param eventId
+   */
+  public void setEventId(int eventId) {
+    this.eventId = eventId;
+  }
+
+  /**
+   * Sets task id. 
+   * @param taskId
+   */
+  public void setTaskAttemptID(TezTaskAttemptID taskId) {
+    this.taskAttemptId = taskId;
+  }
+  
+  /**
+   * Set task status. 
+   * @param status
+   */
+  public void setTaskStatus(Status status) {
+    this.status = status;
+  }
+  
+  /**
+   * Set the user payload
+   * @param userPayload
+   */
+  public void setUserPayload(byte[] userPayload) {
+    this.userPayload = userPayload;
+  }
+    
+  @Override
+  public String toString(){
+    StringBuffer buf = new StringBuffer(); 
+    buf.append("Task Id : "); 
+    buf.append(taskAttemptId); 
+    buf.append(", Status : ");  
+    buf.append(status.name());
+    return buf.toString();
+  }
+    
+  @Override
+  public boolean equals(Object o) {
+    // not counting userPayload as that is a piggyback mechanism
+    if(o == null)
+      return false;
+    if(o.getClass().equals(this.getClass())) {
+      TezDependentTaskCompletionEvent event = (TezDependentTaskCompletionEvent) o;
+      return this.eventId == event.getEventId()
+             && this.status.equals(event.getStatus())
+             && this.taskAttemptId.equals(event.getTaskAttemptID()) 
+             && this.taskRunTime == event.getTaskRunTime()
+             && this.dataSize == event.getDataSize();
+    }
+    return false;
+  }
+
+  @Override
+  public int hashCode() {
+    return toString().hashCode(); 
+  }
+
+  @Override
+  public void write(DataOutput out) throws IOException {
+    taskAttemptId.write(out);
+//    out.writeBoolean(isMap);
+    WritableUtils.writeEnum(out, status);
+    WritableUtils.writeVInt(out, taskRunTime);
+    WritableUtils.writeVInt(out, eventId);
+    WritableUtils.writeCompressedByteArray(out, userPayload);
+  }
+
+  @Override
+  public void readFields(DataInput in) throws IOException {
+    taskAttemptId.readFields(in);
+//    isMap = in.readBoolean();
+    status = WritableUtils.readEnum(in, Status.class);
+    taskRunTime = WritableUtils.readVInt(in);
+    eventId = WritableUtils.readVInt(in);
+    userPayload = WritableUtils.readCompressedByteArray(in);
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-dag/src/main/java/org/apache/tez/engine/records/TezTaskDependencyCompletionEventsUpdate.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/engine/records/TezTaskDependencyCompletionEventsUpdate.java b/tez-dag/src/main/java/org/apache/tez/engine/records/TezTaskDependencyCompletionEventsUpdate.java
new file mode 100644
index 0000000..13c9088
--- /dev/null
+++ b/tez-dag/src/main/java/org/apache/tez/engine/records/TezTaskDependencyCompletionEventsUpdate.java
@@ -0,0 +1,64 @@
+/**
+ * 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.engine.records;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hadoop.io.Writable;
+
+
+public class TezTaskDependencyCompletionEventsUpdate implements Writable {
+  TezDependentTaskCompletionEvent[] events;
+  boolean reset;
+
+  public TezTaskDependencyCompletionEventsUpdate() { }
+
+  public TezTaskDependencyCompletionEventsUpdate(
+      TezDependentTaskCompletionEvent[] events, boolean reset) {
+    this.events = events;
+    this.reset = reset;
+  }
+
+  public boolean shouldReset() {
+    return reset;
+  }
+
+  public TezDependentTaskCompletionEvent[] getDependentTaskCompletionEvents() {
+    return events;
+  }
+  
+  public void write(DataOutput out) throws IOException {
+    out.writeBoolean(reset);
+    out.writeInt(events.length);
+    for (TezDependentTaskCompletionEvent event : events) {
+      event.write(out);
+    }
+  }
+
+  public void readFields(DataInput in) throws IOException {
+    reset = in.readBoolean();
+    events = new TezDependentTaskCompletionEvent[in.readInt()];
+    for (int i = 0; i < events.length; ++i) {
+      events[i] = new TezDependentTaskCompletionEvent();
+      events[i].readFields(in);
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-dag/src/test/java/org/apache/tez/dag/app/rm/container/TestAMContainer.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/container/TestAMContainer.java b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/container/TestAMContainer.java
index d4eae9d..c2457e1 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/container/TestAMContainer.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/container/TestAMContainer.java
@@ -52,7 +52,6 @@ import org.apache.hadoop.yarn.api.records.Priority;
 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.tez.common.TezTaskContext;
 import org.apache.tez.dag.app.AppContext;
 import org.apache.tez.dag.app.ContainerHeartbeatHandler;
 import org.apache.tez.dag.app.ContainerContext;
@@ -66,6 +65,7 @@ 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.engine.api.impl.TaskSpec;
 import org.apache.tez.engine.common.security.JobTokenIdentifier;
 import org.apache.tez.engine.common.security.TokenCache;
 import org.junit.Test;
@@ -109,7 +109,7 @@ public class TestAMContainer {
     wc.verifyState(AMContainerState.RUNNING);
     wc.verifyNoOutgoingEvents();
     assertFalse(pulledTask.shouldDie());
-    assertEquals(wc.tezTaskContext.getTaskAttemptId(), pulledTask.getTask()
+    assertEquals(wc.taskSpec.getTaskAttemptID(), pulledTask.getTask()
         .getTaskAttemptID());
     assertEquals(wc.taskAttemptID, wc.amContainer.getRunningTaskAttempt());
     assertEquals(0, wc.amContainer.getQueuedTaskAttempts().size());
@@ -165,7 +165,7 @@ public class TestAMContainer {
     wc.verifyState(AMContainerState.RUNNING);
     wc.verifyNoOutgoingEvents();
     assertFalse(pulledTask.shouldDie());
-    assertEquals(wc.tezTaskContext.getTaskAttemptId(), pulledTask.getTask()
+    assertEquals(wc.taskSpec.getTaskAttemptID(), pulledTask.getTask()
         .getTaskAttemptID());
     assertEquals(wc.taskAttemptID, wc.amContainer.getRunningTaskAttempt());
     assertEquals(0, wc.amContainer.getQueuedTaskAttempts().size());
@@ -824,7 +824,7 @@ public class TestAMContainer {
     TezTaskID taskID;
     TezTaskAttemptID taskAttemptID;
 
-    TezTaskContext tezTaskContext;
+    TaskSpec taskSpec;
 
     public AMContainerImpl amContainer;
 
@@ -859,8 +859,8 @@ public class TestAMContainer {
       taskID = new TezTaskID(vertexID, 1);
       taskAttemptID = new TezTaskAttemptID(taskID, 1);
 
-      tezTaskContext = mock(TezTaskContext.class);
-      doReturn(taskAttemptID).when(tezTaskContext).getTaskAttemptId();
+      taskSpec = mock(TaskSpec.class);
+      doReturn(taskAttemptID).when(taskSpec).getTaskAttemptID();
 
       amContainer = new AMContainerImpl(container, chh, tal,
           appContext);
@@ -904,7 +904,7 @@ public class TestAMContainer {
     public void assignTaskAttempt(TezTaskAttemptID taID) {
       reset(eventHandler);
       amContainer.handle(new AMContainerEventAssignTA(containerID, taID,
-          tezTaskContext));
+          taskSpec));
     }
 
     public AMContainerTask pullTaskToRun() {

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-dist/src/main/assembly/tez-dist-full.xml
----------------------------------------------------------------------
diff --git a/tez-dist/src/main/assembly/tez-dist-full.xml b/tez-dist/src/main/assembly/tez-dist-full.xml
index 3176dd1..383eb43 100644
--- a/tez-dist/src/main/assembly/tez-dist-full.xml
+++ b/tez-dist/src/main/assembly/tez-dist-full.xml
@@ -24,8 +24,7 @@
     <moduleSet>
       <useAllReactorProjects>true</useAllReactorProjects>
       <includes>
-        <include>org.apache.tez:tez-dag-api</include>
-        <include>org.apache.tez:tez-engine-api</include>
+        <include>org.apache.tez:tez-api</include>
       </includes>
       <binaries>
         <outputDirectory>/</outputDirectory>
@@ -40,8 +39,7 @@
       <outputDirectory>/lib</outputDirectory>
       <!-- Exclude hadoop artifacts. They will be found via HADOOP* env -->
       <excludes>
-        <exclude>org.apache.tez:tez-dag-api</exclude>
-        <exclude>org.apache.tez:tez-engine-api</exclude>
+        <exclude>org.apache.tez:tez-api</exclude>
         <exclude>*:*:test-jar</exclude>
         <exclude>org.apache.hadoop:hadoop-common</exclude>
         <exclude>org.apache.hadoop:hadoop-auth</exclude>

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-dist/src/main/assembly/tez-dist.xml
----------------------------------------------------------------------
diff --git a/tez-dist/src/main/assembly/tez-dist.xml b/tez-dist/src/main/assembly/tez-dist.xml
index 01d0d40..aefb8a2 100644
--- a/tez-dist/src/main/assembly/tez-dist.xml
+++ b/tez-dist/src/main/assembly/tez-dist.xml
@@ -24,8 +24,7 @@
     <moduleSet>
       <useAllReactorProjects>true</useAllReactorProjects>
      <includes>
-        <include>org.apache.tez:tez-dag-api</include>
-        <include>org.apache.tez:tez-engine-api</include>
+       <include>org.apache.tez:tez-api</include>
       </includes>
       <binaries>
         <outputDirectory>/</outputDirectory>
@@ -42,8 +41,7 @@
       <useTransitiveFiltering>true</useTransitiveFiltering>
       <!-- Exclude hadoop artifacts. They will be found via HADOOP* env -->
       <excludes>
-        <exclude>org.apache.tez:tez-dag-api</exclude>
-        <exclude>org.apache.tez:tez-engine-api</exclude>
+        <exclude>org.apache.tez:tez-api</exclude>
         <exclude>*:*:test-jar</exclude>
         <exclude>org.apache.hadoop:hadoop-common</exclude>
         <exclude>org.apache.hadoop:hadoop-auth</exclude>

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine-api/findbugs-exclude.xml
----------------------------------------------------------------------
diff --git a/tez-engine-api/findbugs-exclude.xml b/tez-engine-api/findbugs-exclude.xml
deleted file mode 100644
index 5b11308..0000000
--- a/tez-engine-api/findbugs-exclude.xml
+++ /dev/null
@@ -1,16 +0,0 @@
-<!--
-  Licensed 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. See accompanying LICENSE file.
--->
-<FindBugsFilter>
-
-</FindBugsFilter>

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine-api/pom.xml
----------------------------------------------------------------------
diff --git a/tez-engine-api/pom.xml b/tez-engine-api/pom.xml
deleted file mode 100644
index b19e96b..0000000
--- a/tez-engine-api/pom.xml
+++ /dev/null
@@ -1,91 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-  Licensed 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. See accompanying LICENSE file.
--->
-
-<project xmlns="http://maven.apache.org/POM/4.0.0"
-         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
-  <modelVersion>4.0.0</modelVersion>
-  <parent>
-    <groupId>org.apache.tez</groupId>
-    <artifactId>tez</artifactId>
-    <version>0.2.0-SNAPSHOT</version>
-  </parent>
-  <artifactId>tez-engine-api</artifactId>
-  <dependencies>
-    <dependency>
-      <groupId>org.apache.tez</groupId>
-      <artifactId>tez-common</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-common</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-yarn-api</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-yarn-common</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>com.google.inject</groupId>
-      <artifactId>guice</artifactId>
-    </dependency>
-    <dependency>
-     <groupId>com.google.protobuf</groupId>
-     <artifactId>protobuf-java</artifactId>
-    </dependency>
-  </dependencies>
-
-  <build>
-    <plugins>
-      <plugin>
-        <groupId>org.apache.rat</groupId>
-        <artifactId>apache-rat-plugin</artifactId>
-        <configuration>
-        </configuration>
-      </plugin>
-      <plugin>
-        <groupId>org.apache.hadoop</groupId>
-        <artifactId>hadoop-maven-plugins</artifactId>
-        <executions>
-          <execution>
-            <id>compile-protoc</id>
-            <phase>generate-sources</phase>
-            <goals>
-              <goal>protoc</goal>
-            </goals>
-            <configuration>
-              <protocVersion>${protobuf.version}</protocVersion>
-              <protocCommand>${protoc.path}</protocCommand>
-              <imports>
-                <param>${basedir}/src/main/proto</param>
-              </imports>
-              <source>
-                <directory>${basedir}/src/main/proto</directory>
-                <includes>
-                  <include>Events.proto</include>
-                </includes>
-              </source>
-              <output>${project.build.directory}/generated-sources/java</output>
-            </configuration>
-          </execution>
-        </executions>
-      </plugin>
-    </plugins>
-  </build>
-
-</project>

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine-api/src/main/java/org/apache/tez/engine/api/Input.java
----------------------------------------------------------------------
diff --git a/tez-engine-api/src/main/java/org/apache/tez/engine/api/Input.java b/tez-engine-api/src/main/java/org/apache/tez/engine/api/Input.java
deleted file mode 100644
index 64c3834..0000000
--- a/tez-engine-api/src/main/java/org/apache/tez/engine/api/Input.java
+++ /dev/null
@@ -1,83 +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.engine.api;
-
-import java.io.IOException;
-
-import org.apache.hadoop.conf.Configuration;
-
-/**
- * {@link Input} represents a pipe through which an <em>tez</em> task
- * can get input key/value pairs.
- */
-public interface Input {
-  
-  /**
-   * Initialize <code>Input</code>.
-   * 
-   * @param conf job configuration
-   * @param master master process controlling the task
-   * @throws IOException
-   * @throws InterruptedException
-   */
-  public void initialize(Configuration conf, Master master) 
-      throws IOException, InterruptedException;
-  
-  /**
-   * Check if there is another key/value pair.
-   * 
-   * @return true if a key/value pair was read
-   * @throws IOException
-   * @throws InterruptedException
-   */
-  public boolean hasNext() throws IOException, InterruptedException;
-
-  /**
-   * Get the next key.
-   * 
-   * @return the current key or null if there is no current key
-   * @throws IOException
-   * @throws InterruptedException
-   */
-  public Object getNextKey() throws IOException, InterruptedException;
-  
-  /**
-   * Get the next values.
-   * 
-   * @return the object that was read
-   * @throws IOException
-   * @throws InterruptedException
-   */
-  public Iterable<Object> getNextValues() 
-      throws IOException, InterruptedException;
-  
-  /**
-   * The current progress of the {@link Input} through its data.
-   * 
-   * @return a number between 0.0 and 1.0 that is the fraction of the data read
-   * @throws IOException
-   * @throws InterruptedException
-   */
-  public float getProgress() throws IOException, InterruptedException;
-  
-  /**
-   * Close this <code>Input</code> for future operations.
-   */
-  public void close() throws IOException;
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine-api/src/main/java/org/apache/tez/engine/api/Master.java
----------------------------------------------------------------------
diff --git a/tez-engine-api/src/main/java/org/apache/tez/engine/api/Master.java b/tez-engine-api/src/main/java/org/apache/tez/engine/api/Master.java
deleted file mode 100644
index f3add9a..0000000
--- a/tez-engine-api/src/main/java/org/apache/tez/engine/api/Master.java
+++ /dev/null
@@ -1,39 +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.engine.api;
-
-import org.apache.hadoop.ipc.ProtocolInfo;
-import org.apache.hadoop.ipc.VersionedProtocol;
-import org.apache.tez.dag.records.TezTaskAttemptID;
-import org.apache.tez.engine.records.TezTaskDependencyCompletionEventsUpdate;
-
-/**
- * {@link Master} represents the master controlling the {@link Task}. 
- */
-@ProtocolInfo(protocolName = "Master", protocolVersion = 1)
-public interface Master extends VersionedProtocol {
-
-  // TODO TEZAM3 This likely needs to change to be a little more generic.
-  // Many output / input relationships cannot be captured via this. The current
-  // form works primarily works for the existing MR
-
-  TezTaskDependencyCompletionEventsUpdate getDependentTasksCompletionEvents(
-      int fromEventIdx, int maxEventsToFetch,
-      TezTaskAttemptID taskAttemptId);
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine-api/src/main/java/org/apache/tez/engine/api/Output.java
----------------------------------------------------------------------
diff --git a/tez-engine-api/src/main/java/org/apache/tez/engine/api/Output.java b/tez-engine-api/src/main/java/org/apache/tez/engine/api/Output.java
deleted file mode 100644
index daa80d0..0000000
--- a/tez-engine-api/src/main/java/org/apache/tez/engine/api/Output.java
+++ /dev/null
@@ -1,65 +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.engine.api;
-
-import java.io.IOException;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.tez.engine.records.OutputContext;
-
-/**
- * {@link Output} represents a pipe through which an <em>tez</em> task
- * can send out outputs.
- */
-public interface Output {
-
-  /**
-   * Initialize <code>Output</code>.
-   * 
-   * @param conf job configuration
-   * @param master master process controlling the task
-   * @throws IOException
-   * @throws InterruptedException
-   */
-  public void initialize(Configuration conf, Master master) 
-      throws IOException, InterruptedException;
-
-  /** 
-   * Writes a key/value pair.
-   *
-   * @param key the key to write.
-   * @param value the value to write.
-   * @throws IOException
-   */      
-  public void write(Object key, Object value
-                             ) throws IOException, InterruptedException;
-
-  /**
-   * Returns the OutputContext for the particular <code>Output</code>. 
-   * 
-   * @return the OutputContext for this Output if it exists, otherwise null.
-   */
-  public OutputContext getOutputContext();
-  
-  /** 
-   * Close this <code>Output</code> for future operations.
-   * 
-   * @throws IOException
-   */ 
-  public void close() throws IOException, InterruptedException;
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine-api/src/main/java/org/apache/tez/engine/api/Processor.java
----------------------------------------------------------------------
diff --git a/tez-engine-api/src/main/java/org/apache/tez/engine/api/Processor.java b/tez-engine-api/src/main/java/org/apache/tez/engine/api/Processor.java
deleted file mode 100644
index 550ee73..0000000
--- a/tez-engine-api/src/main/java/org/apache/tez/engine/api/Processor.java
+++ /dev/null
@@ -1,62 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.tez.engine.api;
-
-import java.io.IOException;
-
-import org.apache.hadoop.conf.Configuration;
-
-/**
- * {@link Processor} represents the <em>tez</em> entity responsible for
- * consuming {@link Input} and producing {@link Output}. 
- */
-public interface Processor {
-  
-  /**
-   * Initialize the <code>Processor</code>.
-   * 
-   * @param conf job-configuration
-   * @param master master process controlling the task
-   * @throws IOException 
-   * @throws InterruptedException
-   */
-  public void initialize(Configuration conf, Master master) 
-      throws IOException, InterruptedException;
-  
-  /**
-   * Process input data from <code>input</code> and 
-   * send it to <code>output</code>.
-   * 
-   * @param in input
-   * @param out output
-   * @param master master process controlling the task
-   * @throws IOException
-   * @throws InterruptedException
-   */
-  public void process(Input[] in, Output[]  out)
-      throws IOException, InterruptedException;
-
-  /**
-   * Close the {@link Processor}.
-   * 
-   * @throws IOException
-   * @throws InterruptedException
-   */
-  public void close() throws IOException, InterruptedException;
-
-}


[02/50] [abbrv] git commit: TEZ-470. Fix TestMRRJobsDAGApi to work with modified engine layer. (hitesh)

Posted by ss...@apache.org.
TEZ-470.  Fix TestMRRJobsDAGApi to work with modified engine layer. (hitesh)


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

Branch: refs/heads/master
Commit: 85a9d46e35756cacde115c64335268c6f55cbc3e
Parents: 6ca59ac
Author: Hitesh Shah <hi...@apache.org>
Authored: Fri Sep 20 14:43:13 2013 -0700
Committer: Hitesh Shah <hi...@apache.org>
Committed: Fri Sep 20 14:43:13 2013 -0700

----------------------------------------------------------------------
 .../apache/hadoop/mapred/YarnTezDagChild.java   |  3 ++
 .../apache/tez/mapreduce/TestMRRJobsDAGApi.java | 43 --------------------
 2 files changed, 3 insertions(+), 43 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/85a9d46e/tez-dag/src/main/java/org/apache/hadoop/mapred/YarnTezDagChild.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/hadoop/mapred/YarnTezDagChild.java b/tez-dag/src/main/java/org/apache/hadoop/mapred/YarnTezDagChild.java
index 5034262..c749850 100644
--- a/tez-dag/src/main/java/org/apache/hadoop/mapred/YarnTezDagChild.java
+++ b/tez-dag/src/main/java/org/apache/hadoop/mapred/YarnTezDagChild.java
@@ -421,6 +421,9 @@ public class YarnTezDagChild {
               currentTask.run();
               currentTask.close();
             }
+            LOG.info("Task completed"
+                + ", taskAttemptId=" + currentTaskAttemptID
+                + ", fatalErrorOccurred=" + currentTask.hadFatalError());
             // TODONEWTEZ check if task had a fatal error before
             // sending completed event
             if (!currentTask.hadFatalError()) {

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/85a9d46e/tez-mapreduce-tests/src/test/java/org/apache/tez/mapreduce/TestMRRJobsDAGApi.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce-tests/src/test/java/org/apache/tez/mapreduce/TestMRRJobsDAGApi.java b/tez-mapreduce-tests/src/test/java/org/apache/tez/mapreduce/TestMRRJobsDAGApi.java
index 88dbd97..eb20876 100644
--- a/tez-mapreduce-tests/src/test/java/org/apache/tez/mapreduce/TestMRRJobsDAGApi.java
+++ b/tez-mapreduce-tests/src/test/java/org/apache/tez/mapreduce/TestMRRJobsDAGApi.java
@@ -300,8 +300,6 @@ public class TestMRRJobsDAGApi {
         .valueOf(new Random().nextInt(100000))));
     InputSplitInfo inputSplitInfo = MRHelpers.generateInputSplits(stage1Conf,
         remoteStagingDir);
-    InputSplitInfo inputSplitInfo1 = MRHelpers.generateInputSplits(stage1Conf,
-        remoteStagingDir);
 
     DAG dag = new DAG("testMRRSleepJobDagSubmit");
     Vertex stage1Vertex = new Vertex("map", new ProcessorDescriptor(
@@ -312,14 +310,6 @@ public class TestMRRJobsDAGApi {
         ReduceProcessor.class.getName()).setUserPayload(
         MRHelpers.createUserPayloadFromConf(stage2Conf)),
         1, Resource.newInstance(256, 1));
-    Vertex stage11Vertex = new Vertex("map1", new ProcessorDescriptor(
-        MapProcessor.class.getName()).setUserPayload(
-        MRHelpers.createUserPayloadFromConf(stage1Conf)),
-        inputSplitInfo1.getNumTasks(),  Resource.newInstance(256, 1));
-    Vertex stage22Vertex = new Vertex("ireduce1", new ProcessorDescriptor(
-        ReduceProcessor.class.getName()).setUserPayload(
-        MRHelpers.createUserPayloadFromConf(stage22Conf)),
-        2, Resource.newInstance(256, 1));
     Vertex stage3Vertex = new Vertex("reduce", new ProcessorDescriptor(
         ReduceProcessor.class.getName()).setUserPayload(
         MRHelpers.createUserPayloadFromConf(stage3Conf)),
@@ -353,44 +343,23 @@ public class TestMRRJobsDAGApi {
             LocalResourceType.FILE, LocalResourceVisibility.APPLICATION));
     stage1LocalResources.putAll(commonLocalResources);
 
-    Map<String, LocalResource> stage11LocalResources = new HashMap<String, LocalResource>();
-    stage11LocalResources.put(
-        inputSplitInfo1.getSplitsFile().getName(),
-        createLocalResource(remoteFs, inputSplitInfo1.getSplitsFile(),
-            LocalResourceType.FILE, LocalResourceVisibility.APPLICATION));
-    stage11LocalResources.put(
-        inputSplitInfo1.getSplitsMetaInfoFile().getName(),
-        createLocalResource(remoteFs, inputSplitInfo1.getSplitsMetaInfoFile(),
-            LocalResourceType.FILE, LocalResourceVisibility.APPLICATION));
-    stage11LocalResources.putAll(commonLocalResources);
-
     stage1Vertex.setJavaOpts(MRHelpers.getMapJavaOpts(stage1Conf));
     stage1Vertex.setTaskLocationsHint(inputSplitInfo.getTaskLocationHints());
     stage1Vertex.setTaskLocalResources(stage1LocalResources);
     stage1Vertex.setTaskEnvironment(commonEnv);
 
-    stage11Vertex.setJavaOpts(MRHelpers.getMapJavaOpts(stage1Conf));
-    stage11Vertex.setTaskLocationsHint(inputSplitInfo1.getTaskLocationHints());
-    stage11Vertex.setTaskLocalResources(stage11LocalResources);
-    stage11Vertex.setTaskEnvironment(commonEnv);
     // TODO env, resources
 
     stage2Vertex.setJavaOpts(MRHelpers.getReduceJavaOpts(stage2Conf));
     stage2Vertex.setTaskLocalResources(commonLocalResources);
     stage2Vertex.setTaskEnvironment(commonEnv);
 
-    stage22Vertex.setJavaOpts(MRHelpers.getReduceJavaOpts(stage22Conf));
-    stage22Vertex.setTaskLocalResources(commonLocalResources);
-    stage22Vertex.setTaskEnvironment(commonEnv);
-
     stage3Vertex.setJavaOpts(MRHelpers.getReduceJavaOpts(stage3Conf));
     stage3Vertex.setTaskLocalResources(commonLocalResources);
     stage3Vertex.setTaskEnvironment(commonEnv);
 
     dag.addVertex(stage1Vertex);
-    dag.addVertex(stage11Vertex);
     dag.addVertex(stage2Vertex);
-    dag.addVertex(stage22Vertex);
     dag.addVertex(stage3Vertex);
 
     Edge edge1 = new Edge(stage1Vertex, stage2Vertex, new EdgeProperty(
@@ -398,26 +367,14 @@ public class TestMRRJobsDAGApi {
         SchedulingType.SEQUENTIAL, new OutputDescriptor(
         OnFileSortedOutput.class.getName()), new InputDescriptor(
                 ShuffledMergedInputLegacy.class.getName())));
-    Edge edge11 = new Edge(stage11Vertex, stage22Vertex, new EdgeProperty(
-        DataMovementType.SCATTER_GATHER, DataSourceType.PERSISTED, 
-        SchedulingType.SEQUENTIAL, new OutputDescriptor(
-        OnFileSortedOutput.class.getName()), new InputDescriptor(
-                ShuffledMergedInputLegacy.class.getName())));
     Edge edge2 = new Edge(stage2Vertex, stage3Vertex, new EdgeProperty(
         DataMovementType.SCATTER_GATHER, DataSourceType.PERSISTED, 
         SchedulingType.SEQUENTIAL, new OutputDescriptor(
         OnFileSortedOutput.class.getName()), new InputDescriptor(
                 ShuffledMergedInputLegacy.class.getName())));
-    Edge edge3 = new Edge(stage22Vertex, stage3Vertex, new EdgeProperty(
-        DataMovementType.SCATTER_GATHER, DataSourceType.PERSISTED, 
-        SchedulingType.SEQUENTIAL, new OutputDescriptor(
-        OnFileSortedOutput.class.getName()), new InputDescriptor(
-                ShuffledMergedInputLegacy.class.getName())));
 
     dag.addEdge(edge1);
-    dag.addEdge(edge11);
     dag.addEdge(edge2);
-    dag.addEdge(edge3);
 
     Map<String, LocalResource> amLocalResources =
         new HashMap<String, LocalResource>();


[40/50] [abbrv] Rename tez-engine-api to tez-runtime-api and tez-engine is split into 2: - tez-engine-library for user-visible Input/Output/Processor implementations - tez-engine-internals for framework internals

Posted by ss...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/PipelinedSorter.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/PipelinedSorter.java b/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/PipelinedSorter.java
deleted file mode 100644
index 1bf17a3..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/PipelinedSorter.java
+++ /dev/null
@@ -1,932 +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.engine.common.sort.impl;
-
-import java.io.DataOutputStream;
-import java.io.IOException;
-import java.io.OutputStream;
-import java.nio.BufferOverflowException;
-import java.nio.ByteBuffer;
-import java.nio.ByteOrder;
-import java.nio.IntBuffer;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.PriorityQueue;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.Future;
-
-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.Path;
-import org.apache.hadoop.io.DataInputBuffer;
-import org.apache.hadoop.io.HashComparator;
-import org.apache.hadoop.io.RawComparator;
-import org.apache.hadoop.util.IndexedSortable;
-import org.apache.hadoop.util.IndexedSorter;
-import org.apache.hadoop.util.Progress;
-import org.apache.tez.common.TezJobConfig;
-import org.apache.tez.engine.api.TezOutputContext;
-import org.apache.tez.engine.common.ConfigUtils;
-import org.apache.tez.engine.common.sort.impl.IFile.Writer;
-import org.apache.tez.engine.common.sort.impl.TezMerger.Segment;
-
-@SuppressWarnings({"unchecked", "rawtypes"})
-public class PipelinedSorter extends ExternalSorter {
-  
-  private static final Log LOG = LogFactory.getLog(PipelinedSorter.class);
-  
-  /**
-   * The size of each record in the index file for the map-outputs.
-   */
-  public static final int MAP_OUTPUT_INDEX_RECORD_LENGTH = 24;
-
-  private final static int APPROX_HEADER_LENGTH = 150;
-    
-  int partitionBits;
-  
-  private static final int PARTITION = 0;        // partition offset in acct
-  private static final int KEYSTART = 1;         // key offset in acct
-  private static final int VALSTART = 2;         // val offset in acct
-  private static final int VALLEN = 3;           // val len in acct
-  private static final int NMETA = 4;            // num meta ints
-  private static final int METASIZE = NMETA * 4; // size in bytes
-
-  // spill accounting
-  volatile Throwable sortSpillException = null;
-
-  int numSpills = 0;
-  int minSpillsForCombine;
-  private HashComparator hasher;
-  // SortSpans  
-  private SortSpan span;
-  private ByteBuffer largeBuffer;
-  // Merger
-  private SpanMerger merger; 
-  private ExecutorService sortmaster;
-
-  final ArrayList<TezSpillRecord> indexCacheList =
-    new ArrayList<TezSpillRecord>();
-  private int totalIndexCacheMemory;
-  private int indexCacheMemoryLimit;
-
-  public void initialize(TezOutputContext outputContext, Configuration conf, int numOutputs) throws IOException {
-    super.initialize(outputContext, conf, numOutputs);
-    
-    partitionBits = bitcount(partitions)+1;
-   
-    //sanity checks
-    final float spillper =
-      this.conf.getFloat(
-          TezJobConfig.TEZ_ENGINE_SORT_SPILL_PERCENT, 
-          TezJobConfig.DEFAULT_TEZ_ENGINE_SORT_SPILL_PERCENT);
-    final int sortmb = 
-        this.conf.getInt(
-            TezJobConfig.TEZ_ENGINE_IO_SORT_MB, 
-            TezJobConfig.DEFAULT_TEZ_ENGINE_IO_SORT_MB);
-    indexCacheMemoryLimit = this.conf.getInt(TezJobConfig.TEZ_ENGINE_INDEX_CACHE_MEMORY_LIMIT_BYTES,
-                                       TezJobConfig.DEFAULT_TEZ_ENGINE_INDEX_CACHE_MEMORY_LIMIT_BYTES);
-    if (spillper > (float)1.0 || spillper <= (float)0.0) {
-      throw new IOException("Invalid \"" + TezJobConfig.TEZ_ENGINE_SORT_SPILL_PERCENT +
-          "\": " + spillper);
-    }
-    if ((sortmb & 0x7FF) != sortmb) {
-      throw new IOException(
-          "Invalid \"" + TezJobConfig.TEZ_ENGINE_IO_SORT_MB + "\": " + sortmb);
-    }
-    
-    // buffers and accounting
-    int maxMemUsage = sortmb << 20;
-    maxMemUsage -= maxMemUsage % METASIZE;
-    largeBuffer = ByteBuffer.allocate(maxMemUsage);
-    LOG.info(TezJobConfig.TEZ_ENGINE_IO_SORT_MB + " = " + sortmb);
-    // TODO: configurable setting?
-    span = new SortSpan(largeBuffer, 1024*1024, 16);
-    merger = new SpanMerger(comparator);
-    final int sortThreads = 
-            this.conf.getInt(
-                TezJobConfig.TEZ_ENGINE_SORT_THREADS, 
-                TezJobConfig.DEFAULT_TEZ_ENGINE_SORT_THREADS);
-    sortmaster = Executors.newFixedThreadPool(sortThreads);
-
-    // k/v serialization    
-    if(comparator instanceof HashComparator) {
-      hasher = (HashComparator)comparator;
-      LOG.info("Using the HashComparator");
-    } else {
-      hasher = null;
-    }    
-    valSerializer.open(span.out);
-    keySerializer.open(span.out);
-    minSpillsForCombine = this.conf.getInt(TezJobConfig.TEZ_ENGINE_COMBINE_MIN_SPILLS, 3);
-  }
-
-  private int bitcount(int n) {
-    int bit = 0;
-    while(n!=0) {
-      bit++;
-      n >>= 1;
-    }
-    return bit;
-  }
-  
-  public void sort() throws IOException {
-    SortSpan newSpan = span.next();
-
-    if(newSpan == null) {
-      // sort in the same thread, do not wait for the thread pool
-      merger.add(span.sort(sorter, comparator));
-      spill();
-      int items = 1024*1024;
-      int perItem = 16;
-      if(span.length() != 0) {
-        items = span.length();
-        perItem = span.kvbuffer.limit()/items;
-        items = (largeBuffer.capacity())/(METASIZE+perItem);
-        if(items > 1024*1024) {
-            // our goal is to have 1M splits and sort early
-            items = 1024*1024;
-        }
-      }      
-      span = new SortSpan(largeBuffer, items, perItem);
-    } else {
-      // queue up the sort
-      SortTask task = new SortTask(span, sorter, comparator);
-      Future<SpanIterator> future = sortmaster.submit(task);
-      merger.add(future);
-      span = newSpan;
-    }
-    valSerializer.open(span.out);
-    keySerializer.open(span.out);
-  }
-
-  @Override
-  public void write(Object key, Object value) 
-      throws IOException {
-    collect(
-        key, value, partitioner.getPartition(key, value, partitions));
-  }
-
-  /**
-   * Serialize the key, value to intermediate storage.
-   * When this method returns, kvindex must refer to sufficient unused
-   * storage to store one METADATA.
-   */
-  synchronized void collect(Object key, Object value, final int partition
-                                   ) throws IOException {
-    if (key.getClass() != keyClass) {
-      throw new IOException("Type mismatch in key from map: expected "
-                            + keyClass.getName() + ", received "
-                            + key.getClass().getName());
-    }
-    if (value.getClass() != valClass) {
-      throw new IOException("Type mismatch in value from map: expected "
-                            + valClass.getName() + ", received "
-                            + value.getClass().getName());
-    }
-    if (partition < 0 || partition >= partitions) {
-      throw new IOException("Illegal partition for " + key + " (" +
-          partition + ")");
-    }
-    if(span.kvmeta.remaining() < METASIZE) {
-      this.sort();
-    }
-    int keystart = span.kvbuffer.position();
-    int valstart = -1;
-    int valend = -1;
-    try { 
-      keySerializer.serialize(key);
-      valstart = span.kvbuffer.position();      
-      valSerializer.serialize(value);
-      valend = span.kvbuffer.position();
-    } catch(BufferOverflowException overflow) {
-      // restore limit
-      span.kvbuffer.position(keystart);
-      this.sort();
-      // try again
-      this.collect(key, value, partition);
-      return;
-    }
-
-    int prefix = 0;
-
-    if(hasher != null) {
-      prefix = hasher.getHashCode(key);
-    }
-
-    prefix = (partition << (32 - partitionBits)) | (prefix >>> partitionBits);
-
-    /* maintain order as in PARTITION, KEYSTART, VALSTART, VALLEN */
-    span.kvmeta.put(prefix);
-    span.kvmeta.put(keystart);
-    span.kvmeta.put(valstart);
-    span.kvmeta.put(valend - valstart);
-    if((valstart - keystart) > span.keymax) {
-      span.keymax = (valstart - keystart);
-    }
-    if((valend - valstart) > span.valmax) {
-      span.valmax = (valend - valstart);
-    }
-    mapOutputRecordCounter.increment(1);
-    mapOutputByteCounter.increment(valend - keystart);
-  }
-
-  public void spill() throws IOException { 
-    // create spill file
-    final long size = largeBuffer.capacity() + 
-      (partitions * APPROX_HEADER_LENGTH);
-    final TezSpillRecord spillRec = new TezSpillRecord(partitions);
-    final Path filename =
-      mapOutputFile.getSpillFileForWrite(numSpills, size);    
-    FSDataOutputStream out = rfs.create(filename, true, 4096);
-
-    try {
-      merger.ready(); // wait for all the future results from sort threads
-      LOG.info("Spilling to " + filename.toString());
-      for (int i = 0; i < partitions; ++i) {
-        TezRawKeyValueIterator kvIter = merger.filter(i);
-        //write merged output to disk
-        long segmentStart = out.getPos();
-        Writer writer =
-          new Writer(conf, out, keyClass, valClass, codec,
-              spilledRecordsCounter);
-        writer.setRLE(merger.needsRLE());
-        if (combiner == null) {
-          while(kvIter.next()) {
-            writer.append(kvIter.getKey(), kvIter.getValue());
-          }
-        } else {          
-          runCombineProcessor(kvIter, writer);
-        }
-        //close
-        writer.close();
-
-        // record offsets
-        final TezIndexRecord rec = 
-            new TezIndexRecord(
-                segmentStart, 
-                writer.getRawLength(), 
-                writer.getCompressedLength());
-        spillRec.putIndex(rec, i);
-      }
-
-      Path indexFilename =
-        mapOutputFile.getSpillIndexFileForWrite(numSpills, partitions
-            * MAP_OUTPUT_INDEX_RECORD_LENGTH);
-      // TODO: cache
-      spillRec.writeToFile(indexFilename, conf);
-      ++numSpills;
-    } catch(InterruptedException ie) {
-      // TODO:the combiner has been interrupted
-    } finally {
-      out.close();
-    }
-  }
-
-  @Override
-  public void flush() throws IOException {
-    final String uniqueIdentifier = outputContext.getUniqueIdentifier();
-    Path finalOutputFile =
-        mapOutputFile.getOutputFileForWrite(0); //TODO
-    Path finalIndexFile =
-        mapOutputFile.getOutputIndexFileForWrite(0); //TODO
-
-    LOG.info("Starting flush of map output");
-    span.end();
-    merger.add(span.sort(sorter, comparator));
-    spill();
-    sortmaster.shutdown();
-
-    largeBuffer = null;
-
-    if(numSpills == 1) {
-      // someday be able to pass this directly to shuffle
-      // without writing to disk
-      final Path filename =
-          mapOutputFile.getSpillFile(0);
-      Path indexFilename =
-              mapOutputFile.getSpillIndexFile(0);
-      sameVolRename(filename, finalOutputFile);
-      sameVolRename(indexFilename, finalIndexFile);
-      return;
-    }
-    
-    //The output stream for the final single output file
-    FSDataOutputStream finalOut = rfs.create(finalOutputFile, true, 4096);
-
-    TezMerger.considerFinalMergeForProgress();
-
-    final TezSpillRecord spillRec = new TezSpillRecord(partitions);
-    final ArrayList<TezSpillRecord> indexCacheList = new ArrayList<TezSpillRecord>();
-
-    for(int i = 0; i < numSpills; i++) {
-      // TODO: build this cache before
-      Path indexFilename = mapOutputFile.getSpillIndexFile(i);
-      TezSpillRecord spillIndex = new TezSpillRecord(indexFilename, conf);
-      indexCacheList.add(spillIndex);
-    }
-    
-    for (int parts = 0; parts < partitions; parts++) {
-      //create the segments to be merged
-      List<Segment> segmentList =
-          new ArrayList<Segment>(numSpills);
-      for(int i = 0; i < numSpills; i++) {
-        Path spillFilename = mapOutputFile.getSpillFile(i);
-        TezIndexRecord indexRecord = indexCacheList.get(i).getIndex(parts);
-
-        Segment s =
-            new Segment(conf, rfs, spillFilename, indexRecord.getStartOffset(),
-                             indexRecord.getPartLength(), codec, true);
-        segmentList.add(i, s);
-      }
-
-      int mergeFactor = 
-              this.conf.getInt(TezJobConfig.TEZ_ENGINE_IO_SORT_FACTOR, 
-                  TezJobConfig.DEFAULT_TEZ_ENGINE_IO_SORT_FACTOR);
-      // sort the segments only if there are intermediate merges
-      boolean sortSegments = segmentList.size() > mergeFactor;
-      //merge
-      TezRawKeyValueIterator kvIter = TezMerger.merge(conf, rfs,
-                     keyClass, valClass, codec,
-                     segmentList, mergeFactor,
-                     new Path(uniqueIdentifier),
-                     (RawComparator)ConfigUtils.getIntermediateOutputKeyComparator(conf), 
-                     nullProgressable, sortSegments,
-                     null, spilledRecordsCounter,
-                     null); // Not using any Progress in TezMerger. Should just work.
-
-      //write merged output to disk
-      long segmentStart = finalOut.getPos();
-      Writer writer =
-          new Writer(conf, finalOut, keyClass, valClass, codec,
-                           spilledRecordsCounter);
-      writer.setRLE(merger.needsRLE());
-      if (combiner == null || numSpills < minSpillsForCombine) {
-        TezMerger.writeFile(kvIter, writer, nullProgressable, conf);
-      } else {
-        runCombineProcessor(kvIter, writer);
-      }
-
-      //close
-      writer.close();
-
-      // record offsets
-      final TezIndexRecord rec = 
-          new TezIndexRecord(
-              segmentStart, 
-              writer.getRawLength(), 
-              writer.getCompressedLength());
-      spillRec.putIndex(rec, parts);
-    }
-
-    spillRec.writeToFile(finalIndexFile, conf);
-    finalOut.close();
-    for(int i = 0; i < numSpills; i++) {
-      Path indexFilename = mapOutputFile.getSpillIndexFile(i);
-      Path spillFilename = mapOutputFile.getSpillFile(i);
-      rfs.delete(indexFilename,true);
-      rfs.delete(spillFilename,true);
-    }
-  }
-
-  public void close() { }
-
-  private interface PartitionedRawKeyValueIterator extends TezRawKeyValueIterator {
-    int getPartition();
-  }
-
-  private class BufferStreamWrapper extends OutputStream 
-  {
-    private final ByteBuffer out;
-    public BufferStreamWrapper(ByteBuffer out) {
-      this.out = out;
-    }
-    
-    @Override
-    public void write(int b) throws IOException { out.put((byte)b); }
-    @Override
-    public void write(byte[] b) throws IOException { out.put(b); }
-    @Override
-    public void write(byte[] b, int off, int len) throws IOException { out.put(b, off, len); }
-  }
-
-  protected class InputByteBuffer extends DataInputBuffer {
-    private byte[] buffer = new byte[256]; 
-    private ByteBuffer wrapped = ByteBuffer.wrap(buffer);
-    private void resize(int length) {
-      if(length > buffer.length) {
-        buffer = new byte[length];
-        wrapped = ByteBuffer.wrap(buffer);
-      }
-      wrapped.limit(length);
-    }
-    public void reset(ByteBuffer b, int start, int length) {
-      resize(length);
-      b.position(start);
-      b.get(buffer, 0, length);
-      super.reset(buffer, 0, length);
-    }
-    // clone-ish function
-    public void reset(DataInputBuffer clone) {
-      byte[] data = clone.getData();
-      int start = clone.getPosition();
-      int length = clone.getLength();
-      resize(length);
-      System.arraycopy(data, start, buffer, 0, length);
-      super.reset(buffer, 0, length);
-    }
-  }
-
-  private class SortSpan  implements IndexedSortable {
-    final IntBuffer kvmeta;
-    final ByteBuffer kvbuffer;
-    final DataOutputStream out;    
-    private RawComparator comparator; 
-    final int imeta[] = new int[NMETA];
-    final int jmeta[] = new int[NMETA];
-    int keymax = 1;
-    int valmax = 1;
-    private int i,j;
-    private byte[] ki;
-    private byte[] kj;
-    private int index = 0;
-    private InputByteBuffer hay = new InputByteBuffer();
-    private long eq = 0;
-
-    public SortSpan(ByteBuffer source, int maxItems, int perItem) {
-      int capacity = source.remaining(); 
-      int metasize = METASIZE*maxItems;
-      int dataSize = maxItems * perItem;
-      if(capacity < (metasize+dataSize)) {
-        // try to allocate less meta space, because we have sample data
-        metasize = METASIZE*(capacity/(perItem+METASIZE));
-      }
-      ByteBuffer reserved = source.duplicate();
-      reserved.mark();
-      LOG.info("reserved.remaining() = "+reserved.remaining());
-      LOG.info("reserved.size = "+metasize);
-      reserved.position(metasize);
-      kvbuffer = reserved.slice();
-      reserved.flip();
-      reserved.limit(metasize);
-      kvmeta = reserved
-                .slice()
-                .order(ByteOrder.nativeOrder())
-               .asIntBuffer();
-      out = new DataOutputStream(
-              new BufferStreamWrapper(kvbuffer));
-    }
-
-    public SpanIterator sort(IndexedSorter sorter, RawComparator comparator) {
-    	this.comparator = comparator;
-      ki = new byte[keymax];
-      kj = new byte[keymax];
-      LOG.info("begin sorting Span"+index + " ("+length()+")");
-      if(length() > 1) {
-        sorter.sort(this, 0, length(), nullProgressable);
-      }
-      LOG.info("done sorting Span"+index);
-      return new SpanIterator(this);
-    }
-
-    int offsetFor(int i) {
-      return (i * NMETA);
-    }
-
-    public void swap(final int mi, final int mj) {
-      final int kvi = offsetFor(mi);
-      final int kvj = offsetFor(mj);
-
-      kvmeta.position(kvi); kvmeta.get(imeta);
-      kvmeta.position(kvj); kvmeta.get(jmeta);
-      kvmeta.position(kvj); kvmeta.put(imeta);
-      kvmeta.position(kvi); kvmeta.put(jmeta);
-
-      if(i == mi || j == mj) i = -1;
-      if(i == mi || j == mj) j = -1;
-    }
-
-    public int compare(final int mi, final int mj) {
-      final int kvi = offsetFor(mi);
-      final int kvj = offsetFor(mj);
-      final int kvip = kvmeta.get(kvi + PARTITION);
-      final int kvjp = kvmeta.get(kvj + PARTITION);
-      // sort by partition      
-      if (kvip != kvjp) {
-        return kvip - kvjp;
-      }
-      
-      final int istart = kvmeta.get(kvi + KEYSTART);
-      final int jstart = kvmeta.get(kvj + KEYSTART);
-      final int ilen   = kvmeta.get(kvi + VALSTART) - istart;
-      final int jlen   = kvmeta.get(kvj + VALSTART) - jstart;
-
-      kvbuffer.position(istart);
-      kvbuffer.get(ki, 0, ilen);
-      kvbuffer.position(jstart);
-      kvbuffer.get(kj, 0, jlen);
-      // sort by key
-      final int cmp = comparator.compare(ki, 0, ilen, kj, 0, jlen);
-      if(cmp == 0) eq++;
-      return cmp;
-    }
-
-    public SortSpan next() {
-      ByteBuffer remaining = end();
-      if(remaining != null) {
-        int items = length();
-        int perItem = kvbuffer.position()/items;
-        SortSpan newSpan = new SortSpan(remaining, items, perItem);
-        newSpan.index = index+1;
-        return newSpan;
-      }
-      return null;
-    }
-
-    public int length() {
-      return kvmeta.limit()/NMETA;
-    }
-
-    public ByteBuffer end() {
-      ByteBuffer remaining = kvbuffer.duplicate();
-      remaining.position(kvbuffer.position());
-      remaining = remaining.slice();
-      kvbuffer.limit(kvbuffer.position());
-      kvmeta.limit(kvmeta.position());
-      int items = length();
-      if(items == 0) {
-        return null;
-      }
-      int perItem = kvbuffer.position()/items;
-      LOG.info(String.format("Span%d.length = %d, perItem = %d", index, length(), perItem));
-      if(remaining.remaining() < NMETA+perItem) {
-        return null;
-      }
-      return remaining;
-    }
-
-    private int compareInternal(DataInputBuffer needle, int needlePart, int index) {
-      int cmp = 0;
-      int keystart;
-      int valstart;
-      int partition;
-      partition = kvmeta.get(span.offsetFor(index) + PARTITION);
-      if(partition != needlePart) {
-          cmp = (partition-needlePart);
-      } else {
-        keystart = kvmeta.get(span.offsetFor(index) + KEYSTART);
-        valstart = kvmeta.get(span.offsetFor(index) + VALSTART);
-        // hay is allocated ahead of time
-        hay.reset(kvbuffer, keystart, valstart - keystart);
-        cmp = comparator.compare(hay.getData(), 
-            hay.getPosition(), hay.getLength(),
-            needle.getData(), 
-            needle.getPosition(), needle.getLength());
-      }
-      return cmp;
-    }
-    
-    public long getEq() {
-      return eq;
-    }
-    
-    @Override
-    public String toString() {
-        return String.format("Span[%d,%d]", NMETA*kvmeta.capacity(), kvbuffer.limit());
-    }
-  }
-
-  private class SpanIterator implements PartitionedRawKeyValueIterator, Comparable<SpanIterator> {
-    private int kvindex = -1;
-    private int maxindex;
-    private IntBuffer kvmeta;
-    private ByteBuffer kvbuffer;
-    private SortSpan span;
-    private InputByteBuffer key = new InputByteBuffer();
-    private InputByteBuffer value = new InputByteBuffer();
-    private Progress progress = new Progress();
-
-    private final int minrun = (1 << 4);
-
-    public SpanIterator(SortSpan span) {
-      this.kvmeta = span.kvmeta;
-      this.kvbuffer = span.kvbuffer;
-      this.span = span;
-      this.maxindex = (kvmeta.limit()/NMETA) - 1;
-    }
-
-    public DataInputBuffer getKey() throws IOException {
-      final int keystart = kvmeta.get(span.offsetFor(kvindex) + KEYSTART);
-      final int valstart = kvmeta.get(span.offsetFor(kvindex) + VALSTART);
-      key.reset(kvbuffer, keystart, valstart - keystart);
-      return key;
-    }
-
-    public DataInputBuffer getValue() throws IOException {
-      final int valstart = kvmeta.get(span.offsetFor(kvindex) + VALSTART);
-      final int vallen = kvmeta.get(span.offsetFor(kvindex) + VALLEN);
-      value.reset(kvbuffer, valstart, vallen);
-      return value;
-    }
-
-    public boolean next() throws IOException {
-      // caveat: since we use this as a comparable in the merger 
-      if(kvindex == maxindex) return false;
-      if(kvindex % 100 == 0) {
-          progress.set((kvindex-maxindex) / maxindex);
-      }
-      kvindex += 1;
-      return true;
-    }
-
-    public void close() throws IOException {
-    }
-
-    public Progress getProgress() { 
-      return progress;
-    }
-
-    public int getPartition() {
-      final int partition = kvmeta.get(span.offsetFor(kvindex) + PARTITION);
-      return partition;
-    }
-
-    public int size() {
-      return (maxindex - kvindex);
-    }
-
-    public int compareTo(SpanIterator other) {
-      try {
-        return span.compareInternal(other.getKey(), other.getPartition(), kvindex);
-      } catch(IOException ie) {
-        // since we're not reading off disk, how could getKey() throw exceptions?
-      }
-      return -1;
-    }
-    
-    @Override
-    public String toString() {
-        return String.format("SpanIterator<%d:%d> (span=%s)", kvindex, maxindex, span.toString());
-    }
-
-    /**
-     * bisect returns the next insertion point for a given raw key, skipping keys
-     * which are <= needle using a binary search instead of a linear comparison.
-     * This is massively efficient when long strings of identical keys occur.
-     * @param needle 
-     * @param needlePart
-     * @return
-     */
-    int bisect(DataInputBuffer needle, int needlePart) {
-      int start = kvindex;
-      int end = maxindex-1;
-      int mid = start;
-      int cmp = 0;
-
-      if(end - start < minrun) {
-        return 0;
-      }
-
-      if(span.compareInternal(needle, needlePart, start) > 0) {
-        return kvindex;
-      }
-      
-      // bail out early if we haven't got a min run 
-      if(span.compareInternal(needle, needlePart, start+minrun) > 0) {
-        return 0;
-      }
-
-      if(span.compareInternal(needle, needlePart, end) < 0) {
-        return end - kvindex;
-      }
-      
-      boolean found = false;
-      
-      // we sort 100k items, the max it can do is 20 loops, but break early
-      for(int i = 0; start < end && i < 16; i++) {
-        mid = start + (end - start)/2;
-        cmp = span.compareInternal(needle, needlePart, mid);
-        if(cmp == 0) {
-          start = mid;
-          found = true;
-        } else if(cmp < 0) {
-          start = mid; 
-          found = true;
-        }
-        if(cmp > 0) {
-          end = mid;
-        }
-      }
-
-      if(found) {
-        return start - kvindex;
-      }
-      return 0;
-    }
-  }
-
-  private class SortTask implements Callable<SpanIterator> {
-    private final SortSpan sortable;
-    private final IndexedSorter sorter;
-    private final RawComparator comparator;
-    
-    public SortTask(SortSpan sortable, 
-              IndexedSorter sorter, RawComparator comparator) {
-        this.sortable = sortable;
-        this.sorter = sorter;
-        this.comparator = comparator;
-    }
-
-    public SpanIterator call() {
-      return sortable.sort(sorter, comparator);
-    }
-  }
-
-  private class PartitionFilter implements TezRawKeyValueIterator {
-    private final PartitionedRawKeyValueIterator iter;
-    private int partition;
-    private boolean dirty = false;
-    public PartitionFilter(PartitionedRawKeyValueIterator iter) {
-      this.iter = iter;
-    }
-    public DataInputBuffer getKey() throws IOException { return iter.getKey(); }
-    public DataInputBuffer getValue() throws IOException { return iter.getValue(); }
-    public void close() throws IOException { }
-    public Progress getProgress() {
-      return new Progress();
-    }
-    public boolean next() throws IOException {
-      if(dirty || iter.next()) { 
-        int prefix = iter.getPartition();
-
-        if((prefix >>> (32 - partitionBits)) == partition) {
-          dirty = false; // we found what we were looking for, good
-          return true;
-        } else if(!dirty) {
-          dirty = true; // we did a lookahead and failed to find partition
-        }
-      }
-      return false;
-    }
-
-    public void reset(int partition) {
-      this.partition = partition;
-    }
-
-    public int getPartition() {
-      return this.partition;
-    }
-  }
-
-  private class SpanHeap extends java.util.PriorityQueue<SpanIterator> {
-    public SpanHeap() {
-      super(256);
-    }
-    /**
-     * {@link PriorityQueue}.poll() by a different name 
-     * @return
-     */
-    public SpanIterator pop() {
-      return this.poll();
-    }
-  }
-
-  private class SpanMerger implements PartitionedRawKeyValueIterator {
-    private final RawComparator comparator;
-    InputByteBuffer key = new InputByteBuffer();
-    InputByteBuffer value = new InputByteBuffer();
-    int partition;
-
-    private ArrayList< Future<SpanIterator>> futures = new ArrayList< Future<SpanIterator>>();
-
-    private SpanHeap heap = new SpanHeap();
-    private PartitionFilter partIter;
-
-    private int gallop = 0;
-    private SpanIterator horse;
-    private long total = 0;
-    private long count = 0;
-    private long eq = 0;
-    
-    public SpanMerger(RawComparator comparator) {
-      this.comparator = comparator;
-      partIter = new PartitionFilter(this);
-    }
-
-    public void add(SpanIterator iter) throws IOException{
-      if(iter.next()) {
-        heap.add(iter);
-      }
-    }
-
-    public void add(Future<SpanIterator> iter) throws IOException{
-      this.futures.add(iter);
-    }
-
-    public boolean ready() throws IOException, InterruptedException {
-      try {
-        SpanIterator iter = null;
-        while(this.futures.size() > 0) {
-          Future<SpanIterator> futureIter = this.futures.remove(0);
-          iter = futureIter.get();
-          this.add(iter);
-        }
-        
-        StringBuilder sb = new StringBuilder();
-        for(SpanIterator sp: heap) {
-            sb.append(sp.toString());
-            sb.append(",");
-            total += sp.span.length();
-            eq += sp.span.getEq();
-        }
-        LOG.info("Heap = " + sb.toString());
-        return true;
-      } catch(Exception e) {
-        LOG.info(e.toString());
-        return false;
-      }
-    }
-
-    private SpanIterator pop() throws IOException {
-      if(gallop > 0) {
-        gallop--;
-        return horse;
-      }
-      SpanIterator current = heap.pop();
-      SpanIterator next = heap.peek();
-      if(next != null && current != null &&
-        ((Object)horse) == ((Object)current)) {
-        // TODO: a better threshold check
-        gallop = current.bisect(next.getKey(), next.getPartition())-1;
-      }
-      horse = current;
-      return current;
-    }
-    
-    public boolean needsRLE() {
-      return (eq > 0.1 * total);
-    }
-    
-    private SpanIterator peek() throws IOException {
-    	if(gallop > 0) {
-            return horse;
-        }
-    	return heap.peek();
-    }
-
-    public boolean next() throws IOException {
-      SpanIterator current = pop();
-
-      if(current != null) {
-        // keep local copies, since add() will move it all out
-        key.reset(current.getKey());
-        value.reset(current.getValue());
-        partition = current.getPartition();
-        if(gallop <= 0) {
-          this.add(current);
-        } else {
-          // galloping
-          current.next();
-        }
-        return true;
-      }
-      return false;
-    }
-
-    public DataInputBuffer getKey() throws IOException { return key; }
-    public DataInputBuffer getValue() throws IOException { return value; }
-    public int getPartition() { return partition; }
-
-    public void close() throws IOException {
-    }
-
-    public Progress getProgress() {
-      // TODO
-      return new Progress();
-    }
-
-    public TezRawKeyValueIterator filter(int partition) {
-      partIter.reset(partition);
-      return partIter;
-    }
-
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/TezIndexRecord.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/TezIndexRecord.java b/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/TezIndexRecord.java
deleted file mode 100644
index ac0267c..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/TezIndexRecord.java
+++ /dev/null
@@ -1,45 +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.engine.common.sort.impl;
-
-public class TezIndexRecord {
-  private long startOffset;
-  private long rawLength;
-  private long partLength;
-
-  public TezIndexRecord() { }
-
-  public TezIndexRecord(long startOffset, long rawLength, long partLength) {
-    this.startOffset = startOffset;
-    this.rawLength = rawLength;
-    this.partLength = partLength;
-  }
-
-  public long getStartOffset() {
-    return startOffset;
-  }
-
-  public long getRawLength() {
-    return rawLength;
-  }
-
-  public long getPartLength() {
-    return partLength;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/TezMerger.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/TezMerger.java b/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/TezMerger.java
deleted file mode 100644
index 7815569..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/TezMerger.java
+++ /dev/null
@@ -1,798 +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.engine.common.sort.impl;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.List;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.ChecksumFileSystem;
-import org.apache.hadoop.fs.FSDataInputStream;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.LocalDirAllocator;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.DataInputBuffer;
-import org.apache.hadoop.io.RawComparator;
-import org.apache.hadoop.io.compress.CompressionCodec;
-import org.apache.hadoop.util.PriorityQueue;
-import org.apache.hadoop.util.Progress;
-import org.apache.hadoop.util.Progressable;
-import org.apache.tez.common.Constants;
-import org.apache.tez.common.TezJobConfig;
-import org.apache.tez.common.counters.TezCounter;
-import org.apache.tez.engine.common.sort.impl.IFile.Reader;
-import org.apache.tez.engine.common.sort.impl.IFile.Writer;
-
-/**
- * Merger is an utility class used by the Map and Reduce tasks for merging
- * both their memory and disk segments
- */
-@InterfaceAudience.Private
-@InterfaceStability.Unstable
-@SuppressWarnings({"unchecked", "rawtypes"})
-public class TezMerger {  
-  private static final Log LOG = LogFactory.getLog(TezMerger.class);
-
-  
-  // Local directories
-  private static LocalDirAllocator lDirAlloc = 
-    new LocalDirAllocator(TezJobConfig.LOCAL_DIRS);
-
-  public static
-  TezRawKeyValueIterator merge(Configuration conf, FileSystem fs,
-                            Class keyClass, Class valueClass, 
-                            CompressionCodec codec,
-                            Path[] inputs, boolean deleteInputs, 
-                            int mergeFactor, Path tmpDir,
-                            RawComparator comparator, Progressable reporter,
-                            TezCounter readsCounter,
-                            TezCounter writesCounter,
-                            Progress mergePhase)
-  throws IOException {
-    return 
-      new MergeQueue(conf, fs, inputs, deleteInputs, codec, comparator, 
-                           reporter, null).merge(keyClass, valueClass,
-                                           mergeFactor, tmpDir,
-                                           readsCounter, writesCounter, 
-                                           mergePhase);
-  }
-
-  public static 
-  TezRawKeyValueIterator merge(Configuration conf, FileSystem fs,
-                            Class keyClass, Class valueClass, 
-                            CompressionCodec codec,
-                            Path[] inputs, boolean deleteInputs, 
-                            int mergeFactor, Path tmpDir,
-                            RawComparator comparator,
-                            Progressable reporter,
-                            TezCounter readsCounter,
-                            TezCounter writesCounter,
-                            TezCounter mergedMapOutputsCounter,
-                            Progress mergePhase)
-  throws IOException {
-    return 
-      new MergeQueue(conf, fs, inputs, deleteInputs, codec, comparator, 
-                           reporter, mergedMapOutputsCounter).merge(
-                                           keyClass, valueClass,
-                                           mergeFactor, tmpDir,
-                                           readsCounter, writesCounter,
-                                           mergePhase);
-  }
-  
-  public static
-  TezRawKeyValueIterator merge(Configuration conf, FileSystem fs, 
-                            Class keyClass, Class valueClass, 
-                            List<Segment> segments, 
-                            int mergeFactor, Path tmpDir,
-                            RawComparator comparator, Progressable reporter,
-                            TezCounter readsCounter,
-                            TezCounter writesCounter,
-                            Progress mergePhase)
-      throws IOException {
-    return merge(conf, fs, keyClass, valueClass, segments, mergeFactor, tmpDir,
-                 comparator, reporter, false, readsCounter, writesCounter,
-                 mergePhase);
-  }
-
-  public static <K extends Object, V extends Object>
-  TezRawKeyValueIterator merge(Configuration conf, FileSystem fs,
-                            Class keyClass, Class valueClass,
-                            List<Segment> segments,
-                            int mergeFactor, Path tmpDir,
-                            RawComparator comparator, Progressable reporter,
-                            boolean sortSegments,
-                            TezCounter readsCounter,
-                            TezCounter writesCounter,
-                            Progress mergePhase)
-      throws IOException {
-    return new MergeQueue(conf, fs, segments, comparator, reporter,
-                           sortSegments).merge(keyClass, valueClass,
-                                               mergeFactor, tmpDir,
-                                               readsCounter, writesCounter,
-                                               mergePhase);
-  }
-
-  public static <K extends Object, V extends Object>
-  TezRawKeyValueIterator merge(Configuration conf, FileSystem fs,
-                            Class keyClass, Class valueClass,
-                            CompressionCodec codec,
-                            List<Segment> segments,
-                            int mergeFactor, Path tmpDir,
-                            RawComparator comparator, Progressable reporter,
-                            boolean sortSegments,
-                            TezCounter readsCounter,
-                            TezCounter writesCounter,
-                            Progress mergePhase)
-      throws IOException {
-    return new MergeQueue(conf, fs, segments, comparator, reporter,
-                           sortSegments, codec).merge(keyClass, valueClass,
-                                               mergeFactor, tmpDir,
-                                               readsCounter, writesCounter,
-                                               mergePhase);
-  }
-
-  public static <K extends Object, V extends Object>
-    TezRawKeyValueIterator merge(Configuration conf, FileSystem fs,
-                            Class keyClass, Class valueClass,
-                            List<Segment> segments,
-                            int mergeFactor, int inMemSegments, Path tmpDir,
-                            RawComparator comparator, Progressable reporter,
-                            boolean sortSegments,
-                            TezCounter readsCounter,
-                            TezCounter writesCounter,
-                            Progress mergePhase)
-      throws IOException {
-    return new MergeQueue(conf, fs, segments, comparator, reporter,
-                           sortSegments).merge(keyClass, valueClass,
-                                               mergeFactor, inMemSegments,
-                                               tmpDir,
-                                               readsCounter, writesCounter,
-                                               mergePhase);
-  }
-
-
-  static <K extends Object, V extends Object>
-  TezRawKeyValueIterator merge(Configuration conf, FileSystem fs,
-                          Class keyClass, Class valueClass,
-                          CompressionCodec codec,
-                          List<Segment> segments,
-                          int mergeFactor, int inMemSegments, Path tmpDir,
-                          RawComparator comparator, Progressable reporter,
-                          boolean sortSegments,
-                          TezCounter readsCounter,
-                          TezCounter writesCounter,
-                          Progress mergePhase)
-    throws IOException {
-  return new MergeQueue(conf, fs, segments, comparator, reporter,
-                         sortSegments, codec).merge(keyClass, valueClass,
-                                             mergeFactor, inMemSegments,
-                                             tmpDir,
-                                             readsCounter, writesCounter,
-                                             mergePhase);
-}
-
-  public static <K extends Object, V extends Object>
-  void writeFile(TezRawKeyValueIterator records, Writer writer, 
-                 Progressable progressable, Configuration conf) 
-  throws IOException {
-    long progressBar = 
-        conf.getLong(TezJobConfig.RECORDS_BEFORE_PROGRESS, 
-            TezJobConfig.DEFAULT_RECORDS_BEFORE_PROGRESS);
-    long recordCtr = 0;
-    while(records.next()) {
-      writer.append(records.getKey(), records.getValue());
-      
-      if (((recordCtr++) % progressBar) == 0) {
-        progressable.progress();
-      }
-    }
-}
-
-  @InterfaceAudience.Private
-  @InterfaceStability.Unstable
-  public static class Segment<K extends Object, V extends Object> {
-    Reader reader = null;
-    final DataInputBuffer key = new DataInputBuffer();
-    
-    Configuration conf = null;
-    FileSystem fs = null;
-    Path file = null;
-    boolean preserve = false;
-    CompressionCodec codec = null;
-    long segmentOffset = 0;
-    long segmentLength = -1;
-    
-    TezCounter mapOutputsCounter = null;
-
-    public Segment(Configuration conf, FileSystem fs, Path file,
-                   CompressionCodec codec, boolean preserve)
-    throws IOException {
-      this(conf, fs, file, codec, preserve, null);
-    }
-
-    public Segment(Configuration conf, FileSystem fs, Path file,
-                   CompressionCodec codec, boolean preserve,
-                   TezCounter mergedMapOutputsCounter)
-  throws IOException {
-      this(conf, fs, file, 0, fs.getFileStatus(file).getLen(), codec, preserve, 
-           mergedMapOutputsCounter);
-    }
-
-    public Segment(Configuration conf, FileSystem fs, Path file,
-                   long segmentOffset, long segmentLength,
-                   CompressionCodec codec,
-                   boolean preserve) throws IOException {
-      this(conf, fs, file, segmentOffset, segmentLength, codec, preserve, null);
-    }
-
-    public Segment(Configuration conf, FileSystem fs, Path file,
-        long segmentOffset, long segmentLength, CompressionCodec codec,
-        boolean preserve, TezCounter mergedMapOutputsCounter)
-    throws IOException {
-      this.conf = conf;
-      this.fs = fs;
-      this.file = file;
-      this.codec = codec;
-      this.preserve = preserve;
-
-      this.segmentOffset = segmentOffset;
-      this.segmentLength = segmentLength;
-      
-      this.mapOutputsCounter = mergedMapOutputsCounter;
-    }
-    
-    public Segment(Reader reader, boolean preserve) {
-      this(reader, preserve, null);
-    }
-    
-    public Segment(Reader reader, boolean preserve, 
-                   TezCounter mapOutputsCounter) {
-      this.reader = reader;
-      this.preserve = preserve;
-      
-      this.segmentLength = reader.getLength();
-      
-      this.mapOutputsCounter = mapOutputsCounter;
-    }
-
-    void init(TezCounter readsCounter) throws IOException {
-      if (reader == null) {
-        FSDataInputStream in = fs.open(file);
-        in.seek(segmentOffset);
-        reader = new Reader(conf, in, segmentLength, codec, readsCounter);
-      }
-      
-      if (mapOutputsCounter != null) {
-        mapOutputsCounter.increment(1);
-      }
-    }
-    
-    boolean inMemory() {
-      return fs == null;
-    }
-    
-    DataInputBuffer getKey() { return key; }
-
-    DataInputBuffer getValue(DataInputBuffer value) throws IOException {
-      nextRawValue(value);
-      return value;
-    }
-
-    public long getLength() { 
-      return (reader == null) ?
-        segmentLength : reader.getLength();
-    }
-    
-    boolean nextRawKey() throws IOException {
-      return reader.nextRawKey(key);
-    }
-
-    void nextRawValue(DataInputBuffer value) throws IOException {
-      reader.nextRawValue(value);
-    }
-
-    void closeReader() throws IOException {
-      if (reader != null) {
-        reader.close();
-        reader = null;
-      }
-    }
-    
-    void close() throws IOException {
-      closeReader();
-      if (!preserve && fs != null) {
-        fs.delete(file, false);
-      }
-    }
-
-    public long getPosition() throws IOException {
-      return reader.getPosition();
-    }
-
-    // This method is used by BackupStore to extract the 
-    // absolute position after a reset
-    long getActualPosition() throws IOException {
-      return segmentOffset + reader.getPosition();
-    }
-
-    Reader getReader() {
-      return reader;
-    }
-    
-    // This method is used by BackupStore to reinitialize the
-    // reader to start reading from a different segment offset
-    void reinitReader(int offset) throws IOException {
-      if (!inMemory()) {
-        closeReader();
-        segmentOffset = offset;
-        segmentLength = fs.getFileStatus(file).getLen() - segmentOffset;
-        init(null);
-      }
-    }
-  }
-  
-  // Boolean variable for including/considering final merge as part of sort
-  // phase or not. This is true in map task, false in reduce task. It is
-  // used in calculating mergeProgress.
-  static boolean includeFinalMerge = false;
-  
-  /**
-   * Sets the boolean variable includeFinalMerge to true. Called from
-   * map task before calling merge() so that final merge of map task
-   * is also considered as part of sort phase.
-   */
-  public static void considerFinalMergeForProgress() {
-    includeFinalMerge = true;
-  }
-  
-  private static class MergeQueue<K extends Object, V extends Object> 
-  extends PriorityQueue<Segment> implements TezRawKeyValueIterator {
-    Configuration conf;
-    FileSystem fs;
-    CompressionCodec codec;
-    
-    List<Segment> segments = new ArrayList<Segment>();
-    
-    RawComparator comparator;
-    
-    private long totalBytesProcessed;
-    private float progPerByte;
-    private Progress mergeProgress = new Progress();
-    
-    Progressable reporter;
-    
-    DataInputBuffer key;
-    final DataInputBuffer value = new DataInputBuffer();
-    final DataInputBuffer diskIFileValue = new DataInputBuffer();
-    
-    Segment minSegment;
-    Comparator<Segment> segmentComparator =   
-      new Comparator<Segment>() {
-      public int compare(Segment o1, Segment o2) {
-        if (o1.getLength() == o2.getLength()) {
-          return 0;
-        }
-
-        return o1.getLength() < o2.getLength() ? -1 : 1;
-      }
-    };
-
-    public MergeQueue(Configuration conf, FileSystem fs, 
-                      Path[] inputs, boolean deleteInputs, 
-                      CompressionCodec codec, RawComparator comparator,
-                      Progressable reporter, 
-                      TezCounter mergedMapOutputsCounter) 
-    throws IOException {
-      this.conf = conf;
-      this.fs = fs;
-      this.codec = codec;
-      this.comparator = comparator;
-      this.reporter = reporter;
-      
-      for (Path file : inputs) {
-        LOG.debug("MergeQ: adding: " + file);
-        segments.add(new Segment(conf, fs, file, codec, !deleteInputs, 
-                                       (file.toString().endsWith(
-                                           Constants.MERGED_OUTPUT_PREFIX) ? 
-                                        null : mergedMapOutputsCounter)));
-      }
-      
-      // Sort segments on file-lengths
-      Collections.sort(segments, segmentComparator); 
-    }
-    
-    public MergeQueue(Configuration conf, FileSystem fs, 
-        List<Segment> segments, RawComparator comparator,
-        Progressable reporter, boolean sortSegments) {
-      this.conf = conf;
-      this.fs = fs;
-      this.comparator = comparator;
-      this.segments = segments;
-      this.reporter = reporter;
-      if (sortSegments) {
-        Collections.sort(segments, segmentComparator);
-      }
-    }
-
-    public MergeQueue(Configuration conf, FileSystem fs,
-        List<Segment> segments, RawComparator comparator,
-        Progressable reporter, boolean sortSegments, CompressionCodec codec) {
-      this(conf, fs, segments, comparator, reporter, sortSegments);
-      this.codec = codec;
-    }
-
-    public void close() throws IOException {
-      Segment segment;
-      while((segment = pop()) != null) {
-        segment.close();
-      }
-    }
-
-    public DataInputBuffer getKey() throws IOException {
-      return key;
-    }
-
-    public DataInputBuffer getValue() throws IOException {
-      return value;
-    }
-
-    private void adjustPriorityQueue(Segment reader) throws IOException{
-      long startPos = reader.getPosition();
-      boolean hasNext = reader.nextRawKey();
-      long endPos = reader.getPosition();
-      totalBytesProcessed += endPos - startPos;
-      mergeProgress.set(totalBytesProcessed * progPerByte);
-      if (hasNext) {
-        adjustTop();
-      } else {
-        pop();
-        reader.close();
-      }
-    }
-
-    public boolean next() throws IOException {
-      if (size() == 0)
-        return false;
-
-      if (minSegment != null) {
-        //minSegment is non-null for all invocations of next except the first
-        //one. For the first invocation, the priority queue is ready for use
-        //but for the subsequent invocations, first adjust the queue 
-        adjustPriorityQueue(minSegment);
-        if (size() == 0) {
-          minSegment = null;
-          return false;
-        }
-      }
-      minSegment = top();
-      if (!minSegment.inMemory()) {
-        //When we load the value from an inmemory segment, we reset
-        //the "value" DIB in this class to the inmem segment's byte[].
-        //When we load the value bytes from disk, we shouldn't use
-        //the same byte[] since it would corrupt the data in the inmem
-        //segment. So we maintain an explicit DIB for value bytes
-        //obtained from disk, and if the current segment is a disk
-        //segment, we reset the "value" DIB to the byte[] in that (so 
-        //we reuse the disk segment DIB whenever we consider
-        //a disk segment).
-        value.reset(diskIFileValue.getData(), diskIFileValue.getLength());
-      }
-      long startPos = minSegment.getPosition();
-      key = minSegment.getKey();
-      minSegment.getValue(value);
-      long endPos = minSegment.getPosition();
-      totalBytesProcessed += endPos - startPos;
-      mergeProgress.set(totalBytesProcessed * progPerByte);
-      return true;
-    }
-
-    protected boolean lessThan(Object a, Object b) {
-      DataInputBuffer key1 = ((Segment)a).getKey();
-      DataInputBuffer key2 = ((Segment)b).getKey();
-      int s1 = key1.getPosition();
-      int l1 = key1.getLength() - s1;
-      int s2 = key2.getPosition();
-      int l2 = key2.getLength() - s2;
-
-      return comparator.compare(key1.getData(), s1, l1, key2.getData(), s2, l2) < 0;
-    }
-    
-    public TezRawKeyValueIterator merge(Class keyClass, Class valueClass,
-                                     int factor, Path tmpDir,
-                                     TezCounter readsCounter,
-                                     TezCounter writesCounter,
-                                     Progress mergePhase)
-        throws IOException {
-      return merge(keyClass, valueClass, factor, 0, tmpDir,
-                   readsCounter, writesCounter, mergePhase);
-    }
-
-    TezRawKeyValueIterator merge(Class keyClass, Class valueClass,
-                                     int factor, int inMem, Path tmpDir,
-                                     TezCounter readsCounter,
-                                     TezCounter writesCounter,
-                                     Progress mergePhase)
-        throws IOException {
-      LOG.info("Merging " + segments.size() + " sorted segments");
-
-      /*
-       * If there are inMemory segments, then they come first in the segments
-       * list and then the sorted disk segments. Otherwise(if there are only
-       * disk segments), then they are sorted segments if there are more than
-       * factor segments in the segments list.
-       */
-      int numSegments = segments.size();
-      int origFactor = factor;
-      int passNo = 1;
-      if (mergePhase != null) {
-        mergeProgress = mergePhase;
-      }
-
-      long totalBytes = computeBytesInMerges(factor, inMem);
-      if (totalBytes != 0) {
-        progPerByte = 1.0f / (float)totalBytes;
-      }
-      
-      //create the MergeStreams from the sorted map created in the constructor
-      //and dump the final output to a file
-      do {
-        //get the factor for this pass of merge. We assume in-memory segments
-        //are the first entries in the segment list and that the pass factor
-        //doesn't apply to them
-        factor = getPassFactor(factor, passNo, numSegments - inMem);
-        if (1 == passNo) {
-          factor += inMem;
-        }
-        List<Segment> segmentsToMerge =
-          new ArrayList<Segment>();
-        int segmentsConsidered = 0;
-        int numSegmentsToConsider = factor;
-        long startBytes = 0; // starting bytes of segments of this merge
-        while (true) {
-          //extract the smallest 'factor' number of segments  
-          //Call cleanup on the empty segments (no key/value data)
-          List<Segment> mStream = 
-            getSegmentDescriptors(numSegmentsToConsider);
-          for (Segment segment : mStream) {
-            // Initialize the segment at the last possible moment;
-            // this helps in ensuring we don't use buffers until we need them
-            segment.init(readsCounter);
-            long startPos = segment.getPosition();
-            boolean hasNext = segment.nextRawKey();
-            long endPos = segment.getPosition();
-            
-            if (hasNext) {
-              startBytes += endPos - startPos;
-              segmentsToMerge.add(segment);
-              segmentsConsidered++;
-            }
-            else {
-              segment.close();
-              numSegments--; //we ignore this segment for the merge
-            }
-          }
-          //if we have the desired number of segments
-          //or looked at all available segments, we break
-          if (segmentsConsidered == factor || 
-              segments.size() == 0) {
-            break;
-          }
-            
-          numSegmentsToConsider = factor - segmentsConsidered;
-        }
-        
-        //feed the streams to the priority queue
-        initialize(segmentsToMerge.size());
-        clear();
-        for (Segment segment : segmentsToMerge) {
-          put(segment);
-        }
-        
-        //if we have lesser number of segments remaining, then just return the
-        //iterator, else do another single level merge
-        if (numSegments <= factor) {
-          if (!includeFinalMerge) { // for reduce task
-
-            // Reset totalBytesProcessed and recalculate totalBytes from the
-            // remaining segments to track the progress of the final merge.
-            // Final merge is considered as the progress of the reducePhase,
-            // the 3rd phase of reduce task.
-            totalBytesProcessed = 0;
-            totalBytes = 0;
-            for (int i = 0; i < segmentsToMerge.size(); i++) {
-              totalBytes += segmentsToMerge.get(i).getLength();
-            }
-          }
-          if (totalBytes != 0) //being paranoid
-            progPerByte = 1.0f / (float)totalBytes;
-          
-          totalBytesProcessed += startBytes;         
-          if (totalBytes != 0)
-            mergeProgress.set(totalBytesProcessed * progPerByte);
-          else
-            mergeProgress.set(1.0f); // Last pass and no segments left - we're done
-          
-          LOG.info("Down to the last merge-pass, with " + numSegments + 
-                   " segments left of total size: " +
-                   (totalBytes - totalBytesProcessed) + " bytes");
-          return this;
-        } else {
-          LOG.info("Merging " + segmentsToMerge.size() + 
-                   " intermediate segments out of a total of " + 
-                   (segments.size()+segmentsToMerge.size()));
-          
-          long bytesProcessedInPrevMerges = totalBytesProcessed;
-          totalBytesProcessed += startBytes;
-
-          //we want to spread the creation of temp files on multiple disks if 
-          //available under the space constraints
-          long approxOutputSize = 0; 
-          for (Segment s : segmentsToMerge) {
-            approxOutputSize += s.getLength() + 
-                                ChecksumFileSystem.getApproxChkSumLength(
-                                s.getLength());
-          }
-          Path tmpFilename = 
-            new Path(tmpDir, "intermediate").suffix("." + passNo);
-
-          Path outputFile =  lDirAlloc.getLocalPathForWrite(
-                                              tmpFilename.toString(),
-                                              approxOutputSize, conf);
-
-          Writer writer = 
-            new Writer(conf, fs, outputFile, keyClass, valueClass, codec,
-                             writesCounter);
-          writeFile(this, writer, reporter, conf);
-          writer.close();
-          
-          //we finished one single level merge; now clean up the priority 
-          //queue
-          this.close();
-
-          // Add the newly create segment to the list of segments to be merged
-          Segment tempSegment = 
-            new Segment(conf, fs, outputFile, codec, false);
-
-          // Insert new merged segment into the sorted list
-          int pos = Collections.binarySearch(segments, tempSegment,
-                                             segmentComparator);
-          if (pos < 0) {
-            // binary search failed. So position to be inserted at is -pos-1
-            pos = -pos-1;
-          }
-          segments.add(pos, tempSegment);
-          numSegments = segments.size();
-          
-          // Subtract the difference between expected size of new segment and 
-          // actual size of new segment(Expected size of new segment is
-          // inputBytesOfThisMerge) from totalBytes. Expected size and actual
-          // size will match(almost) if combiner is not called in merge.
-          long inputBytesOfThisMerge = totalBytesProcessed -
-                                       bytesProcessedInPrevMerges;
-          totalBytes -= inputBytesOfThisMerge - tempSegment.getLength();
-          if (totalBytes != 0) {
-            progPerByte = 1.0f / (float)totalBytes;
-          }
-          
-          passNo++;
-        }
-        //we are worried about only the first pass merge factor. So reset the 
-        //factor to what it originally was
-        factor = origFactor;
-      } while(true);
-    }
-    
-    /**
-     * Determine the number of segments to merge in a given pass. Assuming more
-     * than factor segments, the first pass should attempt to bring the total
-     * number of segments - 1 to be divisible by the factor - 1 (each pass
-     * takes X segments and produces 1) to minimize the number of merges.
-     */
-    private int getPassFactor(int factor, int passNo, int numSegments) {
-      if (passNo > 1 || numSegments <= factor || factor == 1) 
-        return factor;
-      int mod = (numSegments - 1) % (factor - 1);
-      if (mod == 0)
-        return factor;
-      return mod + 1;
-    }
-    
-    /** Return (& remove) the requested number of segment descriptors from the
-     * sorted map.
-     */
-    private List<Segment> getSegmentDescriptors(int numDescriptors) {
-      if (numDescriptors > segments.size()) {
-        List<Segment> subList = new ArrayList<Segment>(segments);
-        segments.clear();
-        return subList;
-      }
-      
-      List<Segment> subList = 
-        new ArrayList<Segment>(segments.subList(0, numDescriptors));
-      for (int i=0; i < numDescriptors; ++i) {
-        segments.remove(0);
-      }
-      return subList;
-    }
-    
-    /**
-     * Compute expected size of input bytes to merges, will be used in
-     * calculating mergeProgress. This simulates the above merge() method and
-     * tries to obtain the number of bytes that are going to be merged in all
-     * merges(assuming that there is no combiner called while merging).
-     * @param factor mapreduce.task.io.sort.factor
-     * @param inMem  number of segments in memory to be merged
-     */
-    long computeBytesInMerges(int factor, int inMem) {
-      int numSegments = segments.size();
-      List<Long> segmentSizes = new ArrayList<Long>(numSegments);
-      long totalBytes = 0;
-      int n = numSegments - inMem;
-      // factor for 1st pass
-      int f = getPassFactor(factor, 1, n) + inMem;
-      n = numSegments;
- 
-      for (int i = 0; i < numSegments; i++) {
-        // Not handling empty segments here assuming that it would not affect
-        // much in calculation of mergeProgress.
-        segmentSizes.add(segments.get(i).getLength());
-      }
-      
-      // If includeFinalMerge is true, allow the following while loop iterate
-      // for 1 more iteration. This is to include final merge as part of the
-      // computation of expected input bytes of merges
-      boolean considerFinalMerge = includeFinalMerge;
-      
-      while (n > f || considerFinalMerge) {
-        if (n <=f ) {
-          considerFinalMerge = false;
-        }
-        long mergedSize = 0;
-        f = Math.min(f, segmentSizes.size());
-        for (int j = 0; j < f; j++) {
-          mergedSize += segmentSizes.remove(0);
-        }
-        totalBytes += mergedSize;
-        
-        // insert new size into the sorted list
-        int pos = Collections.binarySearch(segmentSizes, mergedSize);
-        if (pos < 0) {
-          pos = -pos-1;
-        }
-        segmentSizes.add(pos, mergedSize);
-        
-        n -= (f-1);
-        f = factor;
-      }
-
-      return totalBytes;
-    }
-
-    public Progress getProgress() {
-      return mergeProgress;
-    }
-
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/TezRawKeyValueIterator.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/TezRawKeyValueIterator.java b/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/TezRawKeyValueIterator.java
deleted file mode 100644
index 39cffcb..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/TezRawKeyValueIterator.java
+++ /dev/null
@@ -1,70 +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.engine.common.sort.impl;
-
-import java.io.IOException;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.io.DataInputBuffer;
-import org.apache.hadoop.util.Progress;
-
-/**
- * <code>TezRawKeyValueIterator</code> is an iterator used to iterate over
- * the raw keys and values during sort/merge of intermediate data. 
- */
-@InterfaceAudience.Private
-@InterfaceStability.Unstable
-public interface TezRawKeyValueIterator {
-  /** 
-   * Gets the current raw key.
-   * 
-   * @return Gets the current raw key as a DataInputBuffer
-   * @throws IOException
-   */
-  DataInputBuffer getKey() throws IOException;
-  
-  /** 
-   * Gets the current raw value.
-   * 
-   * @return Gets the current raw value as a DataInputBuffer 
-   * @throws IOException
-   */
-  DataInputBuffer getValue() throws IOException;
-  
-  /** 
-   * Sets up the current key and value (for getKey and getValue).
-   * 
-   * @return <code>true</code> if there exists a key/value, 
-   *         <code>false</code> otherwise. 
-   * @throws IOException
-   */
-  boolean next() throws IOException;
-  
-  /** 
-   * Closes the iterator so that the underlying streams can be closed.
-   * 
-   * @throws IOException
-   */
-  void close() throws IOException;
-  
-  /** Gets the Progress object; this has a float (0.0 - 1.0) 
-   * indicating the bytes processed by the iterator so far
-   */
-  Progress getProgress();
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/TezSpillRecord.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/TezSpillRecord.java b/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/TezSpillRecord.java
deleted file mode 100644
index 19fbd7f..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/TezSpillRecord.java
+++ /dev/null
@@ -1,146 +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.engine.common.sort.impl;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.nio.LongBuffer;
-import java.util.zip.CheckedInputStream;
-import java.util.zip.CheckedOutputStream;
-import java.util.zip.Checksum;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.ChecksumException;
-import org.apache.hadoop.fs.FSDataInputStream;
-import org.apache.hadoop.fs.FSDataOutputStream;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.IOUtils;
-import org.apache.hadoop.util.PureJavaCrc32;
-import org.apache.tez.common.Constants;
-
-public class TezSpillRecord {
-
-  /** Backing store */
-  private final ByteBuffer buf;
-  /** View of backing storage as longs */
-  private final LongBuffer entries;
-
-  public TezSpillRecord(int numPartitions) {
-    buf = ByteBuffer.allocate(
-        numPartitions * Constants.MAP_OUTPUT_INDEX_RECORD_LENGTH);
-    entries = buf.asLongBuffer();
-  }
-
-  public TezSpillRecord(Path indexFileName, Configuration job) throws IOException {
-    this(indexFileName, job, null);
-  }
-
-  public TezSpillRecord(Path indexFileName, Configuration job, String expectedIndexOwner)
-    throws IOException {
-    this(indexFileName, job, new PureJavaCrc32(), expectedIndexOwner);
-  }
-
-  public TezSpillRecord(Path indexFileName, Configuration job, Checksum crc,
-                     String expectedIndexOwner)
-      throws IOException {
-
-    final FileSystem rfs = FileSystem.getLocal(job).getRaw();
-    final FSDataInputStream in = rfs.open(indexFileName);
-    try {
-      final long length = rfs.getFileStatus(indexFileName).getLen();
-      final int partitions = 
-          (int) length / Constants.MAP_OUTPUT_INDEX_RECORD_LENGTH;
-      final int size = partitions * Constants.MAP_OUTPUT_INDEX_RECORD_LENGTH;
-
-      buf = ByteBuffer.allocate(size);
-      if (crc != null) {
-        crc.reset();
-        CheckedInputStream chk = new CheckedInputStream(in, crc);
-        IOUtils.readFully(chk, buf.array(), 0, size);
-        if (chk.getChecksum().getValue() != in.readLong()) {
-          throw new ChecksumException("Checksum error reading spill index: " +
-                                indexFileName, -1);
-        }
-      } else {
-        IOUtils.readFully(in, buf.array(), 0, size);
-      }
-      entries = buf.asLongBuffer();
-    } finally {
-      in.close();
-    }
-  }
-
-  /**
-   * Return number of IndexRecord entries in this spill.
-   */
-  public int size() {
-    return entries.capacity() / (Constants.MAP_OUTPUT_INDEX_RECORD_LENGTH / 8);
-  }
-
-  /**
-   * Get spill offsets for given partition.
-   */
-  public TezIndexRecord getIndex(int partition) {
-    final int pos = partition * Constants.MAP_OUTPUT_INDEX_RECORD_LENGTH / 8;
-    return new TezIndexRecord(entries.get(pos), entries.get(pos + 1),
-                           entries.get(pos + 2));
-  }
-
-  /**
-   * Set spill offsets for given partition.
-   */
-  public void putIndex(TezIndexRecord rec, int partition) {
-    final int pos = partition * Constants.MAP_OUTPUT_INDEX_RECORD_LENGTH / 8;
-    entries.put(pos, rec.getStartOffset());
-    entries.put(pos + 1, rec.getRawLength());
-    entries.put(pos + 2, rec.getPartLength());
-  }
-
-  /**
-   * Write this spill record to the location provided.
-   */
-  public void writeToFile(Path loc, Configuration job)
-      throws IOException {
-    writeToFile(loc, job, new PureJavaCrc32());
-  }
-
-  public void writeToFile(Path loc, Configuration job, Checksum crc)
-      throws IOException {
-    final FileSystem rfs = FileSystem.getLocal(job).getRaw();
-    CheckedOutputStream chk = null;
-    final FSDataOutputStream out = rfs.create(loc);
-    try {
-      if (crc != null) {
-        crc.reset();
-        chk = new CheckedOutputStream(out, crc);
-        chk.write(buf.array());
-        out.writeLong(chk.getChecksum().getValue());
-      } else {
-        out.write(buf.array());
-      }
-    } finally {
-      if (chk != null) {
-        chk.close();
-      } else {
-        out.close();
-      }
-    }
-  }
-
-}


[17/50] [abbrv] TEZ-443. Merge tez-dag-api and tez-engine-api into a single module - tez-api (part of TEZ-398). (sseth)

Posted by ss...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine-api/src/main/java/org/apache/tez/engine/api/Task.java
----------------------------------------------------------------------
diff --git a/tez-engine-api/src/main/java/org/apache/tez/engine/api/Task.java b/tez-engine-api/src/main/java/org/apache/tez/engine/api/Task.java
deleted file mode 100644
index 8dc9601..0000000
--- a/tez-engine-api/src/main/java/org/apache/tez/engine/api/Task.java
+++ /dev/null
@@ -1,79 +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.engine.api;
-
-import java.io.IOException;
-
-import org.apache.hadoop.conf.Configuration;
-
-/**
- * {@link Task} is the base <em>tez</em> entity which consumes 
- * input key/value pairs through an {@link Input} pipe, 
- * processes them via a {@link Processor} and 
- * produces output key/value pairs for an {@link Output} pipe.
- */
-public interface Task {
-  
-  /**
-   * Initialize the {@link Task}.
-   * 
-   * @param conf Tez configuration. TODO: Replace with a context object.
-   * @param master master controlling the task
-   * @throws IOException
-   * @throws InterruptedException
-   */
-  public void initialize(Configuration conf, byte[] userPayload,
-      Master master) throws IOException, InterruptedException;
-  
-  /**
-   * Get {@link Input} of the task.
-   * @return <code>Input</code> of the task
-   */
-  public Input[] getInputs();
-
-  /**
-   * Get {@link Processor}s of the task.
-   * @return <code>Processor</code>s of the task
-   */
-  public Processor getProcessor();
-
-  /**
-   * Get {@link Output}s of the task.
-   * @return <code>Output</code>s of the task
-   */
-  public Output[] getOutputs();
-
-  /**
-   * Run the {@link Task}.
-   * 
-   * @throws IOException
-   * @throws InterruptedException
-   */
-  public void run() throws IOException, InterruptedException;
-  
-  /**
-   * Stop the {@link Task}.
-   * 
-   * @throws IOException
-   * @throws InterruptedException
-   */
-  public void close() throws IOException, InterruptedException;
-  
-  
-  public Configuration getConfiguration();
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine-api/src/main/java/org/apache/tez/engine/common/objectregistry/ObjectLifeCycle.java
----------------------------------------------------------------------
diff --git a/tez-engine-api/src/main/java/org/apache/tez/engine/common/objectregistry/ObjectLifeCycle.java b/tez-engine-api/src/main/java/org/apache/tez/engine/common/objectregistry/ObjectLifeCycle.java
deleted file mode 100644
index 7099299..0000000
--- a/tez-engine-api/src/main/java/org/apache/tez/engine/common/objectregistry/ObjectLifeCycle.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tez.engine.common.objectregistry;
-
-/**
- * Defines the valid lifecycle and scope of Objects stored in ObjectRegistry.
- * Objects are guaranteed to not be valid outside of their defined life-cycle
- * period. Objects are not guaranteed to be retained through the defined period
- * as they may be evicted for various reasons.
- */
-public enum ObjectLifeCycle {
-  /** Objects are valid for the lifetime of the Tez JVM/Session
-   */
-  SESSION,
-  /** Objects are valid for the lifetime of the DAG.
-   */
-  DAG,
-  /** Objects are valid for the lifetime of the Vertex.
-   */
-  VERTEX,
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine-api/src/main/java/org/apache/tez/engine/common/objectregistry/ObjectRegistry.java
----------------------------------------------------------------------
diff --git a/tez-engine-api/src/main/java/org/apache/tez/engine/common/objectregistry/ObjectRegistry.java b/tez-engine-api/src/main/java/org/apache/tez/engine/common/objectregistry/ObjectRegistry.java
deleted file mode 100644
index a27903d..0000000
--- a/tez-engine-api/src/main/java/org/apache/tez/engine/common/objectregistry/ObjectRegistry.java
+++ /dev/null
@@ -1,56 +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.engine.common.objectregistry;
-
-/**
- * Preliminary version of a simple shared object cache to re-use
- * objects across multiple tasks within the same container/JVM.
- */
-public interface ObjectRegistry {
-
-  /**
-   * Insert or update object into the registry. This will remove an object
-   * associated with the same key with a different life-cycle as there is only
-   * one instance of an Object stored for a given key irrespective of the
-   * life-cycle attached to the Object.
-   * @param lifeCycle What life-cycle is the Object valid for
-   * @param key Key to identify the Object
-   * @param value Object to be inserted
-   * @return Previous Object associated with the key attached if present
-   * else null. Could return the same object if the object was associated with
-   * the same key for a different life-cycle.
-   */
-  public Object add(ObjectLifeCycle lifeCycle, String key, Object value);
-
-  /**
-   * Return the object associated with the provided key
-   * @param key Key to find object
-   * @return Object if found else null
-   */
-  public Object get(String key);
-
-  /**
-   * Delete the object associated with the provided key
-   * @param lifeCycle What life-cycle is the Object valid for
-   * @param key Key to find object
-   * @return True if an object was found and removed
-   */
-  public boolean delete(String key);
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine-api/src/main/java/org/apache/tez/engine/common/objectregistry/ObjectRegistryFactory.java
----------------------------------------------------------------------
diff --git a/tez-engine-api/src/main/java/org/apache/tez/engine/common/objectregistry/ObjectRegistryFactory.java b/tez-engine-api/src/main/java/org/apache/tez/engine/common/objectregistry/ObjectRegistryFactory.java
deleted file mode 100644
index 94352b3..0000000
--- a/tez-engine-api/src/main/java/org/apache/tez/engine/common/objectregistry/ObjectRegistryFactory.java
+++ /dev/null
@@ -1,32 +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.engine.common.objectregistry;
-
-import com.google.inject.Inject;
-
-public class ObjectRegistryFactory {
-
-  @Inject
-  private static ObjectRegistry objectRegistry;
-
-  public static ObjectRegistry getObjectRegistry() {
-    return objectRegistry;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/Event.java
----------------------------------------------------------------------
diff --git a/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/Event.java b/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/Event.java
deleted file mode 100644
index 20157c7..0000000
--- a/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/Event.java
+++ /dev/null
@@ -1,28 +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.engine.newapi;
-
-/**
- * Base class for all events generated within the Tez execution engine.
- * Used as the primary mode of communication between the AM, Inputs, Processors
- * and Outputs.
- */
-public abstract class Event {
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/Input.java
----------------------------------------------------------------------
diff --git a/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/Input.java b/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/Input.java
deleted file mode 100644
index 5608a85..0000000
--- a/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/Input.java
+++ /dev/null
@@ -1,71 +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.engine.newapi;
-
-import java.util.List;
-
-/**
- * Represents an input through which a TezProcessor receives data on an edge.
- * </p>
- *
- * <code>Input</code> classes must have a 0 argument public constructor for Tez
- * to construct the <code>Input</code>. Tez will take care of initializing and
- * closing the Input after a {@link Processor} completes. </p>
- */
-public interface Input {
-
-  /**
-   * Initializes the <code>Input</code>.
-   *
-   * @param inputContext
-   *          the {@link TezInputContext}
-   * @return
-   * @throws Exception
-   *           if an error occurs
-   */
-  public List<Event> initialize(TezInputContext inputContext)
-      throws Exception;
-
-  /**
-   * Gets an instance of the {@link Reader} for this <code>Output</code>
-   *
-   * @return
-   * @throws Exception
-   *           if an error occurs
-   */
-  public Reader getReader() throws Exception;
-
-  /**
-   * Handles user and system generated {@link Events}s, which typically carry
-   * information such as an output being available on the previous vertex.
-   *
-   * @param inputEvents
-   *          the list of {@link Event}s
-   */
-  public void handleEvents(List<Event> inputEvents);
-
-  /**
-   * Closes the <code>Input</code>
-   *
-   * @return
-   * @throws Exception
-   *           if an error occurs
-   */
-  public List<Event> close() throws Exception;
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/LogicalIOProcessor.java
----------------------------------------------------------------------
diff --git a/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/LogicalIOProcessor.java b/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/LogicalIOProcessor.java
deleted file mode 100644
index 53b5184..0000000
--- a/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/LogicalIOProcessor.java
+++ /dev/null
@@ -1,43 +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.engine.newapi;
-
-import java.util.Map;
-
-/**
- * Represents a processor which consumes {@link LogicalInput}s and produces
- * {@link LogicalOutput}s
- */
-public interface LogicalIOProcessor extends Processor {
-
-  /**
-   * Runs the {@link LogicalProcessor}
-   * 
-   * @param inputs
-   *          a map of the source vertex name to {@link LogicalInput} - one per
-   *          incoming edge.
-   * @param outputs
-   *          a map of the destination vertex name to {@link LogicalOutput} -
-   *          one per outgoing edge
-   * @throws Exception TODO
-   */
-  public void run(Map<String, LogicalInput> inputs,
-      Map<String, LogicalOutput> outputs) throws Exception;
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/LogicalInput.java
----------------------------------------------------------------------
diff --git a/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/LogicalInput.java b/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/LogicalInput.java
deleted file mode 100644
index 554172c..0000000
--- a/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/LogicalInput.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tez.engine.newapi;
-
-/**
- * An @link {@link Input} which handles all incoming physical connections on an
- * edge. A {@link LogicalIOProcessor} sees a single Logical Input per incoming
- * edge.
- */
-public interface LogicalInput extends Input {
-
-  /**
-   * Sets the number of physical inputs that this <code>LogicalInput</code> will
-   * receive. This will be called by the Tez framework before initializing the
-   * <code>LogicalInput</code>
-   * 
-   * @param numInputs
-   *          the number of physical inputs.
-   */
-  public void setNumPhysicalInputs(int numInputs);
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/LogicalOutput.java
----------------------------------------------------------------------
diff --git a/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/LogicalOutput.java b/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/LogicalOutput.java
deleted file mode 100644
index d88e043..0000000
--- a/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/LogicalOutput.java
+++ /dev/null
@@ -1,36 +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.engine.newapi;
-
-/**
- * An @link {@link Output} which handles all outgoing physical connections on an
- * edge. A {@link LogicalIOProcessor} sees a single Logical Output per outgoing
- * edge.
- */
-public interface LogicalOutput extends Output {
-  /**
-   * Sets the number of physical ouputs that this <code>LogicalOutput</code>
-   * will receive. This will be called by the Tez framework before initializing
-   * the <code>LogicalOutput</code>
-   * 
-   * @param numOutputs
-   *          the number of physical outputs
-   */
-  public void setNumPhysicalOutputs(int numOutputs);
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/Output.java
----------------------------------------------------------------------
diff --git a/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/Output.java b/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/Output.java
deleted file mode 100644
index 5a6b5da..0000000
--- a/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/Output.java
+++ /dev/null
@@ -1,71 +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.engine.newapi;
-
-import java.util.List;
-
-/**
- * Represents an Output through which a TezProcessor writes information on an
- * edge. </p>
- *
- * <code>Output</code> implementations must have a 0 argument public constructor
- * for Tez to construct the <code>Output</code>. Tez will take care of
- * initializing and closing the Input after a {@link Processor} completes. </p>
- */
-public interface Output {
-
-  /**
-   * Initializes the <code>Output</code>
-   *
-   * @param outputContext
-   *          the {@link TezOutputContext}
-   * @return
-   * @throws Exception
-   *           if an error occurs
-   */
-  public List<Event> initialize(TezOutputContext outputContext)
-      throws Exception;
-
-  /**
-   * Gets an instance of the {@link Writer} in an <code>Output</code>
-   *
-   * @return
-   * @throws Exception
-   *           if an error occurs
-   */
-  public Writer getWriter() throws Exception;
-
-  /**
-   * Handles user and system generated {@link Events}s, which typically carry
-   * information such as a downstream vertex being ready to consume input.
-   *
-   * @param outputEvents
-   *          the list of {@link Event}s
-   */
-  public void handleEvents(List<Event> outputEvents);
-
-  /**
-   * Closes the <code>Output</code>
-   *
-   * @return
-   * @throws Exception
-   *           if an error occurs
-   */
-  public List<Event> close() throws Exception;
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/Processor.java
----------------------------------------------------------------------
diff --git a/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/Processor.java b/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/Processor.java
deleted file mode 100644
index 3e71b58..0000000
--- a/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/Processor.java
+++ /dev/null
@@ -1,58 +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.engine.newapi;
-
-import java.io.IOException;
-import java.util.List;
-
-import org.apache.tez.engine.api.Input;
-import org.apache.tez.engine.api.Output;
-
-/**
- * {@link Processor} represents the <em>Tez</em> entity responsible for
- * consuming {@link Input} and producing {@link Output}.
- */
-public interface Processor {
-
-  /**
-   * Initializes the <code>Processor</code>
-   *
-   * @param processorContext
-   * @throws IOException
-   *           if an error occurs
-   */
-  public void initialize(TezProcessorContext processorContext)
-      throws Exception;
-
-  /**
-   * Handles user and system generated {@link Events}s.
-   *
-   * @param processorEvents
-   *          the list of {@link Event}s
-   */
-  public void handleEvents(List<Event> processorEvents);
-
-  /**
-   * Closes the <code>Processor</code>
-   *
-   * @throws IOException
-   *           if an error occurs
-   */
-  public void close() throws Exception;
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/Reader.java
----------------------------------------------------------------------
diff --git a/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/Reader.java b/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/Reader.java
deleted file mode 100644
index 8b8750c..0000000
--- a/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/Reader.java
+++ /dev/null
@@ -1,26 +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.engine.newapi;
-
-/**
- * A <code>Reader</code> represents the data being read in an {@link Input}
- */
-public interface Reader {
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/TezInputContext.java
----------------------------------------------------------------------
diff --git a/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/TezInputContext.java b/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/TezInputContext.java
deleted file mode 100644
index a56c9d4..0000000
--- a/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/TezInputContext.java
+++ /dev/null
@@ -1,32 +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.engine.newapi;
-
-/**
- * Context handle for the Input to initialize itself.
- */
-public interface TezInputContext extends TezTaskContext {
-
-  /**
-   * Get the Vertex Name of the Source that generated data for this Input
-   * @return Name of the Source Vertex
-   */
-  public String getSourceVertexName();
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/TezOutputContext.java
----------------------------------------------------------------------
diff --git a/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/TezOutputContext.java b/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/TezOutputContext.java
deleted file mode 100644
index 34c6028..0000000
--- a/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/TezOutputContext.java
+++ /dev/null
@@ -1,33 +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.engine.newapi;
-
-/**
- * Context handle for the Output to initialize itself.
- */
-public interface TezOutputContext extends TezTaskContext {
-
-  /**
-   * Get the Vertex Name of the Destination that is the recipient of this
-   * Output's data
-   * @return Name of the Destination Vertex
-   */
-  public String getDestinationVertexName();
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/TezProcessorContext.java
----------------------------------------------------------------------
diff --git a/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/TezProcessorContext.java b/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/TezProcessorContext.java
deleted file mode 100644
index 5b44f23..0000000
--- a/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/TezProcessorContext.java
+++ /dev/null
@@ -1,41 +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.engine.newapi;
-
-import java.io.IOException;
-
-/**
- * Context handle for the Processor to initialize itself.
- */
-public interface TezProcessorContext extends TezTaskContext {
-
-  /**
-   * Set the overall progress of this Task Attempt
-   * @param progress Progress in the range from [0.0 - 1.0f]
-   */
-  public void setProgress(float progress);
-
-  /**
-   * Check whether this attempt can commit its output
-   * @return true if commit allowed
-   * @throws IOException
-   */
-  public boolean canCommit() throws IOException;
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/TezTaskContext.java
----------------------------------------------------------------------
diff --git a/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/TezTaskContext.java b/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/TezTaskContext.java
deleted file mode 100644
index 80f2ca0..0000000
--- a/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/TezTaskContext.java
+++ /dev/null
@@ -1,130 +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.engine.newapi;
-
-import java.nio.ByteBuffer;
-import java.util.List;
-
-import org.apache.hadoop.yarn.api.records.ApplicationId;
-import org.apache.tez.common.counters.TezCounters;
-
-/**
- * Base interface for Context classes used to initialize the Input, Output
- * and Processor instances.
- */
-public interface TezTaskContext {
-
-  // TODO NEWTEZ
-  // Scale the maximum events we fetch per RPC call to mitigate OOM issues
-  // on the ApplicationMaster when a thundering herd of reducers fetch events
-  // This should not be necessary after HADOOP-8942
-
-  /**
-   * Get the {@link ApplicationId} for the running app
-   * @return the {@link ApplicationId}
-   */
-  public ApplicationId getApplicationId();
-
-  /**
-   * Get the current DAG Attempt Number
-   * @return DAG Attempt Number
-   */
-  public int getDAGAttemptNumber();
-
-  /**
-   * Get the index of this Task
-   * @return Task Index
-   */
-  public int getTaskIndex();
-
-  /**
-   * Get the current Task Attempt Number
-   * @return Task Attempt Number
-   */
-  public int getTaskAttemptNumber();
-
-  /**
-   * Get the name of the DAG
-   * @return the DAG name
-   */
-  public String getDAGName();
-
-  /**
-   * Get the name of the Vertex in which the task is running
-   * @return Vertex Name
-   */
-  public String getTaskVertexName();
-
-  public TezCounters getCounters();
-
-  /**
-   * Send Events to the AM and/or dependent Vertices
-   * @param events Events to be sent
-   */
-  public void sendEvents(List<Event> events);
-
-  /**
-   * Get the User Payload for the Input/Output/Processor
-   * @return User Payload
-   */
-  public byte[] getUserPayload();
-
-  /**
-   * Get the work diectories for the Input/Output/Processor
-   * @return an array of work dirs
-   */
-  public String[] getWorkDirs();
-
-  /**
-   * Returns an identifier which is unique to the specific Input, Processor or
-   * Output
-   *
-   * @return
-   */
-  public String getUniqueIdentifier();
-
-  /**
-   * Report a fatal error to the framework. This will cause the entire task to
-   * fail and should not be used for reporting temporary or recoverable errors
-   *
-   * @param exception an exception representing the error
-   */
-  public void fatalError(Throwable exception, String message);
-
-  /**
-   * Returns meta-data for the specified service. As an example, when the MR
-   * ShuffleHandler is used - this would return the jobToken serialized as bytes
-   *
-   * @param serviceName
-   *          the name of the service for which meta-data is required
-   * @return a ByteBuffer representing the meta-data
-   */
-  public ByteBuffer getServiceConsumerMetaData(String serviceName);
-
-  /**
-   * Return Provider meta-data for the specified service As an example, when the
-   * MR ShuffleHandler is used - this would return the shuffle port serialized
-   * as bytes
-   *
-   * @param serviceName
-   *          the name of the service for which provider meta-data is required
-   * @return a ByteBuffer representing the meta-data
-   */
-  public ByteBuffer getServiceProviderMetaData(String serviceName);
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/Writer.java
----------------------------------------------------------------------
diff --git a/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/Writer.java b/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/Writer.java
deleted file mode 100644
index d8c0b8a..0000000
--- a/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/Writer.java
+++ /dev/null
@@ -1,26 +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.engine.newapi;
-
-/**
- * A <code>Writer</code> represents the data being written by an {@link Output}
- */
-public interface Writer {
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/events/DataMovementEvent.java
----------------------------------------------------------------------
diff --git a/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/events/DataMovementEvent.java b/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/events/DataMovementEvent.java
deleted file mode 100644
index 92006b8..0000000
--- a/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/events/DataMovementEvent.java
+++ /dev/null
@@ -1,109 +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.engine.newapi.events;
-
-import org.apache.hadoop.classification.InterfaceAudience.Private;
-import org.apache.tez.engine.newapi.Event;
-
-/**
- * Event used by user code to send information between tasks. An output can
- * generate an Event of this type to sending information regarding output data
- * ( such as URI for file-based output data, port info in case of
- * streaming-based data transfers ) to the Input on the destination vertex.
- */
-public final class DataMovementEvent extends Event {
-
-  /**
-   * Index(i) of the i-th (physical) Input or Output that generated an Event.
-   * For a Processor-generated event, this is ignored.
-   */
-  private final int sourceIndex;
-
-  /**
-   * Index(i) of the i-th (physical) Input or Output that is meant to receive
-   * this Event. For a Processor event, this is ignored.
-   */
-  private int targetIndex;
-
-  /**
-   * User Payload for this Event
-   */
-  private final byte[] userPayload;
-
-  /**
-   * Version number to indicate what attempt generated this Event
-   */
-  private int version;
-
-  /**
-   * User Event constructor
-   * @param sourceIndex Index to identify the physical edge of the input/output
-   * that generated the event
-   * @param userPayload User Payload of the User Event
-   */
-  public DataMovementEvent(int sourceIndex,
-      byte[] userPayload) {
-    this.userPayload = userPayload;
-    this.sourceIndex = sourceIndex;
-  }
-
-  @Private
-  public DataMovementEvent(int sourceIndex,
-      int targetIndex,
-      byte[] userPayload) {
-    this.userPayload = userPayload;
-    this.sourceIndex = sourceIndex;
-    this.targetIndex = targetIndex;
-  }
-
-  /**
-   * Constructor for Processor-generated User Events
-   * @param userPayload
-   */
-  public DataMovementEvent(byte[] userPayload) {
-    this(-1, userPayload);
-  }
-
-  public byte[] getUserPayload() {
-    return userPayload;
-  }
-
-  public int getSourceIndex() {
-    return sourceIndex;
-  }
-
-  public int getTargetIndex() {
-    return targetIndex;
-  }
-
-  @Private
-  public void setTargetIndex(int targetIndex) {
-    this.targetIndex = targetIndex;
-  }
-
-  public int getVersion() {
-    return version;
-  }
-
-  @Private
-  public void setVersion(int version) {
-    this.version = version;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/events/InputFailedEvent.java
----------------------------------------------------------------------
diff --git a/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/events/InputFailedEvent.java b/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/events/InputFailedEvent.java
deleted file mode 100644
index 0ca4a3e..0000000
--- a/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/events/InputFailedEvent.java
+++ /dev/null
@@ -1,89 +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.engine.newapi.events;
-
-import org.apache.hadoop.classification.InterfaceAudience.Private;
-import org.apache.tez.engine.newapi.Event;
-
-/**
- * Event sent from the AM to an Input to indicate that one of it's sources has
- * failed - effectively the input is no longer available from the particular
- * source.
- * Users are not expected to send this event.
- */
-public class InputFailedEvent extends Event{
-
-  /**
-   * Index(i) of the i-th (physical) Input or Output that generated the data.
-   * For a Processor-generated event, this is ignored.
-   */
-  private final int sourceIndex;
-
-  /**
-   * Index(i) of the i-th (physical) Input or Output that is meant to receive
-   * this Event. For a Processor event, this is ignored.
-   */
-  private int targetIndex;
-
-  /**
-   * Version number to indicate what attempt generated this Event
-   */
-  private int version;
-
-  /**
-   * User Event constructor
-   * @param sourceIndex Index to identify the physical edge of the input/output
-   * that generated the event
-   * @param userPayload User Payload of the User Event
-   */
-  public InputFailedEvent(int sourceIndex) {
-    this.sourceIndex = sourceIndex;
-  }
-
-  @Private
-  public InputFailedEvent(int sourceIndex,
-      int targetIndex,
-      int version) {
-    this.sourceIndex = sourceIndex;
-    this.targetIndex = targetIndex;
-    this.version = version;
-  }
-
-  public int getSourceIndex() {
-    return sourceIndex;
-  }
-
-  public int getTargetIndex() {
-    return targetIndex;
-  }
-
-  @Private
-  public void setTargetIndex(int targetIndex) {
-    this.targetIndex = targetIndex;
-  }
-
-  public int getVersion() {
-    return version;
-  }
-
-  @Private
-  public void setVersion(int version) {
-    this.version = version;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/events/InputInformationEvent.java
----------------------------------------------------------------------
diff --git a/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/events/InputInformationEvent.java b/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/events/InputInformationEvent.java
deleted file mode 100644
index a452a98..0000000
--- a/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/events/InputInformationEvent.java
+++ /dev/null
@@ -1,41 +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.engine.newapi.events;
-
-import org.apache.tez.engine.newapi.Event;
-
-/**
- * Event used to send user specific data from the user 
- * code in the AM to the task input
- */
-public class InputInformationEvent extends Event {
-
-  /**
-   * User Payload for this Event
-   */
-  private final byte[] userPayload;
-  public InputInformationEvent(byte[] userPayload) {
-    this.userPayload = userPayload;
-  }
-
-  public byte[] getUserPayload() {
-    return userPayload;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/events/InputReadErrorEvent.java
----------------------------------------------------------------------
diff --git a/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/events/InputReadErrorEvent.java b/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/events/InputReadErrorEvent.java
deleted file mode 100644
index f2517eb..0000000
--- a/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/events/InputReadErrorEvent.java
+++ /dev/null
@@ -1,65 +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.engine.newapi.events;
-
-import org.apache.tez.engine.newapi.Event;
-
-/**
- * Event generated by an Input to indicate error when trying to retrieve data.
- * This is not necessarily a fatal event - it's an indication to the AM to retry
- * source data generation.
- */
-public final class InputReadErrorEvent extends Event {
-
-  /**
-   * Diagnostics/trace of the error that occurred on the Input's edge.
-   */
-  private final String diagnostics;
-
-  /**
-   * Index of the physical edge on which the error occurred.
-   */
-  private final int index;
-
-  /**
-   * Version of the data on which the error occurred.
-   */
-  private final int version;
-
-  public InputReadErrorEvent(String diagnostics, int index,
-      int version) {
-    super();
-    this.diagnostics = diagnostics;
-    this.index = index;
-    this.version = version;
-  }
-
-  public String getDiagnostics() {
-    return diagnostics;
-  }
-
-  public int getIndex() {
-    return index;
-  }
-
-  public int getVersion() {
-    return version;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine-api/src/main/java/org/apache/tez/engine/records/OutputContext.java
----------------------------------------------------------------------
diff --git a/tez-engine-api/src/main/java/org/apache/tez/engine/records/OutputContext.java b/tez-engine-api/src/main/java/org/apache/tez/engine/records/OutputContext.java
deleted file mode 100644
index a1e9f03..0000000
--- a/tez-engine-api/src/main/java/org/apache/tez/engine/records/OutputContext.java
+++ /dev/null
@@ -1,61 +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.engine.records;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-
-import org.apache.hadoop.io.Writable;
-
-//TODO EVENTUALLY Add more interfaces. Maybe make this an abstract class.
-/**
- * Contains context information for Output. For example, shuffle headers, size
- * of output, etc. TODO Ideally should be Output specific.
- */
-
-public class OutputContext implements Writable {
-
-  public OutputContext(int shufflePort) {
-    this.shufflePort = shufflePort;
-  }
-  
-  public OutputContext() {
-  }
-
-  public int shufflePort;
-
-  public int getShufflePort() {
-    return this.shufflePort;
-  }
-
-  @Override
-  public void write(DataOutput out) throws IOException {
-    out.writeInt(shufflePort);
-  }
-
-  @Override
-  public void readFields(DataInput in) throws IOException {
-    shufflePort = in.readInt();
-  }
-  
-  @Override
-  public String toString() {
-    return "shufflePort: " + shufflePort;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine-api/src/main/java/org/apache/tez/engine/records/TezDependentTaskCompletionEvent.java
----------------------------------------------------------------------
diff --git a/tez-engine-api/src/main/java/org/apache/tez/engine/records/TezDependentTaskCompletionEvent.java b/tez-engine-api/src/main/java/org/apache/tez/engine/records/TezDependentTaskCompletionEvent.java
deleted file mode 100644
index 7a4dd13..0000000
--- a/tez-engine-api/src/main/java/org/apache/tez/engine/records/TezDependentTaskCompletionEvent.java
+++ /dev/null
@@ -1,228 +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.engine.records;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.io.WritableUtils;
-import org.apache.tez.dag.records.TezTaskAttemptID;
-
-/**
- * This is used to track task completion events on 
- * job tracker. 
- */
-@InterfaceAudience.Public
-@InterfaceStability.Stable
-// TODO TEZAM3 This needs to be more generic. Maybe some kind of a serialized
-// blob - which can be interpretted by the Input plugin.
-public class TezDependentTaskCompletionEvent implements Writable {
-  @InterfaceAudience.Public
-  @InterfaceStability.Evolving
-  // TODO EVENTUALLY - Remove TIPFAILED state ?
-  static public enum Status {FAILED, KILLED, SUCCEEDED, OBSOLETE, TIPFAILED};
-    
-  private int eventId;
-  private int taskRunTime; // using int since runtime is the time difference
-  private TezTaskAttemptID taskAttemptId;
-  private long dataSize;
-  Status status;
-  byte[] userPayload;
-  // TODO TEZAM2 Get rid of the isMap field. Job specific type information can be determined from TaskAttemptId.getTaskType
-//  boolean isMap = false;
-  public static final TezDependentTaskCompletionEvent[] EMPTY_ARRAY = 
-    new TezDependentTaskCompletionEvent[0];
-
-  public TezDependentTaskCompletionEvent() {
-    taskAttemptId = new TezTaskAttemptID();
-  }
-  
-  /**
-   * Constructor. eventId should be created externally and incremented
-   * per event for each job. 
-   * @param eventId event id, event id should be unique and assigned in
-   *  incrementally, starting from 0. 
-   * @param taskAttemptId task id
-   * @param status task's status 
-   * @param taskTrackerHttp task tracker's host:port for http. 
-   */
-  public TezDependentTaskCompletionEvent(int eventId, 
-                             TezTaskAttemptID taskAttemptId,
-//                             boolean isMap,
-                             Status status, 
-                             int runTime,
-                             long dataSize){
-      
-    this.taskAttemptId = taskAttemptId;
-//    this.isMap = isMap;
-    this.eventId = eventId; 
-    this.status =status; 
-    this.taskRunTime = runTime;
-    this.dataSize = dataSize;
-  }
-  
-  public TezDependentTaskCompletionEvent clone() {
-    TezDependentTaskCompletionEvent clone = new TezDependentTaskCompletionEvent(
-        this.eventId, this.taskAttemptId, this.status, 
-        this.taskRunTime, this.dataSize);
-    
-    return clone;
-  }
-  
-  /**
-   * Returns event Id. 
-   * @return event id
-   */
-  public int getEventId() {
-    return eventId;
-  }
-
-  /**
-   * Returns task id. 
-   * @return task id
-   */
-  public TezTaskAttemptID getTaskAttemptID() {
-    return taskAttemptId;
-  }
-  
-  /**
-   * Returns enum Status.SUCESS or Status.FAILURE.
-   * @return task tracker status
-   */
-  public Status getStatus() {
-    return status;
-  }
-  
-  /**
-   * Returns time (in millisec) the task took to complete. 
-   */
-  public int getTaskRunTime() {
-    return taskRunTime;
-  }
-  
-  /**
-   * Return size of output produced by the task
-   */
-  public long getDataSize() {
-    return dataSize;
-  }
-  
-  /**
-   * @return user payload. Maybe null
-   */
-  public byte[] getUserPayload() {
-    return userPayload;
-  }
-
-  /**
-   * Set the task completion time
-   * @param taskCompletionTime time (in millisec) the task took to complete
-   */
-  protected void setTaskRunTime(int taskCompletionTime) {
-    this.taskRunTime = taskCompletionTime;
-  }
-
-  /**
-   * set event Id. should be assigned incrementally starting from 0. 
-   * @param eventId
-   */
-  public void setEventId(int eventId) {
-    this.eventId = eventId;
-  }
-
-  /**
-   * Sets task id. 
-   * @param taskId
-   */
-  public void setTaskAttemptID(TezTaskAttemptID taskId) {
-    this.taskAttemptId = taskId;
-  }
-  
-  /**
-   * Set task status. 
-   * @param status
-   */
-  public void setTaskStatus(Status status) {
-    this.status = status;
-  }
-  
-  /**
-   * Set the user payload
-   * @param userPayload
-   */
-  public void setUserPayload(byte[] userPayload) {
-    this.userPayload = userPayload;
-  }
-    
-  @Override
-  public String toString(){
-    StringBuffer buf = new StringBuffer(); 
-    buf.append("Task Id : "); 
-    buf.append(taskAttemptId); 
-    buf.append(", Status : ");  
-    buf.append(status.name());
-    return buf.toString();
-  }
-    
-  @Override
-  public boolean equals(Object o) {
-    // not counting userPayload as that is a piggyback mechanism
-    if(o == null)
-      return false;
-    if(o.getClass().equals(this.getClass())) {
-      TezDependentTaskCompletionEvent event = (TezDependentTaskCompletionEvent) o;
-      return this.eventId == event.getEventId()
-             && this.status.equals(event.getStatus())
-             && this.taskAttemptId.equals(event.getTaskAttemptID()) 
-             && this.taskRunTime == event.getTaskRunTime()
-             && this.dataSize == event.getDataSize();
-    }
-    return false;
-  }
-
-  @Override
-  public int hashCode() {
-    return toString().hashCode(); 
-  }
-
-  @Override
-  public void write(DataOutput out) throws IOException {
-    taskAttemptId.write(out);
-//    out.writeBoolean(isMap);
-    WritableUtils.writeEnum(out, status);
-    WritableUtils.writeVInt(out, taskRunTime);
-    WritableUtils.writeVInt(out, eventId);
-    WritableUtils.writeCompressedByteArray(out, userPayload);
-  }
-
-  @Override
-  public void readFields(DataInput in) throws IOException {
-    taskAttemptId.readFields(in);
-//    isMap = in.readBoolean();
-    status = WritableUtils.readEnum(in, Status.class);
-    taskRunTime = WritableUtils.readVInt(in);
-    eventId = WritableUtils.readVInt(in);
-    userPayload = WritableUtils.readCompressedByteArray(in);
-  }
-  
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine-api/src/main/java/org/apache/tez/engine/records/TezTaskDependencyCompletionEventsUpdate.java
----------------------------------------------------------------------
diff --git a/tez-engine-api/src/main/java/org/apache/tez/engine/records/TezTaskDependencyCompletionEventsUpdate.java b/tez-engine-api/src/main/java/org/apache/tez/engine/records/TezTaskDependencyCompletionEventsUpdate.java
deleted file mode 100644
index 13c9088..0000000
--- a/tez-engine-api/src/main/java/org/apache/tez/engine/records/TezTaskDependencyCompletionEventsUpdate.java
+++ /dev/null
@@ -1,64 +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.engine.records;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-
-import org.apache.hadoop.io.Writable;
-
-
-public class TezTaskDependencyCompletionEventsUpdate implements Writable {
-  TezDependentTaskCompletionEvent[] events;
-  boolean reset;
-
-  public TezTaskDependencyCompletionEventsUpdate() { }
-
-  public TezTaskDependencyCompletionEventsUpdate(
-      TezDependentTaskCompletionEvent[] events, boolean reset) {
-    this.events = events;
-    this.reset = reset;
-  }
-
-  public boolean shouldReset() {
-    return reset;
-  }
-
-  public TezDependentTaskCompletionEvent[] getDependentTaskCompletionEvents() {
-    return events;
-  }
-  
-  public void write(DataOutput out) throws IOException {
-    out.writeBoolean(reset);
-    out.writeInt(events.length);
-    for (TezDependentTaskCompletionEvent event : events) {
-      event.write(out);
-    }
-  }
-
-  public void readFields(DataInput in) throws IOException {
-    reset = in.readBoolean();
-    events = new TezDependentTaskCompletionEvent[in.readInt()];
-    for (int i = 0; i < events.length; ++i) {
-      events[i] = new TezDependentTaskCompletionEvent();
-      events[i].readFields(in);
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine-api/src/main/proto/Events.proto
----------------------------------------------------------------------
diff --git a/tez-engine-api/src/main/proto/Events.proto b/tez-engine-api/src/main/proto/Events.proto
deleted file mode 100644
index 21cacf6..0000000
--- a/tez-engine-api/src/main/proto/Events.proto
+++ /dev/null
@@ -1,44 +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.
- */
-
-option java_package = "org.apache.tez.engine.api.events";
-option java_outer_classname = "EventProtos";
-option java_generate_equals_and_hash = true;
-
-message DataMovementEventProto {
-  optional int32 source_index = 1;
-  optional int32 target_index = 2;
-  optional bytes user_payload = 3;
-  optional int32 version = 4;
-}
-
-message InputReadErrorEventProto {
-  optional int32 index = 1;
-  optional string diagnostics = 2;
-  optional int32 version = 3;
-}
-
-message InputFailedEventProto {
-  optional int32 source_index = 1;
-  optional int32 target_index = 2;
-  optional int32 version = 4;
-}
-
-message InputInformationEventProto {
-  optional bytes user_payload = 1;
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine/pom.xml
----------------------------------------------------------------------
diff --git a/tez-engine/pom.xml b/tez-engine/pom.xml
index 82eb48a..498f2f2 100644
--- a/tez-engine/pom.xml
+++ b/tez-engine/pom.xml
@@ -35,11 +35,7 @@
     </dependency>
     <dependency>
       <groupId>org.apache.tez</groupId>
-      <artifactId>tez-dag-api</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.tez</groupId>
-      <artifactId>tez-engine-api</artifactId>
+      <artifactId>tez-api</artifactId>
     </dependency>
     <dependency>
       <groupId>org.apache.hadoop</groupId>

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine/src/main/java/org/apache/tez/common/Constants.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/common/Constants.java b/tez-engine/src/main/java/org/apache/tez/common/Constants.java
new file mode 100644
index 0000000..8ea2909
--- /dev/null
+++ b/tez-engine/src/main/java/org/apache/tez/common/Constants.java
@@ -0,0 +1,57 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership. The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.tez.common;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+
+
+public class Constants {
+
+  // TODO NEWTEZ Check which of these constants are expecting specific pieces of information which are being removed - like taskAttemptId
+  
+  public static final String TEZ = "tez";
+
+  public static final String MAP_OUTPUT_FILENAME_STRING = "file.out";
+  public static final String MAP_OUTPUT_INDEX_SUFFIX_STRING = ".index";
+  public static final String REDUCE_INPUT_FILE_FORMAT_STRING = "%s/map_%d.out";
+
+  public static final int MAP_OUTPUT_INDEX_RECORD_LENGTH = 24;
+  public static String MERGED_OUTPUT_PREFIX = ".merged";
+  
+  // TODO NEWTEZ Remove this constant once the old code is removed.
+  public static final String TEZ_ENGINE_TASK_ATTEMPT_ID = 
+      "tez.engine.task.attempt.id";
+
+  public static final String TEZ_ENGINE_TASK_OUTPUT_FILENAME_STRING = "file.out";
+
+  public static final String TEZ_ENGINE_TASK_OUTPUT_INDEX_SUFFIX_STRING = ".index";
+
+  public static final String TEZ_ENGINE_TASK_INPUT_FILE_FORMAT_STRING = "%s/task_%d.out"; 
+
+  public static final String TEZ_ENGINE_JOB_CREDENTIALS = 
+      "tez.engine.job.credentials";
+  
+  @Private
+  public static final String TEZ_ENGINE_TASK_MEMORY =  "tez.engine.task.memory";
+  
+  public static final String TASK_OUTPUT_DIR = "output";
+  
+  public static final String TEZ_ENGINE_TASK_OUTPUT_MANAGER = 
+      "tez.engine.task.local.output.manager";
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine/src/main/java/org/apache/tez/common/ContainerContext.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/common/ContainerContext.java b/tez-engine/src/main/java/org/apache/tez/common/ContainerContext.java
new file mode 100644
index 0000000..df92bdc
--- /dev/null
+++ b/tez-engine/src/main/java/org/apache/tez/common/ContainerContext.java
@@ -0,0 +1,64 @@
+/**
+* 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.common;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.Writable;
+
+// TODO EVENTUALLY move this over to PB. Fix package/module.
+// TODO EVENTUALLY unit tests for functionality.
+public class ContainerContext implements Writable {
+
+  String containerIdentifier;
+  String pid;
+
+  public ContainerContext() {
+    containerIdentifier = "";
+    pid = "";
+  }
+
+  public ContainerContext(String containerIdStr, String pid) {
+    this.containerIdentifier = containerIdStr;
+    this.pid = pid;
+  }
+
+  public String getContainerIdentifier() {
+    return containerIdentifier;
+  }
+
+  public String getPid() {
+    return pid;
+  }
+
+  @Override
+  public void readFields(DataInput in) throws IOException {
+    this.containerIdentifier = Text.readString(in);
+    this.pid = Text.readString(in);
+  }
+
+  @Override
+  public void write(DataOutput out) throws IOException {
+    Text.writeString(out, containerIdentifier);
+    Text.writeString(out, pid);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine/src/main/java/org/apache/tez/common/TezTaskUmbilicalProtocol.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/common/TezTaskUmbilicalProtocol.java b/tez-engine/src/main/java/org/apache/tez/common/TezTaskUmbilicalProtocol.java
index 7d81b4c..9e4129f 100644
--- a/tez-engine/src/main/java/org/apache/tez/common/TezTaskUmbilicalProtocol.java
+++ b/tez-engine/src/main/java/org/apache/tez/common/TezTaskUmbilicalProtocol.java
@@ -29,8 +29,6 @@ import org.apache.tez.dag.api.TezException;
 import org.apache.tez.dag.records.TezTaskAttemptID;
 import org.apache.tez.engine.api.impl.TezHeartbeatRequest;
 import org.apache.tez.engine.api.impl.TezHeartbeatResponse;
-import org.apache.tez.engine.records.OutputContext;
-import org.apache.tez.engine.records.TezTaskDependencyCompletionEventsUpdate;
 
 /** Protocol that task child process uses to contact its parent process.  The
  * parent is a daemon which which polls the central master for a new map or
@@ -47,21 +45,6 @@ public interface TezTaskUmbilicalProtocol extends VersionedProtocol {
 
   boolean canCommit(TezTaskAttemptID taskid) throws IOException;
 
-  // TODO TEZAM5 Can commitPending and outputReady be collapsed into a single
-  // call.
-  // IAC outputReady followed by commit is a little confusing - since the output
-  // isn't really in place till a commit is called. Maybe rename to
-  // processingComplete or some such.
-
-  // TODO EVENTUALLY This is not the most useful API. Once there's some kind of
-  // support for the Task handing output over to the Container, this won't rally
-  // be required. i.e. InMemShuffle running as a service in the Container, or
-  // the second task in getTask(). ContainerUmbilical would include getTask and
-  // getServices...
-
-  void outputReady(TezTaskAttemptID taskAttemptId, OutputContext outputContext)
-      throws IOException;
-
   ProceedToCompletionResponse
       proceedToCompletion(TezTaskAttemptID taskAttemptId) throws IOException;
 
@@ -71,7 +54,4 @@ public interface TezTaskUmbilicalProtocol extends VersionedProtocol {
   public TezHeartbeatResponse heartbeat(TezHeartbeatRequest request)
       throws IOException, TezException;
 
-  public TezTaskDependencyCompletionEventsUpdate getDependentTasksCompletionEvents(
-      int fromEventIdx, int maxEventsToFetch,
-      TezTaskAttemptID taskAttemptId);
 }

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine/src/main/java/org/apache/tez/engine/api/KVReader.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/api/KVReader.java b/tez-engine/src/main/java/org/apache/tez/engine/api/KVReader.java
index 150b598..0178b3a 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/api/KVReader.java
+++ b/tez-engine/src/main/java/org/apache/tez/engine/api/KVReader.java
@@ -20,7 +20,7 @@ package org.apache.tez.engine.api;
 
 import java.io.IOException;
 
-import org.apache.tez.engine.newapi.Reader;
+import org.apache.tez.engine.api.Reader;
 
 /**
  * A key/value(s) pair based {@link Reader}.

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine/src/main/java/org/apache/tez/engine/api/KVWriter.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/api/KVWriter.java b/tez-engine/src/main/java/org/apache/tez/engine/api/KVWriter.java
index 079d488..970831b 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/api/KVWriter.java
+++ b/tez-engine/src/main/java/org/apache/tez/engine/api/KVWriter.java
@@ -20,7 +20,7 @@ package org.apache.tez.engine.api;
 
 import java.io.IOException;
 
-import org.apache.tez.engine.newapi.Writer;
+import org.apache.tez.engine.api.Writer;
 
 /**
  * A key/value(s) pair based {@link Writer}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine/src/main/java/org/apache/tez/engine/api/events/TaskAttemptCompletedEvent.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/api/events/TaskAttemptCompletedEvent.java b/tez-engine/src/main/java/org/apache/tez/engine/api/events/TaskAttemptCompletedEvent.java
index 3a90f56..a9f2c98 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/api/events/TaskAttemptCompletedEvent.java
+++ b/tez-engine/src/main/java/org/apache/tez/engine/api/events/TaskAttemptCompletedEvent.java
@@ -18,7 +18,7 @@
 
 package org.apache.tez.engine.api.events;
 
-import org.apache.tez.engine.newapi.Event;
+import org.apache.tez.engine.api.Event;
 
 public class TaskAttemptCompletedEvent extends Event {
 

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine/src/main/java/org/apache/tez/engine/api/events/TaskAttemptFailedEvent.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/api/events/TaskAttemptFailedEvent.java b/tez-engine/src/main/java/org/apache/tez/engine/api/events/TaskAttemptFailedEvent.java
index bd0bc04..fc67472 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/api/events/TaskAttemptFailedEvent.java
+++ b/tez-engine/src/main/java/org/apache/tez/engine/api/events/TaskAttemptFailedEvent.java
@@ -18,7 +18,7 @@
 
 package org.apache.tez.engine.api.events;
 
-import org.apache.tez.engine.newapi.Event;
+import org.apache.tez.engine.api.Event;
 
 public class TaskAttemptFailedEvent extends Event {
 

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine/src/main/java/org/apache/tez/engine/api/events/TaskStatusUpdateEvent.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/api/events/TaskStatusUpdateEvent.java b/tez-engine/src/main/java/org/apache/tez/engine/api/events/TaskStatusUpdateEvent.java
index c0d77da..c0d1ee6 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/api/events/TaskStatusUpdateEvent.java
+++ b/tez-engine/src/main/java/org/apache/tez/engine/api/events/TaskStatusUpdateEvent.java
@@ -24,7 +24,7 @@ import java.io.IOException;
 
 import org.apache.hadoop.io.Writable;
 import org.apache.tez.common.counters.TezCounters;
-import org.apache.tez.engine.newapi.Event;
+import org.apache.tez.engine.api.Event;
 
 public class TaskStatusUpdateEvent extends Event implements Writable {
 

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezEvent.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezEvent.java b/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezEvent.java
index 9d0228d..6841d72 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezEvent.java
+++ b/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezEvent.java
@@ -24,6 +24,11 @@ import java.io.IOException;
 
 import org.apache.hadoop.io.Writable;
 import org.apache.tez.dag.api.TezUncheckedException;
+import org.apache.tez.engine.api.Event;
+import org.apache.tez.engine.api.events.DataMovementEvent;
+import org.apache.tez.engine.api.events.InputFailedEvent;
+import org.apache.tez.engine.api.events.InputInformationEvent;
+import org.apache.tez.engine.api.events.InputReadErrorEvent;
 import org.apache.tez.engine.api.events.TaskAttemptCompletedEvent;
 import org.apache.tez.engine.api.events.TaskAttemptFailedEvent;
 import org.apache.tez.engine.api.events.TaskStatusUpdateEvent;
@@ -33,11 +38,6 @@ import org.apache.tez.engine.api.events.EventProtos.InputInformationEventProto;
 import org.apache.tez.engine.api.events.EventProtos.InputReadErrorEventProto;
 import org.apache.tez.engine.api.events.SystemEventProtos.TaskAttemptCompletedEventProto;
 import org.apache.tez.engine.api.events.SystemEventProtos.TaskAttemptFailedEventProto;
-import org.apache.tez.engine.newapi.Event;
-import org.apache.tez.engine.newapi.events.DataMovementEvent;
-import org.apache.tez.engine.newapi.events.InputFailedEvent;
-import org.apache.tez.engine.newapi.events.InputInformationEvent;
-import org.apache.tez.engine.newapi.events.InputReadErrorEvent;
 
 import com.google.protobuf.ByteString;
 

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezInputContextImpl.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezInputContextImpl.java b/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezInputContextImpl.java
index 2e10a93..245cd3b 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezInputContextImpl.java
+++ b/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezInputContextImpl.java
@@ -27,9 +27,9 @@ import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.tez.common.counters.TezCounters;
 import org.apache.tez.dag.records.TezTaskAttemptID;
+import org.apache.tez.engine.api.Event;
+import org.apache.tez.engine.api.TezInputContext;
 import org.apache.tez.engine.api.impl.EventMetaData.EventProducerConsumerType;
-import org.apache.tez.engine.newapi.Event;
-import org.apache.tez.engine.newapi.TezInputContext;
 import org.apache.tez.engine.newruntime.RuntimeTask;
 
 public class TezInputContextImpl extends TezTaskContextImpl

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezOutputContextImpl.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezOutputContextImpl.java b/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezOutputContextImpl.java
index ef58de2..6b42e13 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezOutputContextImpl.java
+++ b/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezOutputContextImpl.java
@@ -27,9 +27,9 @@ import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.tez.common.counters.TezCounters;
 import org.apache.tez.dag.records.TezTaskAttemptID;
+import org.apache.tez.engine.api.Event;
+import org.apache.tez.engine.api.TezOutputContext;
 import org.apache.tez.engine.api.impl.EventMetaData.EventProducerConsumerType;
-import org.apache.tez.engine.newapi.Event;
-import org.apache.tez.engine.newapi.TezOutputContext;
 import org.apache.tez.engine.newruntime.RuntimeTask;
 
 public class TezOutputContextImpl extends TezTaskContextImpl

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezProcessorContextImpl.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezProcessorContextImpl.java b/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezProcessorContextImpl.java
index 3f20d5c..7ffcfd6 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezProcessorContextImpl.java
+++ b/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezProcessorContextImpl.java
@@ -27,9 +27,9 @@ import java.util.Map;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.tez.common.counters.TezCounters;
 import org.apache.tez.dag.records.TezTaskAttemptID;
+import org.apache.tez.engine.api.Event;
+import org.apache.tez.engine.api.TezProcessorContext;
 import org.apache.tez.engine.api.impl.EventMetaData.EventProducerConsumerType;
-import org.apache.tez.engine.newapi.Event;
-import org.apache.tez.engine.newapi.TezProcessorContext;
 import org.apache.tez.engine.newruntime.RuntimeTask;
 
 public class TezProcessorContextImpl extends TezTaskContextImpl

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezTaskContextImpl.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezTaskContextImpl.java b/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezTaskContextImpl.java
index 2312c49..d5a4037 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezTaskContextImpl.java
+++ b/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezTaskContextImpl.java
@@ -30,7 +30,7 @@ import org.apache.hadoop.yarn.util.AuxiliaryServiceHelper;
 import org.apache.tez.common.TezJobConfig;
 import org.apache.tez.common.counters.TezCounters;
 import org.apache.tez.dag.records.TezTaskAttemptID;
-import org.apache.tez.engine.newapi.TezTaskContext;
+import org.apache.tez.engine.api.TezTaskContext;
 import org.apache.tez.engine.newruntime.RuntimeTask;
 
 public abstract class TezTaskContextImpl implements TezTaskContext {

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine/src/main/java/org/apache/tez/engine/broadcast/input/BroadcastInputManager.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/broadcast/input/BroadcastInputManager.java b/tez-engine/src/main/java/org/apache/tez/engine/broadcast/input/BroadcastInputManager.java
index 927f0ad..1211598 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/broadcast/input/BroadcastInputManager.java
+++ b/tez-engine/src/main/java/org/apache/tez/engine/broadcast/input/BroadcastInputManager.java
@@ -25,9 +25,9 @@ import org.apache.hadoop.fs.LocalDirAllocator;
 import org.apache.tez.common.Constants;
 import org.apache.tez.common.TezJobConfig;
 import org.apache.tez.dag.api.TezUncheckedException;
+import org.apache.tez.engine.api.TezInputContext;
 import org.apache.tez.engine.common.InputAttemptIdentifier;
 import org.apache.tez.engine.common.task.local.output.TezTaskOutputFiles;
-import org.apache.tez.engine.newapi.TezInputContext;
 import org.apache.tez.engine.shuffle.common.DiskFetchedInput;
 import org.apache.tez.engine.shuffle.common.FetchedInput;
 import org.apache.tez.engine.shuffle.common.FetchedInputAllocator;

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine/src/main/java/org/apache/tez/engine/broadcast/input/BroadcastShuffleInputEventHandler.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/broadcast/input/BroadcastShuffleInputEventHandler.java b/tez-engine/src/main/java/org/apache/tez/engine/broadcast/input/BroadcastShuffleInputEventHandler.java
index 703e1d9..e89e892 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/broadcast/input/BroadcastShuffleInputEventHandler.java
+++ b/tez-engine/src/main/java/org/apache/tez/engine/broadcast/input/BroadcastShuffleInputEventHandler.java
@@ -24,13 +24,13 @@ import java.util.List;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.tez.dag.api.TezUncheckedException;
+import org.apache.tez.engine.api.Event;
+import org.apache.tez.engine.api.TezInputContext;
+import org.apache.tez.engine.api.events.DataMovementEvent;
+import org.apache.tez.engine.api.events.InputFailedEvent;
 import org.apache.tez.engine.common.InputAttemptIdentifier;
 import org.apache.tez.engine.common.shuffle.impl.ShuffleInputEventHandler;
 import org.apache.tez.engine.common.shuffle.newimpl.ShuffleUserPayloads.DataMovementEventPayloadProto;
-import org.apache.tez.engine.newapi.Event;
-import org.apache.tez.engine.newapi.TezInputContext;
-import org.apache.tez.engine.newapi.events.DataMovementEvent;
-import org.apache.tez.engine.newapi.events.InputFailedEvent;
 
 import com.google.common.base.Preconditions;
 import com.google.protobuf.InvalidProtocolBufferException;

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine/src/main/java/org/apache/tez/engine/broadcast/input/BroadcastShuffleManager.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/broadcast/input/BroadcastShuffleManager.java b/tez-engine/src/main/java/org/apache/tez/engine/broadcast/input/BroadcastShuffleManager.java
index dace07c..7b205fa 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/broadcast/input/BroadcastShuffleManager.java
+++ b/tez-engine/src/main/java/org/apache/tez/engine/broadcast/input/BroadcastShuffleManager.java
@@ -49,14 +49,14 @@ import org.apache.hadoop.io.compress.Decompressor;
 import org.apache.hadoop.io.compress.DefaultCodec;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.tez.common.TezJobConfig;
+import org.apache.tez.engine.api.Event;
+import org.apache.tez.engine.api.TezInputContext;
+import org.apache.tez.engine.api.events.InputReadErrorEvent;
 import org.apache.tez.engine.common.ConfigUtils;
 import org.apache.tez.engine.common.InputAttemptIdentifier;
 import org.apache.tez.engine.common.InputIdentifier;
 import org.apache.tez.engine.common.TezEngineUtils;
 import org.apache.tez.engine.common.shuffle.server.ShuffleHandler;
-import org.apache.tez.engine.newapi.Event;
-import org.apache.tez.engine.newapi.TezInputContext;
-import org.apache.tez.engine.newapi.events.InputReadErrorEvent;
 import org.apache.tez.engine.shuffle.common.FetchResult;
 import org.apache.tez.engine.shuffle.common.FetchedInput;
 import org.apache.tez.engine.shuffle.common.Fetcher;

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine/src/main/java/org/apache/tez/engine/broadcast/output/FileBasedKVWriter.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/broadcast/output/FileBasedKVWriter.java b/tez-engine/src/main/java/org/apache/tez/engine/broadcast/output/FileBasedKVWriter.java
index 84ddd28..474d1cd 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/broadcast/output/FileBasedKVWriter.java
+++ b/tez-engine/src/main/java/org/apache/tez/engine/broadcast/output/FileBasedKVWriter.java
@@ -30,13 +30,13 @@ import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.tez.common.TezJobConfig;
 import org.apache.tez.common.TezUtils;
 import org.apache.tez.engine.api.KVWriter;
+import org.apache.tez.engine.api.TezOutputContext;
 import org.apache.tez.engine.common.ConfigUtils;
 import org.apache.tez.engine.common.TezEngineUtils;
 import org.apache.tez.engine.common.sort.impl.IFile;
 import org.apache.tez.engine.common.sort.impl.TezIndexRecord;
 import org.apache.tez.engine.common.sort.impl.TezSpillRecord;
 import org.apache.tez.engine.common.task.local.output.TezTaskOutput;
-import org.apache.tez.engine.newapi.TezOutputContext;
 
 public class FileBasedKVWriter implements KVWriter {
 


[06/50] [abbrv] TEZ-444. Rename *.new* packages back to what they should be, remove dead code from the old packages - mapreduce module (part of TEZ-398). (sseth)

Posted by ss...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/3d609458/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/task/MRRuntimeTask.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/task/MRRuntimeTask.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/task/MRRuntimeTask.java
deleted file mode 100644
index 8a4c6c1..0000000
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/task/MRRuntimeTask.java
+++ /dev/null
@@ -1,272 +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.task;
-
-import java.io.ByteArrayOutputStream;
-import java.io.IOException;
-import java.io.PrintStream;
-import java.net.URI;
-import java.util.ArrayList;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map.Entry;
-
-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.FSError;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.MapOutputFile;
-import org.apache.hadoop.mapreduce.MRConfig;
-import org.apache.hadoop.mapreduce.filecache.DistributedCache;
-import org.apache.hadoop.mapreduce.security.token.JobTokenSecretManager;
-import org.apache.hadoop.security.Credentials;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.security.token.Token;
-import org.apache.hadoop.util.StringUtils;
-import org.apache.tez.common.TezEngineTaskContext;
-import org.apache.tez.common.TezJobConfig;
-import org.apache.tez.common.TezTaskUmbilicalProtocol;
-import org.apache.tez.engine.api.Input;
-import org.apache.tez.engine.api.Master;
-import org.apache.tez.engine.api.Output;
-import org.apache.tez.engine.api.Processor;
-import org.apache.tez.engine.common.security.JobTokenIdentifier;
-import org.apache.tez.engine.common.security.TokenCache;
-import org.apache.tez.engine.task.RuntimeTask;
-import org.apache.tez.mapreduce.hadoop.DeprecatedKeys;
-import org.apache.tez.mapreduce.hadoop.MRHelpers;
-import org.apache.tez.mapreduce.hadoop.MRJobConfig;
-import org.apache.tez.mapreduce.processor.MRTask;
-import org.apache.tez.mapreduce.task.impl.YarnOutputFiles;
-
-@SuppressWarnings("deprecation")
-public class MRRuntimeTask extends RuntimeTask {
-
-  private static final Log LOG = LogFactory.getLog(MRRuntimeTask.class);
-
-  private MRTask mrTask;
-
-  public MRRuntimeTask(TezEngineTaskContext taskContext, Processor processor,
-      Input[] inputs, Output[] outputs) {
-    super(taskContext, processor, inputs, outputs);
-  }
-
-  @Override
-  public void initialize(Configuration conf, byte[] userPayload,
-      Master master) throws IOException, InterruptedException {
-
-    DeprecatedKeys.init();
-
-    Configuration taskConf;
-    if (userPayload != null) {
-      taskConf = MRHelpers.createConfFromUserPayload(userPayload);
-    } else {
-      taskConf = new Configuration(false);
-    }
-
-    copyTezConfigParameters(taskConf, conf);
-
-    // TODO Avoid all this extra config manipulation.
-    // FIXME we need I/O/p level configs to be used in init below
-
-    // TODO Post MRR
-    // A single file per vertex will likely be a better solution. Does not
-    // require translation - client can take care of this. Will work independent
-    // of whether the configuration is for intermediate tasks or not. Has the
-    // overhead of localizing multiple files per job - i.e. the client would
-    // need to write these files to hdfs, add them as local resources per
-    // vertex. A solution like this may be more practical once it's possible to
-    // submit configuration parameters to the AM and effectively tasks via RPC.
-
-    final JobConf job = new JobConf(taskConf);
-    job.set(MRJobConfig.VERTEX_NAME, taskContext.getVertexName());
-
-    MRTask mrTask = (MRTask) getProcessor();
-    this.mrTask = mrTask;
-
-    if (LOG.isDebugEnabled() && userPayload != null) {
-      Iterator<Entry<String, String>> iter = taskConf.iterator();
-      String taskIdStr = mrTask.getTaskAttemptId().getTaskID().toString();
-      while (iter.hasNext()) {
-        Entry<String, String> confEntry = iter.next();
-        LOG.debug("TaskConf Entry"
-            + ", taskId=" + taskIdStr
-            + ", key=" + confEntry.getKey()
-            + ", value=" + confEntry.getValue());
-      }
-    }
-
-    configureMRTask(job, mrTask);
-
-    this.conf = job;
-    this.master = master;
-
-    // NOTE: Allow processor to initialize input/output
-    processor.initialize(this.conf, this.master);
-  }
-  
-  /*
-   * Used when creating a conf from the userPayload. Need to copy all the tez
-   * config parameters which are set by YarnTezDagChild
-   */
-  public static void copyTezConfigParameters(Configuration conf,
-      Configuration tezTaskConf) {
-    Iterator<Entry<String, String>> iter = tezTaskConf.iterator();
-    while (iter.hasNext()) {
-      Entry<String, String> entry = iter.next();
-      if (conf.get(entry.getKey()) == null) {
-        conf.set(entry.getKey(), tezTaskConf.get(entry.getKey()));
-      }
-    }
-  }
-
-  @Override
-  public void run() throws IOException, InterruptedException {
-    TezTaskUmbilicalProtocol umbilical = (TezTaskUmbilicalProtocol) master;
-    try {
-      super.run();
-    } catch (FSError e) {
-      throw e;
-    } catch (Exception exception) {
-      LOG.warn("Exception running child : "
-          + StringUtils.stringifyException(exception));
-      try {
-        if (mrTask != null) {
-          mrTask.taskCleanup(umbilical);
-        }
-      } catch (Exception e) {
-        LOG.info("Exception cleanup up: " + StringUtils.stringifyException(e));
-      }
-      ByteArrayOutputStream baos = new ByteArrayOutputStream();
-      exception.printStackTrace(new PrintStream(baos));
-      /* broken code due to engine re-factor
-      if (taskContext.getTaskAttemptId() != null) {
-        umbilical.fatalError(taskContext.getTaskAttemptId(), baos.toString());
-      }
-      */
-    }
-  }
-
-  @Override
-  public void close() throws IOException, InterruptedException {
-    // NOTE: Allow processor to close input/output
-    processor.close();
-  }
-
-  private static void configureMRTask(JobConf job, MRTask task)
-      throws IOException, InterruptedException {
-
-    Credentials credentials = UserGroupInformation.getCurrentUser()
-        .getCredentials();
-    job.setCredentials(credentials);
-    // TODO Can this be avoided all together. Have the MRTezOutputCommitter use
-    // the Tez parameter.
-    // TODO This could be fetched from the env if YARN is setting it for all
-    // Containers.
-    // Set it in conf, so as to be able to be used the the OutputCommitter.
-    job.setInt(MRJobConfig.APPLICATION_ATTEMPT_ID,
-        job.getInt(MRJobConfig.APPLICATION_ATTEMPT_ID, -1));
-
-    job.setClass(MRConfig.TASK_LOCAL_OUTPUT_CLASS, YarnOutputFiles.class,
-        MapOutputFile.class); // MR
-
-    Token<JobTokenIdentifier> jobToken = TokenCache.getJobToken(credentials);
-    if (jobToken != null) {
-      // Will MR ever run without a job token.
-      SecretKey sk = JobTokenSecretManager.createSecretKey(jobToken
-          .getPassword());
-      task.setJobTokenSecret(sk);
-    } else {
-      LOG.warn("No job token set");
-    }
-
-    job.set(MRJobConfig.JOB_LOCAL_DIR, job.get(TezJobConfig.JOB_LOCAL_DIR));
-    job.set(MRConfig.LOCAL_DIR, job.get(TezJobConfig.LOCAL_DIRS));
-    if (job.get(TezJobConfig.DAG_CREDENTIALS_BINARY) != null) {
-      job.set(MRJobConfig.MAPREDUCE_JOB_CREDENTIALS_BINARY,
-          job.get(TezJobConfig.DAG_CREDENTIALS_BINARY));
-    }
-
-    // setup the child's attempt directories
-    // Do the task-type specific localization
-    task.localizeConfiguration(job);
-
-    // Set up the DistributedCache related configs
-    setupDistributedCacheConfig(job);
-
-    task.setConf(job);
-  }
-
-  /**
-   * Set up the DistributedCache related configs to make
-   * {@link DistributedCache#getLocalCacheFiles(Configuration)} and
-   * {@link DistributedCache#getLocalCacheArchives(Configuration)} working.
-   * 
-   * @param job
-   * @throws IOException
-   */
-  private static void setupDistributedCacheConfig(final JobConf job)
-      throws IOException {
-
-    String localWorkDir = (job.get(TezJobConfig.TASK_LOCAL_RESOURCE_DIR));
-    // ^ ^ all symlinks are created in the current work-dir
-
-    // Update the configuration object with localized archives.
-    URI[] cacheArchives = DistributedCache.getCacheArchives(job);
-    if (cacheArchives != null) {
-      List<String> localArchives = new ArrayList<String>();
-      for (int i = 0; i < cacheArchives.length; ++i) {
-        URI u = cacheArchives[i];
-        Path p = new Path(u);
-        Path name = new Path((null == u.getFragment()) ? p.getName()
-            : u.getFragment());
-        String linkName = name.toUri().getPath();
-        localArchives.add(new Path(localWorkDir, linkName).toUri().getPath());
-      }
-      if (!localArchives.isEmpty()) {
-        job.set(MRJobConfig.CACHE_LOCALARCHIVES, StringUtils
-            .arrayToString(localArchives.toArray(new String[localArchives
-                .size()])));
-      }
-    }
-
-    // Update the configuration object with localized files.
-    URI[] cacheFiles = DistributedCache.getCacheFiles(job);
-    if (cacheFiles != null) {
-      List<String> localFiles = new ArrayList<String>();
-      for (int i = 0; i < cacheFiles.length; ++i) {
-        URI u = cacheFiles[i];
-        Path p = new Path(u);
-        Path name = new Path((null == u.getFragment()) ? p.getName()
-            : u.getFragment());
-        String linkName = name.toUri().getPath();
-        localFiles.add(new Path(localWorkDir, linkName).toUri().getPath());
-      }
-      if (!localFiles.isEmpty()) {
-        job.set(MRJobConfig.CACHE_LOCALFILES, StringUtils
-            .arrayToString(localFiles.toArray(new String[localFiles.size()])));
-      }
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/3d609458/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/processor/MapUtils.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/processor/MapUtils.java b/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/processor/MapUtils.java
index c5f4e84..2bc327c 100644
--- a/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/processor/MapUtils.java
+++ b/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/processor/MapUtils.java
@@ -54,7 +54,7 @@ import org.apache.tez.engine.api.Task;
 import org.apache.tez.engine.runtime.RuntimeUtils;
 import org.apache.tez.mapreduce.TezTestUtils;
 import org.apache.tez.mapreduce.hadoop.MRJobConfig;
-import org.apache.tez.mapreduce.newprocessor.map.MapProcessor;
+import org.apache.tez.mapreduce.processor.map.MapProcessor;
 
 public class MapUtils {
 

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/3d609458/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/processor/map/TestMapProcessor.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/processor/map/TestMapProcessor.java b/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/processor/map/TestMapProcessor.java
index f5d0b02..84f1f81 100644
--- a/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/processor/map/TestMapProcessor.java
+++ b/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/processor/map/TestMapProcessor.java
@@ -44,7 +44,7 @@ import org.apache.tez.mapreduce.hadoop.MRJobConfig;
 import org.apache.tez.mapreduce.hadoop.MultiStageMRConfToTezTranslator;
 import org.apache.tez.mapreduce.hadoop.MultiStageMRConfigUtil;
 import org.apache.tez.mapreduce.hadoop.mapreduce.TezNullOutputCommitter;
-import org.apache.tez.mapreduce.newinput.SimpleInputLegacy;
+import org.apache.tez.mapreduce.input.SimpleInputLegacy;
 import org.apache.tez.mapreduce.processor.MRTask;
 import org.apache.tez.mapreduce.processor.MapUtils;
 import org.junit.After;

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/3d609458/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/processor/reduce/TestReduceProcessor.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/processor/reduce/TestReduceProcessor.java b/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/processor/reduce/TestReduceProcessor.java
index 7ed18d6..8bcd353 100644
--- a/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/processor/reduce/TestReduceProcessor.java
+++ b/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/processor/reduce/TestReduceProcessor.java
@@ -49,10 +49,11 @@ import org.apache.tez.mapreduce.hadoop.IDConverter;
 import org.apache.tez.mapreduce.hadoop.MRJobConfig;
 import org.apache.tez.mapreduce.hadoop.MultiStageMRConfToTezTranslator;
 import org.apache.tez.mapreduce.hadoop.MultiStageMRConfigUtil;
-import org.apache.tez.mapreduce.newinput.SimpleInputLegacy;
-import org.apache.tez.mapreduce.newoutput.SimpleOutput;
+import org.apache.tez.mapreduce.input.SimpleInputLegacy;
+import org.apache.tez.mapreduce.output.SimpleOutput;
 import org.apache.tez.mapreduce.processor.MRTask;
 import org.apache.tez.mapreduce.processor.MapUtils;
+import org.apache.tez.mapreduce.processor.reduce.ReduceProcessor;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
@@ -150,7 +151,8 @@ public class TestReduceProcessor {
     t.initialize(reduceConf, null, new TestUmbilicalProtocol());
     t.run();
     MRTask mrTask = (MRTask)t.getProcessor();
-    Assert.assertNull(mrTask.getPartitioner());
+//    TODO NEWTEZ Verify the partitioner has been created
+//    Assert.assertNull(mrTask.getPartitioner());
     t.close();
     
     // Can this be done via some utility class ? MapOutputFile derivative, or

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/3d609458/tez-yarn-client/src/main/java/org/apache/tez/mapreduce/YARNRunner.java
----------------------------------------------------------------------
diff --git a/tez-yarn-client/src/main/java/org/apache/tez/mapreduce/YARNRunner.java b/tez-yarn-client/src/main/java/org/apache/tez/mapreduce/YARNRunner.java
index 0dd3c60..56f9035 100644
--- a/tez-yarn-client/src/main/java/org/apache/tez/mapreduce/YARNRunner.java
+++ b/tez-yarn-client/src/main/java/org/apache/tez/mapreduce/YARNRunner.java
@@ -101,9 +101,9 @@ import org.apache.tez.mapreduce.hadoop.MRHelpers;
 import org.apache.tez.mapreduce.hadoop.MRJobConfig;
 import org.apache.tez.mapreduce.hadoop.MultiStageMRConfToTezTranslator;
 import org.apache.tez.mapreduce.hadoop.MultiStageMRConfigUtil;
-import org.apache.tez.mapreduce.newinput.ShuffledMergedInputLegacy;
-import org.apache.tez.mapreduce.newprocessor.map.MapProcessor;
-import org.apache.tez.mapreduce.newprocessor.reduce.ReduceProcessor;
+import org.apache.tez.mapreduce.input.ShuffledMergedInputLegacy;
+import org.apache.tez.mapreduce.processor.map.MapProcessor;
+import org.apache.tez.mapreduce.processor.reduce.ReduceProcessor;
 
 import com.google.common.annotations.VisibleForTesting;
 


[32/50] [abbrv] Rename tez-engine-api to tez-runtime-api and tez-engine is split into 2: - tez-engine-library for user-visible Input/Output/Processor implementations - tez-engine-internals for framework internals

Posted by ss...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/Shuffle.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/Shuffle.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/Shuffle.java
new file mode 100644
index 0000000..8689d11
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/Shuffle.java
@@ -0,0 +1,278 @@
+/**
+ * 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.library.common.shuffle.impl;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.FutureTask;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import javax.crypto.SecretKey;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.LocalDirAllocator;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.tez.common.TezJobConfig;
+import org.apache.tez.common.counters.TaskCounter;
+import org.apache.tez.common.counters.TezCounter;
+import org.apache.tez.dag.api.TezUncheckedException;
+import org.apache.tez.runtime.api.Event;
+import org.apache.tez.runtime.api.TezInputContext;
+import org.apache.tez.runtime.library.common.TezRuntimeUtils;
+import org.apache.tez.runtime.library.common.combine.Combiner;
+import org.apache.tez.runtime.library.common.shuffle.server.ShuffleHandler;
+import org.apache.tez.runtime.library.common.sort.impl.TezRawKeyValueIterator;
+import org.apache.tez.runtime.library.shuffle.common.ShuffleUtils;
+
+import com.google.common.base.Preconditions;
+
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class Shuffle implements ExceptionReporter {
+  
+  private static final Log LOG = LogFactory.getLog(Shuffle.class);
+  private static final int PROGRESS_FREQUENCY = 2000;
+  
+  private final Configuration conf;
+  private final TezInputContext inputContext;
+  private final ShuffleClientMetrics metrics;
+
+  private final ShuffleInputEventHandler eventHandler;
+  private final ShuffleScheduler scheduler;
+  private final MergeManager merger;
+  private Throwable throwable = null;
+  private String throwingThreadName = null;
+  private final int numInputs;
+  private final AtomicInteger reduceStartId;
+  private final SecretKey jobTokenSecret;
+  private AtomicInteger reduceRange = new AtomicInteger(
+      TezJobConfig.TEZ_RUNTIME_SHUFFLE_PARTITION_RANGE_DEFAULT);
+
+  private FutureTask<TezRawKeyValueIterator> runShuffleFuture;
+
+  public Shuffle(TezInputContext inputContext, Configuration conf, int numInputs) throws IOException {
+    this.inputContext = inputContext;
+    this.conf = conf;
+    this.metrics = new ShuffleClientMetrics(inputContext.getDAGName(),
+        inputContext.getTaskVertexName(), inputContext.getTaskIndex(),
+        this.conf, UserGroupInformation.getCurrentUser().getShortUserName());
+            
+    this.numInputs = numInputs;
+    
+    this.jobTokenSecret = ShuffleUtils
+        .getJobTokenSecretFromTokenBytes(inputContext
+            .getServiceConsumerMetaData(ShuffleHandler.MAPREDUCE_SHUFFLE_SERVICEID));
+    
+    Combiner combiner = TezRuntimeUtils.instantiateCombiner(conf, inputContext);
+    
+    FileSystem localFS = FileSystem.getLocal(this.conf);
+    LocalDirAllocator localDirAllocator = 
+        new LocalDirAllocator(TezJobConfig.LOCAL_DIRS);
+
+    // TODO TEZ Get rid of Map / Reduce references.
+    TezCounter shuffledMapsCounter = 
+        inputContext.getCounters().findCounter(TaskCounter.SHUFFLED_MAPS);
+    TezCounter reduceShuffleBytes =
+        inputContext.getCounters().findCounter(TaskCounter.REDUCE_SHUFFLE_BYTES);
+    TezCounter failedShuffleCounter =
+        inputContext.getCounters().findCounter(TaskCounter.FAILED_SHUFFLE);
+    TezCounter spilledRecordsCounter = 
+        inputContext.getCounters().findCounter(TaskCounter.SPILLED_RECORDS);
+    TezCounter reduceCombineInputCounter =
+        inputContext.getCounters().findCounter(TaskCounter.COMBINE_INPUT_RECORDS);
+    TezCounter mergedMapOutputsCounter =
+        inputContext.getCounters().findCounter(TaskCounter.MERGED_MAP_OUTPUTS);
+    
+    reduceStartId = new AtomicInteger(inputContext.getTaskIndex());
+    LOG.info("Shuffle assigned reduce start id: " + reduceStartId.get()
+        + " with default reduce range: " + reduceRange.get());
+
+    scheduler = new ShuffleScheduler(
+          this.inputContext,
+          this.conf,
+          this.numInputs,
+          this,
+          shuffledMapsCounter,
+          reduceShuffleBytes,
+          failedShuffleCounter);
+    eventHandler= new ShuffleInputEventHandler(
+          inputContext,
+          this,
+          scheduler);
+    merger = new MergeManager(
+          this.conf,
+          localFS,
+          localDirAllocator,
+          inputContext,
+          combiner,
+          spilledRecordsCounter,
+          reduceCombineInputCounter,
+          mergedMapOutputsCounter,
+          this);
+  }
+
+  public void handleEvents(List<Event> events) {
+    eventHandler.handleEvents(events);
+  }
+  
+  /**
+   * Indicates whether the Shuffle and Merge processing is complete.
+   * @return false if not complete, true if complete or if an error occurred.
+   */
+  public boolean isInputReady() {
+    if (runShuffleFuture == null) {
+      return false;
+    }
+    return runShuffleFuture.isDone();
+    //return scheduler.isDone() && merger.isMergeComplete();
+  }
+
+  /**
+   * Waits for the Shuffle and Merge to complete, and returns an iterator over the input.
+   * @return an iterator over the fetched input.
+   * @throws IOException
+   * @throws InterruptedException
+   */
+  public TezRawKeyValueIterator waitForInput() throws IOException, InterruptedException {
+    Preconditions.checkState(runShuffleFuture != null,
+        "waitForInput can only be called after run");
+    TezRawKeyValueIterator kvIter;
+    try {
+      kvIter = runShuffleFuture.get();
+    } catch (ExecutionException e) {
+      Throwable cause = e.getCause();
+      if (cause instanceof IOException) {
+        throw (IOException) cause;
+      } else if (cause instanceof InterruptedException) {
+        throw (InterruptedException) cause;
+      } else {
+        throw new TezUncheckedException(
+            "Unexpected exception type while running Shuffle and Merge", cause);
+      }
+    }
+    return kvIter;
+  }
+
+  public void run() {
+    RunShuffleCallable runShuffle = new RunShuffleCallable();
+    runShuffleFuture = new FutureTask<TezRawKeyValueIterator>(runShuffle);
+    new Thread(runShuffleFuture, "ShuffleMergeRunner").start();
+  }
+  
+  private class RunShuffleCallable implements Callable<TezRawKeyValueIterator> {
+    @Override
+    public TezRawKeyValueIterator call() throws IOException, InterruptedException {
+      // TODO NEWTEZ Limit # fetchers to number of inputs
+      final int numFetchers = 
+          conf.getInt(
+              TezJobConfig.TEZ_RUNTIME_SHUFFLE_PARALLEL_COPIES, 
+              TezJobConfig.DEFAULT_TEZ_RUNTIME_SHUFFLE_PARALLEL_COPIES);
+      Fetcher[] fetchers = new Fetcher[numFetchers];
+      for (int i = 0; i < numFetchers; ++i) {
+        fetchers[i] = new Fetcher(conf, scheduler, merger, metrics, Shuffle.this, jobTokenSecret, inputContext);
+        fetchers[i].start();
+      }
+      
+      while (!scheduler.waitUntilDone(PROGRESS_FREQUENCY)) {
+        synchronized (this) {
+          if (throwable != null) {
+            throw new ShuffleError("error in shuffle in " + throwingThreadName,
+                                   throwable);
+          }
+        }
+      }
+      
+      // Stop the map-output fetcher threads
+      for (Fetcher fetcher : fetchers) {
+        fetcher.shutDown();
+      }
+      fetchers = null;
+      
+      // stop the scheduler
+      scheduler.close();
+
+
+      // Finish the on-going merges...
+      TezRawKeyValueIterator kvIter = null;
+      try {
+        kvIter = merger.close();
+      } catch (Throwable e) {
+        throw new ShuffleError("Error while doing final merge " , e);
+      }
+      
+      // Sanity check
+      synchronized (Shuffle.this) {
+        if (throwable != null) {
+          throw new ShuffleError("error in shuffle in " + throwingThreadName,
+                                 throwable);
+        }
+      }
+      return kvIter;
+    }
+  }
+  
+  public int getReduceStartId() {
+    return reduceStartId.get();
+  }
+  
+  public int getReduceRange() {
+    return reduceRange.get();
+  }
+  
+  public synchronized void reportException(Throwable t) {
+    if (throwable == null) {
+      throwable = t;
+      throwingThreadName = Thread.currentThread().getName();
+      // Notify the scheduler so that the reporting thread finds the 
+      // exception immediately.
+      synchronized (scheduler) {
+        scheduler.notifyAll();
+      }
+    }
+  }
+  
+  public static class ShuffleError extends IOException {
+    private static final long serialVersionUID = 5753909320586607881L;
+
+    ShuffleError(String msg, Throwable t) {
+      super(msg, t);
+    }
+  }
+
+  public void setPartitionRange(int range) {
+    if (range == reduceRange.get()) {
+      return;
+    }
+    if (reduceRange.compareAndSet(
+        TezJobConfig.TEZ_RUNTIME_SHUFFLE_PARTITION_RANGE_DEFAULT, range)) {
+      LOG.info("Reduce range set to: " + range);
+    } else {
+      TezUncheckedException e = 
+          new TezUncheckedException("Reduce range can be set only once.");
+      reportException(e);
+      throw e; 
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleClientMetrics.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleClientMetrics.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleClientMetrics.java
new file mode 100644
index 0000000..70de31f
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleClientMetrics.java
@@ -0,0 +1,91 @@
+/**
+ * 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.library.common.shuffle.impl;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.metrics.MetricsContext;
+import org.apache.hadoop.metrics.MetricsRecord;
+import org.apache.hadoop.metrics.MetricsUtil;
+import org.apache.hadoop.metrics.Updater;
+import org.apache.tez.common.TezJobConfig;
+import org.apache.tez.runtime.library.common.Constants;
+import org.apache.tez.runtime.library.common.TezRuntimeUtils;
+
+class ShuffleClientMetrics implements Updater {
+
+  private MetricsRecord shuffleMetrics = null;
+  private int numFailedFetches = 0;
+  private int numSuccessFetches = 0;
+  private long numBytes = 0;
+  private int numThreadsBusy = 0;
+  private final int numCopiers;
+  
+  ShuffleClientMetrics(String dagName, String vertexName, int taskIndex, Configuration conf, 
+      String user) {
+    this.numCopiers = 
+        conf.getInt(
+            TezJobConfig.TEZ_RUNTIME_SHUFFLE_PARALLEL_COPIES, 
+            TezJobConfig.DEFAULT_TEZ_RUNTIME_SHUFFLE_PARALLEL_COPIES);
+
+    MetricsContext metricsContext = MetricsUtil.getContext(Constants.TEZ);
+    this.shuffleMetrics = 
+      MetricsUtil.createRecord(metricsContext, "shuffleInput");
+    this.shuffleMetrics.setTag("user", user);
+    this.shuffleMetrics.setTag("dagName", dagName);
+    this.shuffleMetrics.setTag("taskId", TezRuntimeUtils.getTaskIdentifier(vertexName, taskIndex));
+    this.shuffleMetrics.setTag("sessionId", 
+        conf.get(
+            TezJobConfig.TEZ_RUNTIME_METRICS_SESSION_ID, 
+            TezJobConfig.DEFAULT_TEZ_RUNTIME_METRICS_SESSION_ID));
+    metricsContext.registerUpdater(this);
+  }
+  public synchronized void inputBytes(long numBytes) {
+    this.numBytes += numBytes;
+  }
+  public synchronized void failedFetch() {
+    ++numFailedFetches;
+  }
+  public synchronized void successFetch() {
+    ++numSuccessFetches;
+  }
+  public synchronized void threadBusy() {
+    ++numThreadsBusy;
+  }
+  public synchronized void threadFree() {
+    --numThreadsBusy;
+  }
+  public void doUpdates(MetricsContext unused) {
+    synchronized (this) {
+      shuffleMetrics.incrMetric("shuffle_input_bytes", numBytes);
+      shuffleMetrics.incrMetric("shuffle_failed_fetches", 
+                                numFailedFetches);
+      shuffleMetrics.incrMetric("shuffle_success_fetches", 
+                                numSuccessFetches);
+      if (numCopiers != 0) {
+        shuffleMetrics.setMetric("shuffle_fetchers_busy_percent",
+            100*((float)numThreadsBusy/numCopiers));
+      } else {
+        shuffleMetrics.setMetric("shuffle_fetchers_busy_percent", 0);
+      }
+      numBytes = 0;
+      numSuccessFetches = 0;
+      numFailedFetches = 0;
+    }
+    shuffleMetrics.update();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleHeader.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleHeader.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleHeader.java
new file mode 100644
index 0000000..327473e
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleHeader.java
@@ -0,0 +1,94 @@
+/**
+ * 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.library.common.shuffle.impl;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableUtils;
+
+/**
+ * Shuffle Header information that is sent by the TaskTracker and 
+ * deciphered by the Fetcher thread of Reduce task
+ *
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Stable
+public class ShuffleHeader implements Writable {
+  
+  /** Header info of the shuffle http request/response */
+  public static final String HTTP_HEADER_NAME = "name";
+  public static final String DEFAULT_HTTP_HEADER_NAME = "mapreduce";
+  public static final String HTTP_HEADER_VERSION = "version";
+  public static final String DEFAULT_HTTP_HEADER_VERSION = "1.0.0";
+
+  /**
+   * The longest possible length of task attempt id that we will accept.
+   */
+  private static final int MAX_ID_LENGTH = 1000;
+
+  String mapId;
+  long uncompressedLength;
+  long compressedLength;
+  int forReduce;
+  
+  public ShuffleHeader() { }
+  
+  public ShuffleHeader(String mapId, long compressedLength,
+      long uncompressedLength, int forReduce) {
+    this.mapId = mapId;
+    this.compressedLength = compressedLength;
+    this.uncompressedLength = uncompressedLength;
+    this.forReduce = forReduce;
+  }
+  
+  public String getMapId() {
+    return this.mapId;
+  }
+  
+  public int getPartition() {
+    return this.forReduce;
+  }
+  
+  public long getUncompressedLength() {
+    return uncompressedLength;
+  }
+
+  public long getCompressedLength() {
+    return compressedLength;
+  }
+
+  public void readFields(DataInput in) throws IOException {
+    mapId = WritableUtils.readStringSafely(in, MAX_ID_LENGTH);
+    compressedLength = WritableUtils.readVLong(in);
+    uncompressedLength = WritableUtils.readVLong(in);
+    forReduce = WritableUtils.readVInt(in);
+  }
+
+  public void write(DataOutput out) throws IOException {
+    Text.writeString(out, mapId);
+    WritableUtils.writeVLong(out, compressedLength);
+    WritableUtils.writeVLong(out, uncompressedLength);
+    WritableUtils.writeVInt(out, forReduce);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleInputEventHandler.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleInputEventHandler.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleInputEventHandler.java
new file mode 100644
index 0000000..8b323b5
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleInputEventHandler.java
@@ -0,0 +1,134 @@
+/**
+ * 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.library.common.shuffle.impl;
+
+import java.net.URI;
+import java.util.List;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.tez.dag.api.TezUncheckedException;
+import org.apache.tez.runtime.api.Event;
+import org.apache.tez.runtime.api.TezInputContext;
+import org.apache.tez.runtime.api.events.DataMovementEvent;
+import org.apache.tez.runtime.api.events.InputFailedEvent;
+import org.apache.tez.runtime.api.events.InputInformationEvent;
+import org.apache.tez.runtime.library.common.InputAttemptIdentifier;
+import org.apache.tez.runtime.library.shuffle.impl.ShuffleUserPayloads.DataMovementEventPayloadProto;
+import org.apache.tez.runtime.library.shuffle.impl.ShuffleUserPayloads.InputInformationEventPayloadProto;
+
+import com.google.common.base.Preconditions;
+import com.google.protobuf.InvalidProtocolBufferException;
+
+public class ShuffleInputEventHandler {
+  
+  private static final Log LOG = LogFactory.getLog(ShuffleInputEventHandler.class);
+
+  private final ShuffleScheduler scheduler;
+  private final TezInputContext inputContext;
+  private final Shuffle shuffle;
+
+  private int maxMapRuntime = 0;
+  private boolean shuffleRangeSet = false;
+  
+  public ShuffleInputEventHandler(TezInputContext inputContext,
+      Shuffle shuffle, ShuffleScheduler scheduler) {
+    this.inputContext = inputContext;
+    this.shuffle = shuffle;
+    this.scheduler = scheduler;
+  }
+
+  public void handleEvents(List<Event> events) {
+    for (Event event : events) {
+      handleEvent(event);
+    }
+  }
+  
+  
+  private void handleEvent(Event event) {
+    if (event instanceof InputInformationEvent) {
+      processInputInformationEvent((InputInformationEvent) event);
+    }
+    else if (event instanceof DataMovementEvent) {
+      processDataMovementEvent((DataMovementEvent) event);      
+    } else if (event instanceof InputFailedEvent) {
+      processTaskFailedEvent((InputFailedEvent) event);
+    }
+  }
+
+  private void processInputInformationEvent(InputInformationEvent iiEvent) {
+    InputInformationEventPayloadProto inputInfoPayload;
+    try {
+      inputInfoPayload = InputInformationEventPayloadProto.parseFrom(iiEvent.getUserPayload());
+    } catch (InvalidProtocolBufferException e) {
+      throw new TezUncheckedException("Unable to parse InputInformationEvent payload", e);
+    }
+    int partitionRange = inputInfoPayload.getPartitionRange();
+    shuffle.setPartitionRange(partitionRange);
+    this.shuffleRangeSet = true;
+  }
+
+  private void processDataMovementEvent(DataMovementEvent dmEvent) {
+    // FIXME TODO NEWTEZ
+    // Preconditions.checkState(shuffleRangeSet == true, "Shuffle Range must be set before a DataMovementEvent is processed");
+    DataMovementEventPayloadProto shufflePayload;
+    try {
+      shufflePayload = DataMovementEventPayloadProto.parseFrom(dmEvent.getUserPayload());
+    } catch (InvalidProtocolBufferException e) {
+      throw new TezUncheckedException("Unable to parse DataMovementEvent payload", e);
+    } 
+    int partitionId = dmEvent.getSourceIndex();
+    URI baseUri = getBaseURI(shufflePayload.getHost(), shufflePayload.getPort(), partitionId);
+
+    InputAttemptIdentifier srcAttemptIdentifier = new InputAttemptIdentifier(dmEvent.getTargetIndex(), dmEvent.getVersion(), shufflePayload.getPathComponent());
+    scheduler.addKnownMapOutput(shufflePayload.getHost(), partitionId, baseUri.toString(), srcAttemptIdentifier);
+    
+    // TODO NEWTEZ See if this duration hack can be removed.
+    int duration = shufflePayload.getRunDuration();
+    if (duration > maxMapRuntime) {
+      maxMapRuntime = duration;
+      scheduler.informMaxMapRunTime(maxMapRuntime);
+    }
+  }
+  
+  private void processTaskFailedEvent(InputFailedEvent ifEvent) {
+    InputAttemptIdentifier taIdentifier = new InputAttemptIdentifier(ifEvent.getSourceIndex(), ifEvent.getVersion());
+    scheduler.obsoleteMapOutput(taIdentifier);
+    LOG.info("Obsoleting output of src-task: " + taIdentifier);
+  }
+
+  // TODO NEWTEZ Handle encrypted shuffle
+  private URI getBaseURI(String host, int port, int partitionId) {
+    StringBuilder sb = new StringBuilder("http://");
+    sb.append(host);
+    sb.append(":");
+    sb.append(String.valueOf(port));
+    sb.append("/");
+    
+    sb.append("mapOutput?job=");
+    // Required to use the existing ShuffleHandler
+    sb.append(inputContext.getApplicationId().toString().replace("application", "job"));
+    
+    sb.append("&reduce=");
+    sb.append(partitionId);
+    sb.append("&map=");
+    URI u = URI.create(sb.toString());
+    return u;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleScheduler.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleScheduler.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleScheduler.java
new file mode 100644
index 0000000..a682a09
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleScheduler.java
@@ -0,0 +1,521 @@
+/**
+ * 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.library.common.shuffle.impl;
+
+import java.io.IOException;
+import java.text.DecimalFormat;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.DelayQueue;
+import java.util.concurrent.Delayed;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.commons.lang.mutable.MutableInt;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.IntWritable;
+import org.apache.tez.common.TezJobConfig;
+import org.apache.tez.common.counters.TezCounter;
+import org.apache.tez.runtime.api.Event;
+import org.apache.tez.runtime.api.TezInputContext;
+import org.apache.tez.runtime.api.events.InputReadErrorEvent;
+import org.apache.tez.runtime.library.common.InputAttemptIdentifier;
+import org.apache.tez.runtime.library.common.TezRuntimeUtils;
+
+import com.google.common.collect.Lists;
+
+class ShuffleScheduler {
+  static ThreadLocal<Long> shuffleStart = new ThreadLocal<Long>() {
+    protected Long initialValue() {
+      return 0L;
+    }
+  };
+
+  private static final Log LOG = LogFactory.getLog(ShuffleScheduler.class);
+  private static final int MAX_MAPS_AT_ONCE = 20;
+  private static final long INITIAL_PENALTY = 10000;
+  private static final float PENALTY_GROWTH_RATE = 1.3f;
+  
+  // TODO NEWTEZ May need to be a string if attempting to fetch from multiple inputs.
+  private final Map<Integer, MutableInt> finishedMaps;
+  private final int numInputs;
+  private int remainingMaps;
+  private Map<InputAttemptIdentifier, MapHost> mapLocations = new HashMap<InputAttemptIdentifier, MapHost>();
+  //TODO NEWTEZ Clean this and other maps at some point
+  private ConcurrentMap<String, InputAttemptIdentifier> pathToIdentifierMap = new ConcurrentHashMap<String, InputAttemptIdentifier>(); 
+  private Set<MapHost> pendingHosts = new HashSet<MapHost>();
+  private Set<InputAttemptIdentifier> obsoleteMaps = new HashSet<InputAttemptIdentifier>();
+  
+  private final Random random = new Random(System.currentTimeMillis());
+  private final DelayQueue<Penalty> penalties = new DelayQueue<Penalty>();
+  private final Referee referee = new Referee();
+  private final Map<InputAttemptIdentifier, IntWritable> failureCounts =
+    new HashMap<InputAttemptIdentifier,IntWritable>(); 
+  private final Map<String,IntWritable> hostFailures = 
+    new HashMap<String,IntWritable>();
+  private final TezInputContext inputContext;
+  private final Shuffle shuffle;
+  private final int abortFailureLimit;
+  private final TezCounter shuffledMapsCounter;
+  private final TezCounter reduceShuffleBytes;
+  private final TezCounter failedShuffleCounter;
+  
+  private final long startTime;
+  private long lastProgressTime;
+  
+  private int maxMapRuntime = 0;
+  private int maxFailedUniqueFetches = 5;
+  private int maxFetchFailuresBeforeReporting;
+  
+  private long totalBytesShuffledTillNow = 0;
+  private DecimalFormat  mbpsFormat = new DecimalFormat("0.00");
+
+  private boolean reportReadErrorImmediately = true;
+  
+  public ShuffleScheduler(TezInputContext inputContext,
+                          Configuration conf,
+                          int tasksInDegree,
+                          Shuffle shuffle,
+                          TezCounter shuffledMapsCounter,
+                          TezCounter reduceShuffleBytes,
+                          TezCounter failedShuffleCounter) {
+    this.inputContext = inputContext;
+    this.numInputs = tasksInDegree;
+    abortFailureLimit = Math.max(30, tasksInDegree / 10);
+    remainingMaps = tasksInDegree;
+  //TODO NEWTEZ May need to be a string or a more usable construct if attempting to fetch from multiple inputs. Define a taskId / taskAttemptId pair
+    finishedMaps = new HashMap<Integer, MutableInt>(remainingMaps);
+    this.shuffle = shuffle;
+    this.shuffledMapsCounter = shuffledMapsCounter;
+    this.reduceShuffleBytes = reduceShuffleBytes;
+    this.failedShuffleCounter = failedShuffleCounter;
+    this.startTime = System.currentTimeMillis();
+    this.lastProgressTime = startTime;
+    referee.start();
+    this.maxFailedUniqueFetches = Math.min(tasksInDegree,
+        this.maxFailedUniqueFetches);
+    this.maxFetchFailuresBeforeReporting = 
+        conf.getInt(
+            TezJobConfig.TEZ_RUNTIME_SHUFFLE_FETCH_FAILURES, 
+            TezJobConfig.DEFAULT_TEZ_RUNTIME_SHUFFLE_FETCH_FAILURES_LIMIT);
+    this.reportReadErrorImmediately = 
+        conf.getBoolean(
+            TezJobConfig.TEZ_RUNTIME_SHUFFLE_NOTIFY_READERROR, 
+            TezJobConfig.DEFAULT_TEZ_RUNTIME_SHUFFLE_NOTIFY_READERROR);
+  }
+
+  public synchronized void copySucceeded(InputAttemptIdentifier srcAttemptIdentifier, 
+                                         MapHost host,
+                                         long bytes,
+                                         long milis,
+                                         MapOutput output
+                                         ) throws IOException {
+    String taskIdentifier = TezRuntimeUtils.getTaskAttemptIdentifier(srcAttemptIdentifier.getInputIdentifier().getSrcTaskIndex(), srcAttemptIdentifier.getAttemptNumber());
+    failureCounts.remove(taskIdentifier);
+    hostFailures.remove(host.getHostName());
+    
+    if (!isFinishedTaskTrue(srcAttemptIdentifier.getInputIdentifier().getSrcTaskIndex())) {
+      output.commit();
+      if(incrementTaskCopyAndCheckCompletion(srcAttemptIdentifier.getInputIdentifier().getSrcTaskIndex())) {
+        shuffledMapsCounter.increment(1);
+        if (--remainingMaps == 0) {
+          notifyAll();
+        }
+      }
+
+      // update the status
+      lastProgressTime = System.currentTimeMillis();
+      totalBytesShuffledTillNow += bytes;
+      logProgress();
+      reduceShuffleBytes.increment(bytes);
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("src task: "
+            + TezRuntimeUtils.getTaskAttemptIdentifier(
+                inputContext.getSourceVertexName(), srcAttemptIdentifier.getInputIdentifier().getSrcTaskIndex(),
+                srcAttemptIdentifier.getAttemptNumber()) + " done");
+      }
+    }
+    // TODO NEWTEZ Should this be releasing the output, if not committed ? Possible memory leak in case of speculation.
+  }
+
+  private void logProgress() {
+    float mbs = (float) totalBytesShuffledTillNow / (1024 * 1024);
+    int mapsDone = numInputs - remainingMaps;
+    long secsSinceStart = (System.currentTimeMillis() - startTime) / 1000 + 1;
+
+    float transferRate = mbs / secsSinceStart;
+    LOG.info("copy(" + mapsDone + " of " + numInputs + " at "
+        + mbpsFormat.format(transferRate) + " MB/s)");
+  }
+
+  public synchronized void copyFailed(InputAttemptIdentifier srcAttempt,
+                                      MapHost host,
+                                      boolean readError) {
+    host.penalize();
+    int failures = 1;
+    if (failureCounts.containsKey(srcAttempt)) {
+      IntWritable x = failureCounts.get(srcAttempt);
+      x.set(x.get() + 1);
+      failures = x.get();
+    } else {
+      failureCounts.put(srcAttempt, new IntWritable(1));      
+    }
+    String hostname = host.getHostName();
+    if (hostFailures.containsKey(hostname)) {
+      IntWritable x = hostFailures.get(hostname);
+      x.set(x.get() + 1);
+    } else {
+      hostFailures.put(hostname, new IntWritable(1));
+    }
+    if (failures >= abortFailureLimit) {
+      try {
+        throw new IOException(failures
+            + " failures downloading "
+            + TezRuntimeUtils.getTaskAttemptIdentifier(
+                inputContext.getSourceVertexName(), srcAttempt.getInputIdentifier().getSrcTaskIndex(),
+                srcAttempt.getAttemptNumber()));
+      } catch (IOException ie) {
+        shuffle.reportException(ie);
+      }
+    }
+    
+    checkAndInformJobTracker(failures, srcAttempt, readError);
+
+    checkReducerHealth();
+    
+    long delay = (long) (INITIAL_PENALTY *
+        Math.pow(PENALTY_GROWTH_RATE, failures));
+    
+    penalties.add(new Penalty(host, delay));
+    
+    failedShuffleCounter.increment(1);
+  }
+  
+  // Notify the JobTracker  
+  // after every read error, if 'reportReadErrorImmediately' is true or
+  // after every 'maxFetchFailuresBeforeReporting' failures
+  private void checkAndInformJobTracker(
+      int failures, InputAttemptIdentifier srcAttempt, boolean readError) {
+    if ((reportReadErrorImmediately && readError)
+        || ((failures % maxFetchFailuresBeforeReporting) == 0)) {
+      LOG.info("Reporting fetch failure for "
+          + TezRuntimeUtils.getTaskAttemptIdentifier(
+              inputContext.getSourceVertexName(), srcAttempt.getInputIdentifier().getSrcTaskIndex(),
+              srcAttempt.getAttemptNumber()) + " to jobtracker.");
+
+      List<Event> failedEvents = Lists.newArrayListWithCapacity(1);
+      failedEvents.add(new InputReadErrorEvent("Fetch failure for "
+          + TezRuntimeUtils.getTaskAttemptIdentifier(
+              inputContext.getSourceVertexName(), srcAttempt.getInputIdentifier().getSrcTaskIndex(),
+              srcAttempt.getAttemptNumber()) + " to jobtracker.", srcAttempt.getInputIdentifier()
+          .getSrcTaskIndex(), srcAttempt.getAttemptNumber()));
+
+      inputContext.sendEvents(failedEvents);      
+      //status.addFailedDependency(mapId);
+    }
+  }
+    
+  private void checkReducerHealth() {
+    final float MAX_ALLOWED_FAILED_FETCH_ATTEMPT_PERCENT = 0.5f;
+    final float MIN_REQUIRED_PROGRESS_PERCENT = 0.5f;
+    final float MAX_ALLOWED_STALL_TIME_PERCENT = 0.5f;
+
+    long totalFailures = failedShuffleCounter.getValue();
+    int doneMaps = numInputs - remainingMaps;
+    
+    boolean reducerHealthy =
+      (((float)totalFailures / (totalFailures + doneMaps))
+          < MAX_ALLOWED_FAILED_FETCH_ATTEMPT_PERCENT);
+    
+    // check if the reducer has progressed enough
+    boolean reducerProgressedEnough =
+      (((float)doneMaps / numInputs)
+          >= MIN_REQUIRED_PROGRESS_PERCENT);
+
+    // check if the reducer is stalled for a long time
+    // duration for which the reducer is stalled
+    int stallDuration =
+      (int)(System.currentTimeMillis() - lastProgressTime);
+    
+    // duration for which the reducer ran with progress
+    int shuffleProgressDuration =
+      (int)(lastProgressTime - startTime);
+
+    // min time the reducer should run without getting killed
+    int minShuffleRunDuration =
+      (shuffleProgressDuration > maxMapRuntime)
+      ? shuffleProgressDuration
+          : maxMapRuntime;
+    
+    boolean reducerStalled =
+      (((float)stallDuration / minShuffleRunDuration)
+          >= MAX_ALLOWED_STALL_TIME_PERCENT);
+
+    // kill if not healthy and has insufficient progress
+    if ((failureCounts.size() >= maxFailedUniqueFetches ||
+        failureCounts.size() == (numInputs - doneMaps))
+        && !reducerHealthy
+        && (!reducerProgressedEnough || reducerStalled)) {
+      LOG.fatal("Shuffle failed with too many fetch failures " +
+      "and insufficient progress!");
+      String errorMsg = "Exceeded MAX_FAILED_UNIQUE_FETCHES; bailing-out.";
+      shuffle.reportException(new IOException(errorMsg));
+    }
+
+  }
+  
+  public synchronized void tipFailed(int srcTaskIndex) {
+    if (!isFinishedTaskTrue(srcTaskIndex)) {
+      setFinishedTaskTrue(srcTaskIndex);
+      if (--remainingMaps == 0) {
+        notifyAll();
+      }
+      logProgress();
+    }
+  }
+  
+  public synchronized void addKnownMapOutput(String hostName,
+                                             int partitionId,
+                                             String hostUrl,
+                                             InputAttemptIdentifier srcAttempt) {
+    String identifier = MapHost.createIdentifier(hostName, partitionId);
+    MapHost host = mapLocations.get(identifier);
+    if (host == null) {
+      host = new MapHost(partitionId, hostName, hostUrl);
+      assert identifier.equals(host.getIdentifier());
+      mapLocations.put(srcAttempt, host);
+    }
+    host.addKnownMap(srcAttempt);
+    pathToIdentifierMap.put(srcAttempt.getPathComponent(), srcAttempt);
+
+    // Mark the host as pending
+    if (host.getState() == MapHost.State.PENDING) {
+      pendingHosts.add(host);
+      notifyAll();
+    }
+  }
+  
+  public synchronized void obsoleteMapOutput(InputAttemptIdentifier srcAttempt) {
+    // The incoming srcAttempt does not contain a path component.
+    obsoleteMaps.add(srcAttempt);
+  }
+  
+  public synchronized void putBackKnownMapOutput(MapHost host,
+                                                 InputAttemptIdentifier srcAttempt) {
+    host.addKnownMap(srcAttempt);
+  }
+
+  public synchronized MapHost getHost() throws InterruptedException {
+      while(pendingHosts.isEmpty()) {
+        wait();
+      }
+      
+      MapHost host = null;
+      Iterator<MapHost> iter = pendingHosts.iterator();
+      int numToPick = random.nextInt(pendingHosts.size());
+      for (int i=0; i <= numToPick; ++i) {
+        host = iter.next();
+      }
+      
+      pendingHosts.remove(host);     
+      host.markBusy();
+      
+      LOG.info("Assigning " + host + " with " + host.getNumKnownMapOutputs() + 
+               " to " + Thread.currentThread().getName());
+      shuffleStart.set(System.currentTimeMillis());
+      
+      return host;
+  }
+  
+  public InputAttemptIdentifier getIdentifierForPathComponent(String pathComponent) {
+    return pathToIdentifierMap.get(pathComponent);
+  }
+  
+  public synchronized List<InputAttemptIdentifier> getMapsForHost(MapHost host) {
+    List<InputAttemptIdentifier> list = host.getAndClearKnownMaps();
+    Iterator<InputAttemptIdentifier> itr = list.iterator();
+    List<InputAttemptIdentifier> result = new ArrayList<InputAttemptIdentifier>();
+    int includedMaps = 0;
+    int totalSize = list.size();
+    // find the maps that we still need, up to the limit
+    while (itr.hasNext()) {
+      InputAttemptIdentifier id = itr.next();
+      if (!obsoleteMaps.contains(id) && !isFinishedTaskTrue(id.getInputIdentifier().getSrcTaskIndex())) {
+        result.add(id);
+        if (++includedMaps >= MAX_MAPS_AT_ONCE) {
+          break;
+        }
+      }
+    }
+    // put back the maps left after the limit
+    while (itr.hasNext()) {
+      InputAttemptIdentifier id = itr.next();
+      if (!obsoleteMaps.contains(id) && !isFinishedTaskTrue(id.getInputIdentifier().getSrcTaskIndex())) {
+        host.addKnownMap(id);
+      }
+    }
+    LOG.info("assigned " + includedMaps + " of " + totalSize + " to " +
+             host + " to " + Thread.currentThread().getName());
+    return result;
+  }
+
+  public synchronized void freeHost(MapHost host) {
+    if (host.getState() != MapHost.State.PENALIZED) {
+      if (host.markAvailable() == MapHost.State.PENDING) {
+        pendingHosts.add(host);
+        notifyAll();
+      }
+    }
+    LOG.info(host + " freed by " + Thread.currentThread().getName() + " in " + 
+             (System.currentTimeMillis()-shuffleStart.get()) + "s");
+  }
+    
+  public synchronized void resetKnownMaps() {
+    mapLocations.clear();
+    obsoleteMaps.clear();
+    pendingHosts.clear();
+    pathToIdentifierMap.clear();
+  }
+
+  /**
+   * Utility method to check if the Shuffle data fetch is complete.
+   * @return
+   */
+  public synchronized boolean isDone() {
+    return remainingMaps == 0;
+  }
+
+  /**
+   * Wait until the shuffle finishes or until the timeout.
+   * @param millis maximum wait time
+   * @return true if the shuffle is done
+   * @throws InterruptedException
+   */
+  public synchronized boolean waitUntilDone(int millis
+                                            ) throws InterruptedException {
+    if (remainingMaps > 0) {
+      wait(millis);
+      return remainingMaps == 0;
+    }
+    return true;
+  }
+  
+  /**
+   * A structure that records the penalty for a host.
+   */
+  private static class Penalty implements Delayed {
+    MapHost host;
+    private long endTime;
+    
+    Penalty(MapHost host, long delay) {
+      this.host = host;
+      this.endTime = System.currentTimeMillis() + delay;
+    }
+
+    public long getDelay(TimeUnit unit) {
+      long remainingTime = endTime - System.currentTimeMillis();
+      return unit.convert(remainingTime, TimeUnit.MILLISECONDS);
+    }
+
+    public int compareTo(Delayed o) {
+      long other = ((Penalty) o).endTime;
+      return endTime == other ? 0 : (endTime < other ? -1 : 1);
+    }
+    
+  }
+  
+  /**
+   * A thread that takes hosts off of the penalty list when the timer expires.
+   */
+  private class Referee extends Thread {
+    public Referee() {
+      setName("ShufflePenaltyReferee");
+      setDaemon(true);
+    }
+
+    public void run() {
+      try {
+        while (true) {
+          // take the first host that has an expired penalty
+          MapHost host = penalties.take().host;
+          synchronized (ShuffleScheduler.this) {
+            if (host.markAvailable() == MapHost.State.PENDING) {
+              pendingHosts.add(host);
+              ShuffleScheduler.this.notifyAll();
+            }
+          }
+        }
+      } catch (InterruptedException ie) {
+        return;
+      } catch (Throwable t) {
+        shuffle.reportException(t);
+      }
+    }
+  }
+  
+  public void close() throws InterruptedException {
+    referee.interrupt();
+    referee.join();
+  }
+
+  public synchronized void informMaxMapRunTime(int duration) {
+    if (duration > maxMapRuntime) {
+      maxMapRuntime = duration;
+    }
+  }
+  
+  void setFinishedTaskTrue(int srcTaskIndex) {
+    synchronized(finishedMaps) {
+      finishedMaps.put(srcTaskIndex, new MutableInt(shuffle.getReduceRange()));
+    }
+  }
+  
+  boolean incrementTaskCopyAndCheckCompletion(int srcTaskIndex) {
+    synchronized(finishedMaps) {
+      MutableInt result = finishedMaps.get(srcTaskIndex);
+      if(result == null) {
+        result = new MutableInt(0);
+        finishedMaps.put(srcTaskIndex, result);
+      }
+      result.increment();
+      return isFinishedTaskTrue(srcTaskIndex);
+    }
+  }
+  
+  boolean isFinishedTaskTrue(int srcTaskIndex) {
+    synchronized (finishedMaps) {
+      MutableInt result = finishedMaps.get(srcTaskIndex);
+      if(result == null) {
+        return false;
+      }
+      if (result.intValue() == shuffle.getReduceRange()) {
+        return true;
+      }
+      return false;      
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/server/ShuffleHandler.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/server/ShuffleHandler.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/server/ShuffleHandler.java
new file mode 100644
index 0000000..9a206c6
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/server/ShuffleHandler.java
@@ -0,0 +1,572 @@
+/**
+* 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.library.common.shuffle.server;
+
+import static org.jboss.netty.buffer.ChannelBuffers.wrappedBuffer;
+import static org.jboss.netty.handler.codec.http.HttpHeaders.Names.CONTENT_TYPE;
+import static org.jboss.netty.handler.codec.http.HttpMethod.GET;
+import static org.jboss.netty.handler.codec.http.HttpResponseStatus.BAD_REQUEST;
+import static org.jboss.netty.handler.codec.http.HttpResponseStatus.FORBIDDEN;
+import static org.jboss.netty.handler.codec.http.HttpResponseStatus.INTERNAL_SERVER_ERROR;
+import static org.jboss.netty.handler.codec.http.HttpResponseStatus.METHOD_NOT_ALLOWED;
+import static org.jboss.netty.handler.codec.http.HttpResponseStatus.NOT_FOUND;
+import static org.jboss.netty.handler.codec.http.HttpResponseStatus.OK;
+import static org.jboss.netty.handler.codec.http.HttpResponseStatus.UNAUTHORIZED;
+import static org.jboss.netty.handler.codec.http.HttpVersion.HTTP_1_1;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.net.URL;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.TimeUnit;
+
+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.io.DataInputByteBuffer;
+import org.apache.hadoop.io.DataOutputBuffer;
+import org.apache.hadoop.metrics2.MetricsSystem;
+import org.apache.hadoop.metrics2.annotation.Metric;
+import org.apache.hadoop.metrics2.annotation.Metrics;
+import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
+import org.apache.hadoop.metrics2.lib.MutableCounterInt;
+import org.apache.hadoop.metrics2.lib.MutableCounterLong;
+import org.apache.hadoop.metrics2.lib.MutableGaugeInt;
+import org.apache.hadoop.security.ssl.SSLFactory;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.server.api.ApplicationInitializationContext;
+import org.apache.hadoop.yarn.server.api.ApplicationTerminationContext;
+import org.apache.hadoop.yarn.server.api.AuxiliaryService;
+import org.apache.tez.common.TezJobConfig;
+import org.apache.tez.runtime.api.TezOutputContext;
+import org.apache.tez.runtime.library.common.security.JobTokenIdentifier;
+import org.apache.tez.runtime.library.common.security.JobTokenSecretManager;
+import org.apache.tez.runtime.library.common.security.SecureShuffleUtils;
+import org.apache.tez.runtime.library.common.shuffle.impl.ShuffleHeader;
+import org.apache.tez.runtime.library.common.sort.impl.ExternalSorter;
+import org.apache.tez.runtime.library.shuffle.common.ShuffleUtils;
+import org.jboss.netty.bootstrap.ServerBootstrap;
+import org.jboss.netty.buffer.ChannelBuffers;
+import org.jboss.netty.channel.Channel;
+import org.jboss.netty.channel.ChannelFactory;
+import org.jboss.netty.channel.ChannelFuture;
+import org.jboss.netty.channel.ChannelFutureListener;
+import org.jboss.netty.channel.ChannelHandlerContext;
+import org.jboss.netty.channel.ChannelPipeline;
+import org.jboss.netty.channel.ChannelPipelineFactory;
+import org.jboss.netty.channel.Channels;
+import org.jboss.netty.channel.ExceptionEvent;
+import org.jboss.netty.channel.MessageEvent;
+import org.jboss.netty.channel.SimpleChannelUpstreamHandler;
+import org.jboss.netty.channel.group.ChannelGroup;
+import org.jboss.netty.channel.group.DefaultChannelGroup;
+import org.jboss.netty.channel.socket.nio.NioServerSocketChannelFactory;
+import org.jboss.netty.handler.codec.frame.TooLongFrameException;
+import org.jboss.netty.handler.codec.http.DefaultHttpResponse;
+import org.jboss.netty.handler.codec.http.HttpChunkAggregator;
+import org.jboss.netty.handler.codec.http.HttpRequest;
+import org.jboss.netty.handler.codec.http.HttpRequestDecoder;
+import org.jboss.netty.handler.codec.http.HttpResponse;
+import org.jboss.netty.handler.codec.http.HttpResponseEncoder;
+import org.jboss.netty.handler.codec.http.HttpResponseStatus;
+import org.jboss.netty.handler.codec.http.QueryStringDecoder;
+import org.jboss.netty.handler.ssl.SslHandler;
+import org.jboss.netty.handler.stream.ChunkedStream;
+import org.jboss.netty.handler.stream.ChunkedWriteHandler;
+import org.jboss.netty.util.CharsetUtil;
+
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+
+public class ShuffleHandler extends AuxiliaryService {
+
+  private static final Log LOG = LogFactory.getLog(ShuffleHandler.class);
+  
+  public static final String SHUFFLE_MANAGE_OS_CACHE = "mapreduce.shuffle.manage.os.cache";
+  public static final boolean DEFAULT_SHUFFLE_MANAGE_OS_CACHE = true;
+
+  public static final String SHUFFLE_READAHEAD_BYTES = "mapreduce.shuffle.readahead.bytes";
+  public static final int DEFAULT_SHUFFLE_READAHEAD_BYTES = 4 * 1024 * 1024;
+
+  private int port;
+  private ChannelFactory selector;
+  private final ChannelGroup accepted = new DefaultChannelGroup();
+  private HttpPipelineFactory pipelineFact;
+  private int sslFileBufferSize;
+
+  public static final String MAPREDUCE_SHUFFLE_SERVICEID =
+      "mapreduce.shuffle";
+
+  private static final Map<String,String> userRsrc =
+    new ConcurrentHashMap<String,String>();
+  private static final JobTokenSecretManager secretManager =
+    new JobTokenSecretManager();
+  private SecretKey tokenSecret;
+
+  public static final String SHUFFLE_PORT_CONFIG_KEY = "mapreduce.shuffle.port";
+  public static final int DEFAULT_SHUFFLE_PORT = 8080;
+
+  public static final String SUFFLE_SSL_FILE_BUFFER_SIZE_KEY =
+    "mapreduce.shuffle.ssl.file.buffer.size";
+
+  public static final int DEFAULT_SUFFLE_SSL_FILE_BUFFER_SIZE = 60 * 1024;
+
+  private ExternalSorter sorter;
+  
+  @Metrics(about="Shuffle output metrics", context="mapred")
+  static class ShuffleMetrics implements ChannelFutureListener {
+    @Metric("Shuffle output in bytes")
+        MutableCounterLong shuffleOutputBytes;
+    @Metric("# of failed shuffle outputs")
+        MutableCounterInt shuffleOutputsFailed;
+    @Metric("# of succeeeded shuffle outputs")
+        MutableCounterInt shuffleOutputsOK;
+    @Metric("# of current shuffle connections")
+        MutableGaugeInt shuffleConnections;
+
+    @Override
+    public void operationComplete(ChannelFuture future) throws Exception {
+      if (future.isSuccess()) {
+        shuffleOutputsOK.incr();
+      } else {
+        shuffleOutputsFailed.incr();
+      }
+      shuffleConnections.decr();
+    }
+  }
+
+  final ShuffleMetrics metrics;
+
+  ShuffleHandler(MetricsSystem ms) {
+    super("httpshuffle");
+    metrics = ms.register(new ShuffleMetrics());
+  }
+
+  public ShuffleHandler(ExternalSorter sorter) {
+    this(DefaultMetricsSystem.instance());
+    this.sorter = sorter;
+  }
+
+  /**
+   * Serialize the shuffle port into a ByteBuffer for use later on.
+   * @param port the port to be sent to the ApplciationMaster
+   * @return the serialized form of the port.
+   */
+  public static ByteBuffer serializeMetaData(int port) throws IOException {
+    //TODO these bytes should be versioned
+    DataOutputBuffer port_dob = new DataOutputBuffer();
+    port_dob.writeInt(port);
+    return ByteBuffer.wrap(port_dob.getData(), 0, port_dob.getLength());
+  }
+
+  /**
+   * A helper function to deserialize the metadata returned by ShuffleHandler.
+   * @param meta the metadata returned by the ShuffleHandler
+   * @return the port the Shuffle Handler is listening on to serve shuffle data.
+   */
+  public static int deserializeMetaData(ByteBuffer meta) throws IOException {
+    //TODO this should be returning a class not just an int
+    DataInputByteBuffer in = new DataInputByteBuffer();
+    in.reset(meta);
+    int port = in.readInt();
+    return port;
+  }
+
+  /**
+   * A helper function to serialize the JobTokenIdentifier to be sent to the
+   * ShuffleHandler as ServiceData.
+   * @param jobToken the job token to be used for authentication of
+   * shuffle data requests.
+   * @return the serialized version of the jobToken.
+   */
+  public static ByteBuffer serializeServiceData(Token<JobTokenIdentifier> jobToken) throws IOException {
+    //TODO these bytes should be versioned
+    DataOutputBuffer jobToken_dob = new DataOutputBuffer();
+    jobToken.write(jobToken_dob);
+    return ByteBuffer.wrap(jobToken_dob.getData(), 0, jobToken_dob.getLength());
+  }
+
+  static Token<JobTokenIdentifier> deserializeServiceData(ByteBuffer secret) throws IOException {
+    DataInputByteBuffer in = new DataInputByteBuffer();
+    in.reset(secret);
+    Token<JobTokenIdentifier> jt = new Token<JobTokenIdentifier>();
+    jt.readFields(in);
+    return jt;
+  }
+
+  
+  @Override
+  public void initializeApplication(
+      ApplicationInitializationContext initAppContext) {
+    // TODO these bytes should be versioned
+    try {
+      String user = initAppContext.getUser();
+      ApplicationId appId = initAppContext.getApplicationId();
+      ByteBuffer secret = initAppContext.getApplicationDataForService();
+      Token<JobTokenIdentifier> jt = deserializeServiceData(secret);
+      // TODO: Once SHuffle is out of NM, this can use MR APIs
+      userRsrc.put(appId.toString(), user);
+      LOG.info("Added token for " + appId.toString());
+      secretManager.addTokenForJob(appId.toString(), jt);
+    } catch (IOException e) {
+      LOG.error("Error during initApp", e);
+      // TODO add API to AuxiliaryServices to report failures
+    }
+  }
+
+  @Override
+  public void stopApplication(ApplicationTerminationContext context) {
+    ApplicationId appId = context.getApplicationId();
+    secretManager.removeTokenForJob(appId.toString());
+    userRsrc.remove(appId.toString());
+  }
+
+  public void initialize(TezOutputContext outputContext, Configuration conf) throws IOException {
+    this.init(new Configuration(conf));
+    tokenSecret = ShuffleUtils.getJobTokenSecretFromTokenBytes(outputContext.getServiceConsumerMetaData(MAPREDUCE_SHUFFLE_SERVICEID));
+  }
+
+  @Override
+  public synchronized void serviceInit(Configuration conf) {
+    ThreadFactory bossFactory = new ThreadFactoryBuilder()
+      .setNameFormat("ShuffleHandler Netty Boss #%d")
+      .build();
+    ThreadFactory workerFactory = new ThreadFactoryBuilder()
+      .setNameFormat("ShuffleHandler Netty Worker #%d")
+      .build();
+    
+    selector = new NioServerSocketChannelFactory(
+        Executors.newCachedThreadPool(bossFactory),
+        Executors.newCachedThreadPool(workerFactory));    
+  }
+
+  // TODO change AbstractService to throw InterruptedException
+  @Override
+  public synchronized void serviceStart() {
+    Configuration conf = getConfig();
+    ServerBootstrap bootstrap = new ServerBootstrap(selector);
+    try {
+      pipelineFact = new HttpPipelineFactory(conf);
+    } catch (Exception ex) {
+      throw new RuntimeException(ex);
+    }
+    bootstrap.setPipelineFactory(pipelineFact);
+    // Let OS pick the port
+    Channel ch = bootstrap.bind(new InetSocketAddress(0));
+    accepted.add(ch);
+    port = ((InetSocketAddress)ch.getLocalAddress()).getPort();
+    conf.set(SHUFFLE_PORT_CONFIG_KEY, Integer.toString(port));
+    pipelineFact.SHUFFLE.setPort(port);
+    LOG.info(getName() + " listening on port " + port);
+
+    sslFileBufferSize = conf.getInt(SUFFLE_SSL_FILE_BUFFER_SIZE_KEY,
+                                    DEFAULT_SUFFLE_SSL_FILE_BUFFER_SIZE);
+  }
+
+  @Override
+  public synchronized void serviceStop() {
+    accepted.close().awaitUninterruptibly(10, TimeUnit.SECONDS);
+    ServerBootstrap bootstrap = new ServerBootstrap(selector);
+    bootstrap.releaseExternalResources();
+    pipelineFact.destroy();
+  }
+
+  @Override
+  public synchronized ByteBuffer getMetaData() {
+    try {
+      return serializeMetaData(port); 
+    } catch (IOException e) {
+      LOG.error("Error during getMeta", e);
+      // TODO add API to AuxiliaryServices to report failures
+      return null;
+    }
+  }
+
+  class HttpPipelineFactory implements ChannelPipelineFactory {
+
+    final Shuffle SHUFFLE;
+    private SSLFactory sslFactory;
+
+    public HttpPipelineFactory(Configuration conf) throws Exception {
+      SHUFFLE = new Shuffle(conf);
+      if (conf.getBoolean(TezJobConfig.TEZ_RUNTIME_SHUFFLE_ENABLE_SSL,
+              TezJobConfig.DEFAULT_TEZ_RUNTIME_SHUFFLE_ENABLE_SSL)) {
+        sslFactory = new SSLFactory(SSLFactory.Mode.SERVER, conf);
+        sslFactory.init();
+      }
+    }
+
+    public void destroy() {
+      if (sslFactory != null) {
+        sslFactory.destroy();
+      }
+    }
+
+    @Override
+    public ChannelPipeline getPipeline() throws Exception {
+      ChannelPipeline pipeline = Channels.pipeline();
+      if (sslFactory != null) {
+        pipeline.addLast("ssl", new SslHandler(sslFactory.createSSLEngine()));
+      }
+      pipeline.addLast("decoder", new HttpRequestDecoder());
+      pipeline.addLast("aggregator", new HttpChunkAggregator(1 << 16));
+      pipeline.addLast("encoder", new HttpResponseEncoder());
+      pipeline.addLast("chunking", new ChunkedWriteHandler());
+      pipeline.addLast("shuffle", SHUFFLE);
+      return pipeline;
+      // TODO factor security manager into pipeline
+      // TODO factor out encode/decode to permit binary shuffle
+      // TODO factor out decode of index to permit alt. models
+    }
+
+  }
+
+  class Shuffle extends SimpleChannelUpstreamHandler {
+
+    private final Configuration conf;
+    private int port;
+
+    public Shuffle(Configuration conf) {
+      this.conf = conf;
+      this.port = conf.getInt(SHUFFLE_PORT_CONFIG_KEY, DEFAULT_SHUFFLE_PORT);
+    }
+    
+    public void setPort(int port) {
+      this.port = port;
+    }
+
+    private List<String> splitMaps(List<String> mapq) {
+      if (null == mapq) {
+        return null;
+      }
+      final List<String> ret = new ArrayList<String>();
+      for (String s : mapq) {
+        Collections.addAll(ret, s.split(","));
+      }
+      return ret;
+    }
+
+    @Override
+    public void messageReceived(ChannelHandlerContext ctx, MessageEvent evt)
+        throws Exception {
+      HttpRequest request = (HttpRequest) evt.getMessage();
+      if (request.getMethod() != GET) {
+          sendError(ctx, METHOD_NOT_ALLOWED);
+          return;
+      }
+      // Check whether the shuffle version is compatible
+      if (!ShuffleHeader.DEFAULT_HTTP_HEADER_NAME.equals(
+          request.getHeader(ShuffleHeader.HTTP_HEADER_NAME))
+          || !ShuffleHeader.DEFAULT_HTTP_HEADER_VERSION.equals(
+              request.getHeader(ShuffleHeader.HTTP_HEADER_VERSION))) {
+        sendError(ctx, "Incompatible shuffle request version", BAD_REQUEST);
+      }
+      final Map<String,List<String>> q =
+        new QueryStringDecoder(request.getUri()).getParameters();
+      final List<String> mapIds = splitMaps(q.get("map"));
+      final List<String> reduceQ = q.get("reduce");
+      final List<String> jobQ = q.get("job");
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("RECV: " + request.getUri() +
+            "\n  mapId: " + mapIds +
+            "\n  reduceId: " + reduceQ +
+            "\n  jobId: " + jobQ);
+      }
+
+      if (mapIds == null || reduceQ == null || jobQ == null) {
+        sendError(ctx, "Required param job, map and reduce", BAD_REQUEST);
+        return;
+      }
+      if (reduceQ.size() != 1 || jobQ.size() != 1) {
+        sendError(ctx, "Too many job/reduce parameters", BAD_REQUEST);
+        return;
+      }
+      int reduceId;
+      String jobId;
+      try {
+        reduceId = Integer.parseInt(reduceQ.get(0));
+        jobId = jobQ.get(0);
+      } catch (NumberFormatException e) {
+        sendError(ctx, "Bad reduce parameter", BAD_REQUEST);
+        return;
+      } catch (IllegalArgumentException e) {
+        sendError(ctx, "Bad job parameter", BAD_REQUEST);
+        return;
+      }
+
+      final String reqUri = request.getUri();
+      if (null == reqUri) {
+        // TODO? add upstream?
+        sendError(ctx, FORBIDDEN);
+        return;
+      }
+      HttpResponse response = new DefaultHttpResponse(HTTP_1_1, OK);
+      try {
+        verifyRequest(jobId, ctx, request, response,
+            new URL("http", "", this.port, reqUri));
+      } catch (IOException e) {
+        LOG.warn("Shuffle failure ", e);
+        sendError(ctx, e.getMessage(), UNAUTHORIZED);
+        return;
+      }
+
+      Channel ch = evt.getChannel();
+      ch.write(response);
+      // TODO refactor the following into the pipeline
+      ChannelFuture lastMap = null;
+      for (String mapId : mapIds) {
+        try {
+          // TODO: Error handling - validate mapId via TezTaskAttemptId.forName
+          
+          // TODO NEWTEZ Fix this. TaskAttemptId is no longer valid. mapId validation will not work anymore.
+//          if (!mapId.equals(sorter.getTaskAttemptId().toString())) {
+//            String errorMessage =
+//                "Illegal shuffle request mapId: " + mapId
+//                    + " while actual mapId is " + sorter.getTaskAttemptId(); 
+//            LOG.warn(errorMessage);
+//            sendError(ctx, errorMessage, BAD_REQUEST);
+//            return;
+//          }
+
+          lastMap =
+            sendMapOutput(ctx, ch, userRsrc.get(jobId), jobId, mapId, reduceId);
+          if (null == lastMap) {
+            sendError(ctx, NOT_FOUND);
+            return;
+          }
+        } catch (IOException e) {
+          LOG.error("Shuffle error ", e);
+          sendError(ctx, e.getMessage(), INTERNAL_SERVER_ERROR);
+          return;
+        }
+      }
+      lastMap.addListener(metrics);
+      lastMap.addListener(ChannelFutureListener.CLOSE);
+    }
+
+    private void verifyRequest(String appid, ChannelHandlerContext ctx,
+        HttpRequest request, HttpResponse response, URL requestUri)
+        throws IOException {
+      if (null == tokenSecret) {
+        LOG.info("Request for unknown token " + appid);
+        throw new IOException("could not find jobid");
+      }
+      // string to encrypt
+      String enc_str = SecureShuffleUtils.buildMsgFrom(requestUri);
+      // hash from the fetcher
+      String urlHashStr =
+        request.getHeader(SecureShuffleUtils.HTTP_HEADER_URL_HASH);
+      if (urlHashStr == null) {
+        LOG.info("Missing header hash for " + appid);
+        throw new IOException("fetcher cannot be authenticated");
+      }
+      if (LOG.isDebugEnabled()) {
+        int len = urlHashStr.length();
+        LOG.debug("verifying request. enc_str=" + enc_str + "; hash=..." +
+            urlHashStr.substring(len-len/2, len-1));
+      }
+      // verify - throws exception
+      SecureShuffleUtils.verifyReply(urlHashStr, enc_str, tokenSecret);
+      // verification passed - encode the reply
+      String reply =
+        SecureShuffleUtils.generateHash(urlHashStr.getBytes(), tokenSecret);
+      response.setHeader(SecureShuffleUtils.HTTP_HEADER_REPLY_URL_HASH, reply);
+      addVersionToHeader(response);
+      if (LOG.isDebugEnabled()) {
+        int len = reply.length();
+        LOG.debug("Fetcher request verfied. enc_str=" + enc_str + ";reply=" +
+            reply.substring(len-len/2, len-1));
+      }
+    }
+
+    protected ChannelFuture sendMapOutput(ChannelHandlerContext ctx, Channel ch,
+        String user, String jobId, String mapId, int reduce)
+        throws IOException {
+      final ShuffleHeader header = sorter.getShuffleHeader(reduce);
+      final DataOutputBuffer dob = new DataOutputBuffer();
+      header.write(dob);
+      ch.write(wrappedBuffer(dob.getData(), 0, dob.getLength()));
+
+      ChannelFuture writeFuture =
+          ch.write(
+              new ChunkedStream(
+                  sorter.getSortedStream(reduce), sslFileBufferSize
+                  )
+              );
+      metrics.shuffleConnections.incr();
+      metrics.shuffleOutputBytes.incr(header.getCompressedLength()); // optimistic
+      return writeFuture;
+    }
+
+    private void sendError(ChannelHandlerContext ctx,
+        HttpResponseStatus status) {
+      sendError(ctx, "", status);
+    }
+
+    private void sendError(ChannelHandlerContext ctx, String message,
+        HttpResponseStatus status) {
+      HttpResponse response = new DefaultHttpResponse(HTTP_1_1, status);
+      response.setHeader(CONTENT_TYPE, "text/plain; charset=UTF-8");
+      addVersionToHeader(response);
+      response.setContent(
+        ChannelBuffers.copiedBuffer(message, CharsetUtil.UTF_8));
+      // Close the connection as soon as the error message is sent.
+      ctx.getChannel().write(response).addListener(ChannelFutureListener.CLOSE);
+    }
+    
+    private void addVersionToHeader(HttpResponse response) {
+      // Put shuffle version into http header
+      response.setHeader(ShuffleHeader.HTTP_HEADER_NAME,
+          ShuffleHeader.DEFAULT_HTTP_HEADER_NAME);
+      response.setHeader(ShuffleHeader.HTTP_HEADER_VERSION,
+          ShuffleHeader.DEFAULT_HTTP_HEADER_VERSION);      
+    }
+
+    @Override
+    public void exceptionCaught(ChannelHandlerContext ctx, ExceptionEvent e)
+        throws Exception {
+      Channel ch = e.getChannel();
+      Throwable cause = e.getCause();
+      if (cause instanceof TooLongFrameException) {
+        sendError(ctx, BAD_REQUEST);
+        return;
+      }
+
+      LOG.error("Shuffle error: ", cause);
+      if (ch.isConnected()) {
+        LOG.error("Shuffle error " + e);
+        sendError(ctx, INTERNAL_SERVER_ERROR);
+      }
+    }
+
+  }
+
+  public int getPort() {
+    return port;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/ExternalSorter.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/ExternalSorter.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/ExternalSorter.java
new file mode 100644
index 0000000..c362d98
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/ExternalSorter.java
@@ -0,0 +1,194 @@
+/**
+* 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.library.common.sort.impl;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+
+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.fs.FileSystem;
+import org.apache.hadoop.fs.LocalFileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RawLocalFileSystem;
+import org.apache.hadoop.io.RawComparator;
+import org.apache.hadoop.io.compress.CompressionCodec;
+import org.apache.hadoop.io.compress.DefaultCodec;
+import org.apache.hadoop.io.serializer.SerializationFactory;
+import org.apache.hadoop.io.serializer.Serializer;
+import org.apache.hadoop.util.IndexedSorter;
+import org.apache.hadoop.util.Progressable;
+import org.apache.hadoop.util.QuickSort;
+import org.apache.hadoop.util.ReflectionUtils;
+import org.apache.tez.common.TezJobConfig;
+import org.apache.tez.common.counters.TaskCounter;
+import org.apache.tez.common.counters.TezCounter;
+import org.apache.tez.runtime.api.TezOutputContext;
+import org.apache.tez.runtime.library.api.Partitioner;
+import org.apache.tez.runtime.library.common.ConfigUtils;
+import org.apache.tez.runtime.library.common.TezRuntimeUtils;
+import org.apache.tez.runtime.library.common.combine.Combiner;
+import org.apache.tez.runtime.library.common.shuffle.impl.ShuffleHeader;
+import org.apache.tez.runtime.library.common.sort.impl.IFile.Writer;
+import org.apache.tez.runtime.library.common.task.local.output.TezTaskOutput;
+import org.apache.tez.runtime.library.hadoop.compat.NullProgressable;
+
+@SuppressWarnings({"unchecked", "rawtypes"})
+public abstract class ExternalSorter {
+
+  private static final Log LOG = LogFactory.getLog(ExternalSorter.class);
+
+  public abstract void close() throws IOException;
+
+  public abstract void flush() throws IOException;
+
+  public abstract void write(Object key, Object value) throws IOException;
+
+  protected Progressable nullProgressable = new NullProgressable();
+  protected TezOutputContext outputContext;
+  protected Combiner combiner;
+  protected Partitioner partitioner;
+  protected Configuration conf;
+  protected FileSystem rfs;
+  protected TezTaskOutput mapOutputFile;
+  protected int partitions;
+  protected Class keyClass;
+  protected Class valClass;
+  protected RawComparator comparator;
+  protected SerializationFactory serializationFactory;
+  protected Serializer keySerializer;
+  protected Serializer valSerializer;
+
+  protected IndexedSorter sorter;
+
+  // Compression for map-outputs
+  protected CompressionCodec codec;
+
+  // Counters
+  // TODO TEZ Rename all counter variables [Mapping of counter to MR for compatibility in the MR layer]
+  protected TezCounter mapOutputByteCounter;
+  protected TezCounter mapOutputRecordCounter;
+  protected TezCounter fileOutputByteCounter;
+  protected TezCounter spilledRecordsCounter;
+
+  public void initialize(TezOutputContext outputContext, Configuration conf, int numOutputs) throws IOException {
+    this.outputContext = outputContext;
+    this.conf = conf;
+    this.partitions = numOutputs;
+
+    rfs = ((LocalFileSystem)FileSystem.getLocal(this.conf)).getRaw();
+
+    // sorter
+    sorter = ReflectionUtils.newInstance(this.conf.getClass(
+        TezJobConfig.TEZ_RUNTIME_INTERNAL_SORTER_CLASS, QuickSort.class,
+        IndexedSorter.class), this.conf);
+
+    comparator = ConfigUtils.getIntermediateOutputKeyComparator(this.conf);
+
+    // k/v serialization
+    keyClass = ConfigUtils.getIntermediateOutputKeyClass(this.conf);
+    valClass = ConfigUtils.getIntermediateOutputValueClass(this.conf);
+    serializationFactory = new SerializationFactory(this.conf);
+    keySerializer = serializationFactory.getSerializer(keyClass);
+    valSerializer = serializationFactory.getSerializer(valClass);
+
+    //    counters
+    mapOutputByteCounter =
+        outputContext.getCounters().findCounter(TaskCounter.MAP_OUTPUT_BYTES);
+    mapOutputRecordCounter =
+        outputContext.getCounters().findCounter(TaskCounter.MAP_OUTPUT_RECORDS);
+    fileOutputByteCounter =
+        outputContext.getCounters().findCounter(TaskCounter.MAP_OUTPUT_MATERIALIZED_BYTES);
+    spilledRecordsCounter =
+        outputContext.getCounters().findCounter(TaskCounter.SPILLED_RECORDS);
+    // compression
+    if (ConfigUtils.shouldCompressIntermediateOutput(this.conf)) {
+      Class<? extends CompressionCodec> codecClass =
+          ConfigUtils.getIntermediateOutputCompressorClass(this.conf, DefaultCodec.class);
+      codec = ReflectionUtils.newInstance(codecClass, this.conf);
+    } else {
+      codec = null;
+    }
+
+    // Task outputs
+    mapOutputFile = TezRuntimeUtils.instantiateTaskOutputManager(conf, outputContext);
+    
+    LOG.info("Instantiating Partitioner: [" + conf.get(TezJobConfig.TEZ_RUNTIME_PARTITIONER_CLASS) + "]");
+    this.conf.setInt(TezJobConfig.TEZ_RUNTIME_NUM_EXPECTED_PARTITIONS, this.partitions);
+    this.partitioner = TezRuntimeUtils.instantiatePartitioner(this.conf);
+    this.combiner = TezRuntimeUtils.instantiateCombiner(this.conf, outputContext);
+  }
+
+  /**
+   * Exception indicating that the allocated sort buffer is insufficient to hold
+   * the current record.
+   */
+  @SuppressWarnings("serial")
+  public static class MapBufferTooSmallException extends IOException {
+    public MapBufferTooSmallException(String s) {
+      super(s);
+    }
+  }
+
+  @Private
+  public TezTaskOutput getMapOutput() {
+    return mapOutputFile;
+  }
+
+  protected void runCombineProcessor(TezRawKeyValueIterator kvIter,
+      Writer writer) throws IOException {
+    try {
+      combiner.combine(kvIter, writer);
+    } catch (InterruptedException e) {
+      throw new IOException(e);
+    }
+  }
+
+  /**
+   * Rename srcPath to dstPath on the same volume. This is the same as
+   * RawLocalFileSystem's rename method, except that it will not fall back to a
+   * copy, and it will create the target directory if it doesn't exist.
+   */
+  protected void sameVolRename(Path srcPath, Path dstPath) throws IOException {
+    RawLocalFileSystem rfs = (RawLocalFileSystem) this.rfs;
+    File src = rfs.pathToFile(srcPath);
+    File dst = rfs.pathToFile(dstPath);
+    if (!dst.getParentFile().exists()) {
+      if (!dst.getParentFile().mkdirs()) {
+        throw new IOException("Unable to rename " + src + " to " + dst
+            + ": couldn't create parent directory");
+      }
+    }
+
+    if (!src.renameTo(dst)) {
+      throw new IOException("Unable to rename " + src + " to " + dst);
+    }
+  }
+
+  public InputStream getSortedStream(int partition) {
+    throw new UnsupportedOperationException("getSortedStream isn't supported!");
+  }
+
+  public ShuffleHeader getShuffleHeader(int reduce) {
+    throw new UnsupportedOperationException("getShuffleHeader isn't supported!");
+  }
+}


[47/50] [abbrv] git commit: Rename tez-engine-api to tez-runtime-api and tez-engine is split into 2: - tez-engine-library for user-visible Input/Output/Processor implementations - tez-engine-internals for framework internals

Posted by ss...@apache.org.
Rename tez-engine-api to tez-runtime-api and tez-engine is split into 2:
  - tez-engine-library for user-visible Input/Output/Processor implementations
  - tez-engine-internals for framework internals


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

Branch: refs/heads/master
Commit: b212ca1d2a087e80329511236022bf297919b960
Parents: 5d86b93
Author: Hitesh Shah <hi...@apache.org>
Authored: Tue Sep 24 15:42:53 2013 -0700
Committer: Hitesh Shah <hi...@apache.org>
Committed: Tue Sep 24 15:42:53 2013 -0700

----------------------------------------------------------------------
 pom.xml                                         |   10 +-
 .../org/apache/tez/common/TezJobConfig.java     |  213 ++--
 .../java/org/apache/tez/engine/api/Event.java   |   28 -
 .../java/org/apache/tez/engine/api/Input.java   |   71 --
 .../tez/engine/api/LogicalIOProcessor.java      |   43 -
 .../org/apache/tez/engine/api/LogicalInput.java |   37 -
 .../apache/tez/engine/api/LogicalOutput.java    |   36 -
 .../java/org/apache/tez/engine/api/Output.java  |   71 --
 .../org/apache/tez/engine/api/Processor.java    |   55 -
 .../java/org/apache/tez/engine/api/Reader.java  |   26 -
 .../apache/tez/engine/api/TezInputContext.java  |   32 -
 .../apache/tez/engine/api/TezOutputContext.java |   33 -
 .../tez/engine/api/TezProcessorContext.java     |   41 -
 .../apache/tez/engine/api/TezTaskContext.java   |  130 --
 .../java/org/apache/tez/engine/api/Writer.java  |   26 -
 .../engine/api/events/DataMovementEvent.java    |  109 --
 .../tez/engine/api/events/InputFailedEvent.java |   89 --
 .../api/events/InputInformationEvent.java       |   41 -
 .../engine/api/events/InputReadErrorEvent.java  |   65 -
 .../common/objectregistry/ObjectLifeCycle.java  |   37 -
 .../common/objectregistry/ObjectRegistry.java   |   56 -
 .../objectregistry/ObjectRegistryFactory.java   |   32 -
 .../java/org/apache/tez/runtime/api/Event.java  |   28 +
 .../java/org/apache/tez/runtime/api/Input.java  |   71 ++
 .../tez/runtime/api/LogicalIOProcessor.java     |   43 +
 .../apache/tez/runtime/api/LogicalInput.java    |   37 +
 .../apache/tez/runtime/api/LogicalOutput.java   |   36 +
 .../java/org/apache/tez/runtime/api/Output.java |   71 ++
 .../org/apache/tez/runtime/api/Processor.java   |   55 +
 .../java/org/apache/tez/runtime/api/Reader.java |   26 +
 .../apache/tez/runtime/api/TezInputContext.java |   32 +
 .../tez/runtime/api/TezOutputContext.java       |   33 +
 .../tez/runtime/api/TezProcessorContext.java    |   41 +
 .../apache/tez/runtime/api/TezTaskContext.java  |  130 ++
 .../java/org/apache/tez/runtime/api/Writer.java |   26 +
 .../runtime/api/events/DataMovementEvent.java   |  109 ++
 .../runtime/api/events/InputFailedEvent.java    |   89 ++
 .../api/events/InputInformationEvent.java       |   41 +
 .../runtime/api/events/InputReadErrorEvent.java |   65 +
 .../common/objectregistry/ObjectLifeCycle.java  |   37 +
 .../common/objectregistry/ObjectRegistry.java   |   56 +
 .../objectregistry/ObjectRegistryFactory.java   |   32 +
 tez-api/src/main/proto/Events.proto             |    2 +-
 tez-dag/pom.xml                                 |    2 +-
 .../apache/hadoop/mapred/YarnTezDagChild.java   |   36 +-
 .../org/apache/tez/dag/app/DAGAppMaster.java    |    2 +-
 .../dag/app/TaskAttemptListenerImpTezDag.java   |    8 +-
 .../org/apache/tez/dag/app/dag/EdgeManager.java |    6 +-
 .../java/org/apache/tez/dag/app/dag/Task.java   |    2 +-
 .../java/org/apache/tez/dag/app/dag/Vertex.java |    6 +-
 .../apache/tez/dag/app/dag/VertexScheduler.java |    2 +-
 .../dag/event/TaskAttemptEventStatusUpdate.java |    2 +-
 .../dag/app/dag/event/TaskEventAddTezEvent.java |    2 +-
 .../app/dag/event/VertexEventRouteEvent.java    |    2 +-
 .../VertexEventSourceTaskAttemptCompleted.java  |    2 +-
 .../event/VertexEventTaskAttemptCompleted.java  |    2 +-
 .../dag/app/dag/impl/BroadcastEdgeManager.java  |    6 +-
 .../apache/tez/dag/app/dag/impl/DAGImpl.java    |    6 +-
 .../org/apache/tez/dag/app/dag/impl/Edge.java   |   16 +-
 .../dag/impl/ImmediateStartVertexScheduler.java |    2 +-
 .../dag/app/dag/impl/OneToOneEdgeManager.java   |    6 +-
 .../app/dag/impl/ScatterGatherEdgeManager.java  |    6 +-
 .../dag/app/dag/impl/ShuffleVertexManager.java  |   12 +-
 .../tez/dag/app/dag/impl/TaskAttemptImpl.java   |    4 +-
 .../apache/tez/dag/app/dag/impl/TaskImpl.java   |    4 +-
 .../apache/tez/dag/app/dag/impl/VertexImpl.java |   18 +-
 .../app/rm/AMSchedulerEventTALaunchRequest.java |    2 +-
 .../rm/container/AMContainerEventAssignTA.java  |    2 +-
 .../app/rm/container/AMContainerHelpers.java    |    8 +-
 .../dag/app/rm/container/AMContainerImpl.java   |    2 +-
 .../dag/app/rm/container/AMContainerTask.java   |    2 +-
 .../apache/tez/dag/utils/TezEngineChildJVM.java |  122 --
 .../tez/dag/utils/TezRuntimeChildJVM.java       |  122 ++
 .../TezDependentTaskCompletionEvent.java        |  228 ----
 ...TezTaskDependencyCompletionEventsUpdate.java |   64 -
 .../TezDependentTaskCompletionEvent.java        |  228 ++++
 ...TezTaskDependencyCompletionEventsUpdate.java |   64 +
 .../tez/dag/app/dag/impl/TestDAGImpl.java       |    2 +-
 .../tez/dag/app/dag/impl/TestTaskAttempt.java   |    2 +-
 .../tez/dag/app/dag/impl/TestVertexImpl.java    |    4 +-
 .../dag/app/dag/impl/TestVertexScheduler.java   |    2 +-
 .../tez/dag/app/rm/TestContainerReuse.java      |    6 +-
 .../dag/app/rm/container/TestAMContainer.java   |    6 +-
 tez-engine/findbugs-exclude.xml                 |   16 -
 tez-engine/pom.xml                              |   92 --
 .../java/org/apache/hadoop/io/BufferUtils.java  |   81 --
 .../org/apache/hadoop/io/HashComparator.java    |   24 -
 .../java/org/apache/tez/common/Constants.java   |   57 -
 .../org/apache/tez/common/ContainerContext.java |   64 -
 .../org/apache/tez/common/ContainerTask.java    |   74 --
 .../tez/common/TezTaskUmbilicalProtocol.java    |   57 -
 .../org/apache/tez/engine/api/KVReader.java     |   81 --
 .../org/apache/tez/engine/api/KVWriter.java     |   40 -
 .../org/apache/tez/engine/api/Partitioner.java  |   52 -
 .../api/events/TaskAttemptCompletedEvent.java   |   28 -
 .../api/events/TaskAttemptFailedEvent.java      |   35 -
 .../api/events/TaskStatusUpdateEvent.java       |   70 --
 .../tez/engine/api/impl/EventMetaData.java      |  152 ---
 .../apache/tez/engine/api/impl/EventType.java   |   29 -
 .../apache/tez/engine/api/impl/InputSpec.java   |   88 --
 .../apache/tez/engine/api/impl/OutputSpec.java  |   87 --
 .../apache/tez/engine/api/impl/TaskSpec.java    |  146 ---
 .../apache/tez/engine/api/impl/TezEvent.java    |  248 ----
 .../engine/api/impl/TezHeartbeatRequest.java    |  137 ---
 .../engine/api/impl/TezHeartbeatResponse.java   |  105 --
 .../engine/api/impl/TezInputContextImpl.java    |   84 --
 .../engine/api/impl/TezOutputContextImpl.java   |   85 --
 .../api/impl/TezProcessorContextImpl.java       |   86 --
 .../tez/engine/api/impl/TezTaskContextImpl.java |  145 ---
 .../tez/engine/api/impl/TezUmbilical.java       |   36 -
 .../broadcast/input/BroadcastInputManager.java  |  138 ---
 .../broadcast/input/BroadcastKVReader.java      |  225 ----
 .../BroadcastShuffleInputEventHandler.java      |   88 --
 .../input/BroadcastShuffleManager.java          |  489 --------
 .../broadcast/output/FileBasedKVWriter.java     |  125 --
 .../apache/tez/engine/common/ConfigUtils.java   |  148 ---
 .../engine/common/InputAttemptIdentifier.java   |   95 --
 .../tez/engine/common/InputIdentifier.java      |   56 -
 .../tez/engine/common/TezEngineUtils.java       |  153 ---
 .../tez/engine/common/ValuesIterator.java       |  194 ---
 .../apache/tez/engine/common/YARNMaster.java    |   57 -
 .../tez/engine/common/combine/Combiner.java     |   42 -
 .../common/localshuffle/LocalShuffle.java       |  120 --
 .../objectregistry/ObjectRegistryImpl.java      |   65 -
 .../objectregistry/ObjectRegistryModule.java    |   43 -
 .../common/security/JobTokenIdentifier.java     |   98 --
 .../common/security/JobTokenSecretManager.java  |  137 ---
 .../common/security/JobTokenSelector.java       |   53 -
 .../tez/engine/common/security/Master.java      |   57 -
 .../common/security/SecureShuffleUtils.java     |  142 ---
 .../tez/engine/common/security/TokenCache.java  |  205 ----
 .../common/shuffle/impl/ExceptionReporter.java  |   25 -
 .../tez/engine/common/shuffle/impl/Fetcher.java |  624 ----------
 .../common/shuffle/impl/InMemoryReader.java     |  156 ---
 .../common/shuffle/impl/InMemoryWriter.java     |  100 --
 .../tez/engine/common/shuffle/impl/MapHost.java |  124 --
 .../engine/common/shuffle/impl/MapOutput.java   |  227 ----
 .../common/shuffle/impl/MergeManager.java       |  782 ------------
 .../engine/common/shuffle/impl/MergeThread.java |  108 --
 .../tez/engine/common/shuffle/impl/Shuffle.java |  278 -----
 .../shuffle/impl/ShuffleClientMetrics.java      |   91 --
 .../common/shuffle/impl/ShuffleHeader.java      |   94 --
 .../shuffle/impl/ShuffleInputEventHandler.java  |  134 ---
 .../common/shuffle/impl/ShuffleScheduler.java   |  521 --------
 .../common/shuffle/server/ShuffleHandler.java   |  572 ---------
 .../engine/common/sort/impl/ExternalSorter.java |  194 ---
 .../tez/engine/common/sort/impl/IFile.java      |  559 ---------
 .../common/sort/impl/IFileInputStream.java      |  276 -----
 .../common/sort/impl/IFileOutputStream.java     |  129 --
 .../common/sort/impl/PipelinedSorter.java       |  932 ---------------
 .../engine/common/sort/impl/TezIndexRecord.java |   45 -
 .../tez/engine/common/sort/impl/TezMerger.java  |  798 -------------
 .../sort/impl/TezRawKeyValueIterator.java       |   70 --
 .../engine/common/sort/impl/TezSpillRecord.java |  146 ---
 .../common/sort/impl/dflt/DefaultSorter.java    | 1108 ------------------
 .../sort/impl/dflt/InMemoryShuffleSorter.java   |  126 --
 .../sort/impl/dflt/SortBufferInputStream.java   |  271 -----
 .../engine/common/task/impl/ValuesIterator.java |  149 ---
 .../local/output/TezLocalTaskOutputFiles.java   |  249 ----
 .../common/task/local/output/TezTaskOutput.java |  165 ---
 .../task/local/output/TezTaskOutputFiles.java   |  246 ----
 .../engine/hadoop/compat/NullProgressable.java  |   33 -
 .../tez/engine/lib/input/LocalMergedInput.java  |   52 -
 .../engine/lib/input/ShuffledMergedInput.java   |  179 ---
 .../lib/input/ShuffledMergedInputLegacy.java    |   30 -
 .../lib/input/ShuffledUnorderedKVInput.java     |   76 --
 .../engine/lib/output/InMemorySortedOutput.java |   81 --
 .../lib/output/LocalOnFileSorterOutput.java     |   63 -
 .../engine/lib/output/OnFileSortedOutput.java   |  123 --
 .../lib/output/OnFileUnorderedKVOutput.java     |   98 --
 .../LogicalIOProcessorRuntimeTask.java          |  475 --------
 .../tez/engine/newruntime/RuntimeTask.java      |  106 --
 .../tez/engine/newruntime/RuntimeUtils.java     |   62 -
 .../engine/shuffle/common/DiskFetchedInput.java |  111 --
 .../tez/engine/shuffle/common/FetchResult.java  |   70 --
 .../tez/engine/shuffle/common/FetchedInput.java |  144 ---
 .../shuffle/common/FetchedInputAllocator.java   |   31 -
 .../shuffle/common/FetchedInputCallback.java    |   29 -
 .../tez/engine/shuffle/common/Fetcher.java      |  608 ----------
 .../engine/shuffle/common/FetcherCallback.java  |   31 -
 .../tez/engine/shuffle/common/InputHost.java    |   90 --
 .../shuffle/common/MemoryFetchedInput.java      |   89 --
 .../tez/engine/shuffle/common/ShuffleUtils.java |   81 --
 tez-engine/src/main/proto/Events.proto          |   28 -
 tez-engine/src/main/proto/ShufflePayloads.proto |   33 -
 .../objectregistry/TestObjectRegistry.java      |   56 -
 .../tez/mapreduce/examples/MRRSleepJob.java     |   10 +-
 .../mapreduce/examples/OrderedWordCount.java    |    4 +-
 .../apache/tez/mapreduce/TestMRRJobsDAGApi.java |    4 +-
 tez-mapreduce/pom.xml                           |    7 +-
 .../apache/hadoop/mapred/LocalJobRunnerTez.java |    2 +-
 .../tez/mapreduce/combine/MRCombiner.java       |   16 +-
 .../tez/mapreduce/hadoop/DeprecatedKeys.java    |   97 +-
 .../apache/tez/mapreduce/hadoop/MRHelpers.java  |    6 +-
 .../hadoop/MultiStageMRConfToTezTranslator.java |   12 +-
 .../hadoop/MultiStageMRConfigUtil.java          |    2 +-
 .../tez/mapreduce/hadoop/mapred/MRReporter.java |    4 +-
 .../hadoop/mapreduce/MapContextImpl.java        |    2 +-
 .../mapreduce/TaskAttemptContextImpl.java       |    2 +-
 .../mapreduce/TaskInputOutputContextImpl.java   |    2 +-
 .../org/apache/tez/mapreduce/input/MRInput.java |    9 +-
 .../apache/tez/mapreduce/output/MROutput.java   |    8 +-
 .../tez/mapreduce/partition/MRPartitioner.java  |    8 +-
 .../apache/tez/mapreduce/processor/MRTask.java  |   17 +-
 .../tez/mapreduce/processor/MRTaskReporter.java |    8 +-
 .../mapreduce/processor/map/MapProcessor.java   |   16 +-
 .../processor/reduce/ReduceProcessor.java       |   22 +-
 .../org/apache/tez/mapreduce/TestUmbilical.java |    6 +-
 .../hadoop/TestConfigTranslationMRToTez.java    |    2 +-
 .../mapreduce/hadoop/TestDeprecatedKeys.java    |   18 +-
 .../tez/mapreduce/processor/MapUtils.java       |   12 +-
 .../processor/map/TestMapProcessor.java         |   24 +-
 .../processor/reduce/TestReduceProcessor.java   |   24 +-
 tez-runtime-internals/findbugs-exclude.xml      |   16 +
 tez-runtime-internals/pom.xml                   |   95 ++
 .../org/apache/tez/common/ContainerContext.java |   64 +
 .../org/apache/tez/common/ContainerTask.java    |   74 ++
 .../tez/common/TezTaskUmbilicalProtocol.java    |   57 +
 .../runtime/LogicalIOProcessorRuntimeTask.java  |  475 ++++++++
 .../org/apache/tez/runtime/RuntimeTask.java     |  106 ++
 .../org/apache/tez/runtime/RuntimeUtils.java    |   62 +
 .../api/events/TaskAttemptCompletedEvent.java   |   28 +
 .../api/events/TaskAttemptFailedEvent.java      |   35 +
 .../api/events/TaskStatusUpdateEvent.java       |   70 ++
 .../tez/runtime/api/impl/EventMetaData.java     |  152 +++
 .../apache/tez/runtime/api/impl/EventType.java  |   29 +
 .../apache/tez/runtime/api/impl/InputSpec.java  |   88 ++
 .../apache/tez/runtime/api/impl/OutputSpec.java |   87 ++
 .../apache/tez/runtime/api/impl/TaskSpec.java   |  146 +++
 .../apache/tez/runtime/api/impl/TezEvent.java   |  248 ++++
 .../runtime/api/impl/TezHeartbeatRequest.java   |  137 +++
 .../runtime/api/impl/TezHeartbeatResponse.java  |  105 ++
 .../runtime/api/impl/TezInputContextImpl.java   |   84 ++
 .../runtime/api/impl/TezOutputContextImpl.java  |   85 ++
 .../api/impl/TezProcessorContextImpl.java       |   86 ++
 .../runtime/api/impl/TezTaskContextImpl.java    |  145 +++
 .../tez/runtime/api/impl/TezUmbilical.java      |   36 +
 .../objectregistry/ObjectRegistryImpl.java      |   68 ++
 .../objectregistry/ObjectRegistryModule.java    |   46 +
 .../src/main/proto/Events.proto                 |   28 +
 .../objectregistry/TestObjectRegistry.java      |   60 +
 tez-runtime-library/pom.xml                     |   79 ++
 .../java/org/apache/hadoop/io/BufferUtils.java  |   81 ++
 .../org/apache/hadoop/io/HashComparator.java    |   24 +
 .../tez/runtime/library/api/KVReader.java       |   81 ++
 .../tez/runtime/library/api/KVWriter.java       |   40 +
 .../tez/runtime/library/api/Partitioner.java    |   52 +
 .../broadcast/input/BroadcastInputManager.java  |  138 +++
 .../broadcast/input/BroadcastKVReader.java      |  225 ++++
 .../BroadcastShuffleInputEventHandler.java      |   88 ++
 .../input/BroadcastShuffleManager.java          |  489 ++++++++
 .../broadcast/output/FileBasedKVWriter.java     |  125 ++
 .../tez/runtime/library/common/ConfigUtils.java |  148 +++
 .../tez/runtime/library/common/Constants.java   |   61 +
 .../library/common/InputAttemptIdentifier.java  |   95 ++
 .../runtime/library/common/InputIdentifier.java |   56 +
 .../runtime/library/common/TezRuntimeUtils.java |  152 +++
 .../runtime/library/common/ValuesIterator.java  |  194 +++
 .../tez/runtime/library/common/YARNMaster.java  |   57 +
 .../library/common/combine/Combiner.java        |   42 +
 .../common/localshuffle/LocalShuffle.java       |  120 ++
 .../common/security/JobTokenIdentifier.java     |   98 ++
 .../common/security/JobTokenSecretManager.java  |  137 +++
 .../common/security/JobTokenSelector.java       |   53 +
 .../runtime/library/common/security/Master.java |   57 +
 .../common/security/SecureShuffleUtils.java     |  142 +++
 .../library/common/security/TokenCache.java     |  205 ++++
 .../common/shuffle/impl/ExceptionReporter.java  |   25 +
 .../library/common/shuffle/impl/Fetcher.java    |  624 ++++++++++
 .../common/shuffle/impl/InMemoryReader.java     |  156 +++
 .../common/shuffle/impl/InMemoryWriter.java     |  100 ++
 .../library/common/shuffle/impl/MapHost.java    |  124 ++
 .../library/common/shuffle/impl/MapOutput.java  |  227 ++++
 .../common/shuffle/impl/MergeManager.java       |  782 ++++++++++++
 .../common/shuffle/impl/MergeThread.java        |  108 ++
 .../library/common/shuffle/impl/Shuffle.java    |  278 +++++
 .../shuffle/impl/ShuffleClientMetrics.java      |   91 ++
 .../common/shuffle/impl/ShuffleHeader.java      |   94 ++
 .../shuffle/impl/ShuffleInputEventHandler.java  |  134 +++
 .../common/shuffle/impl/ShuffleScheduler.java   |  521 ++++++++
 .../common/shuffle/server/ShuffleHandler.java   |  572 +++++++++
 .../common/sort/impl/ExternalSorter.java        |  194 +++
 .../runtime/library/common/sort/impl/IFile.java |  559 +++++++++
 .../common/sort/impl/IFileInputStream.java      |  276 +++++
 .../common/sort/impl/IFileOutputStream.java     |  129 ++
 .../common/sort/impl/PipelinedSorter.java       |  932 +++++++++++++++
 .../common/sort/impl/TezIndexRecord.java        |   45 +
 .../library/common/sort/impl/TezMerger.java     |  798 +++++++++++++
 .../sort/impl/TezRawKeyValueIterator.java       |   70 ++
 .../common/sort/impl/TezSpillRecord.java        |  146 +++
 .../common/sort/impl/dflt/DefaultSorter.java    | 1108 ++++++++++++++++++
 .../sort/impl/dflt/InMemoryShuffleSorter.java   |  126 ++
 .../sort/impl/dflt/SortBufferInputStream.java   |  271 +++++
 .../common/task/impl/ValuesIterator.java        |  149 +++
 .../local/output/TezLocalTaskOutputFiles.java   |  249 ++++
 .../common/task/local/output/TezTaskOutput.java |  165 +++
 .../task/local/output/TezTaskOutputFiles.java   |  246 ++++
 .../library/hadoop/compat/NullProgressable.java |   33 +
 .../runtime/library/input/LocalMergedInput.java |   52 +
 .../library/input/ShuffledMergedInput.java      |  179 +++
 .../input/ShuffledMergedInputLegacy.java        |   30 +
 .../library/input/ShuffledUnorderedKVInput.java |   76 ++
 .../library/output/InMemorySortedOutput.java    |   81 ++
 .../library/output/LocalOnFileSorterOutput.java |   63 +
 .../library/output/OnFileSortedOutput.java      |  123 ++
 .../library/output/OnFileUnorderedKVOutput.java |   98 ++
 .../shuffle/common/DiskFetchedInput.java        |  111 ++
 .../library/shuffle/common/FetchResult.java     |   70 ++
 .../library/shuffle/common/FetchedInput.java    |  144 +++
 .../shuffle/common/FetchedInputAllocator.java   |   31 +
 .../shuffle/common/FetchedInputCallback.java    |   29 +
 .../runtime/library/shuffle/common/Fetcher.java |  608 ++++++++++
 .../library/shuffle/common/FetcherCallback.java |   31 +
 .../library/shuffle/common/InputHost.java       |   90 ++
 .../shuffle/common/MemoryFetchedInput.java      |   89 ++
 .../library/shuffle/common/ShuffleUtils.java    |   81 ++
 .../src/main/proto/ShufflePayloads.proto        |   33 +
 .../org/apache/tez/mapreduce/YARNRunner.java    |    4 +-
 318 files changed, 18962 insertions(+), 18856 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 63f17eb..ce10f6d 100644
--- a/pom.xml
+++ b/pom.xml
@@ -100,7 +100,12 @@
       </dependency>
       <dependency>
         <groupId>org.apache.tez</groupId>
-        <artifactId>tez-engine</artifactId>
+        <artifactId>tez-runtime-internals</artifactId>
+        <version>${project.version}</version>
+      </dependency>
+      <dependency>
+        <groupId>org.apache.tez</groupId>
+        <artifactId>tez-runtime-library</artifactId>
         <version>${project.version}</version>
       </dependency>
       <dependency>
@@ -247,7 +252,8 @@
   <modules>
     <module>tez-api</module>
     <module>tez-common</module>
-    <module>tez-engine</module>
+    <module>tez-runtime-library</module>
+    <module>tez-runtime-internals</module>
     <module>tez-yarn-client</module>
     <module>tez-mapreduce</module>
     <module>tez-mapreduce-examples</module>

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-api/src/main/java/org/apache/tez/common/TezJobConfig.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/common/TezJobConfig.java b/tez-api/src/main/java/org/apache/tez/common/TezJobConfig.java
index 2c4b911..ace87ca 100644
--- a/tez-api/src/main/java/org/apache/tez/common/TezJobConfig.java
+++ b/tez-api/src/main/java/org/apache/tez/common/TezJobConfig.java
@@ -42,16 +42,16 @@ public class TezJobConfig {
   /**
    * Configuration key to enable/disable IFile readahead.
    */
-  public static final String TEZ_ENGINE_IFILE_READAHEAD =
-      "tez.engine.ifile.readahead";
-  public static final boolean DEFAULT_TEZ_ENGINE_IFILE_READAHEAD = true;
+  public static final String TEZ_RUNTIME_IFILE_READAHEAD =
+      "tez.runtime.ifile.readahead";
+  public static final boolean DEFAULT_TEZ_RUNTIME_IFILE_READAHEAD = true;
 
   /**
    * Configuration key to set the IFile readahead length in bytes.
    */
-  public static final String TEZ_ENGINE_IFILE_READAHEAD_BYTES =
-      "tez.engine.ifile.readahead.bytes";
-  public static final int DEFAULT_TEZ_ENGINE_IFILE_READAHEAD_BYTES =
+  public static final String TEZ_RUNTIME_IFILE_READAHEAD_BYTES =
+      "tez.runtime.ifile.readahead.bytes";
+  public static final int DEFAULT_TEZ_RUNTIME_IFILE_READAHEAD_BYTES =
       4 * 1024 * 1024;
 
   /**
@@ -62,102 +62,103 @@ public class TezJobConfig {
   public static final long DEFAULT_RECORDS_BEFORE_PROGRESS = 10000; 
 
   /**
-   * List of directories avialble to the engine. 
+   * List of directories avialble to the Runtime. 
    */
   @Private
-  public static final String LOCAL_DIRS = "tez.engine.local.dirs";
+  public static final String LOCAL_DIRS = "tez.runtime.local.dirs";
   public static final String DEFAULT_LOCAL_DIRS = "/tmp";
 
   /**
    * One local dir for the speicfic job.
    */
-  public static final String JOB_LOCAL_DIR = "tez.engine.job.local.dir";
+  public static final String JOB_LOCAL_DIR = "tez.runtime.job.local.dir";
   
   /**
    * The directory which contains the localized files for this task.
    */
   @Private
-  public static final String TASK_LOCAL_RESOURCE_DIR = "tez.engine.task-local-resource.dir";
+  public static final String TASK_LOCAL_RESOURCE_DIR = "tez.runtime.task-local-resource.dir";
   public static final String DEFAULT_TASK_LOCAL_RESOURCE_DIR = "/tmp";
   
-  public static final String TEZ_TASK_WORKING_DIR = "tez.engine.task.working.dir";
+  public static final String TEZ_TASK_WORKING_DIR = "tez.runtime.task.working.dir";
 
   /**
    * 
    */
-  public static final String TEZ_ENGINE_IO_SORT_FACTOR = 
-      "tez.engine.io.sort.factor";
-  public static final int DEFAULT_TEZ_ENGINE_IO_SORT_FACTOR = 100;
+  public static final String TEZ_RUNTIME_IO_SORT_FACTOR = 
+      "tez.runtime.io.sort.factor";
+  public static final int DEFAULT_TEZ_RUNTIME_IO_SORT_FACTOR = 100;
 
   /**
    * 
    */
-  public static final String TEZ_ENGINE_SORT_SPILL_PERCENT = 
-      "tez.engine.sort.spill.percent";
-  public static float DEFAULT_TEZ_ENGINE_SORT_SPILL_PERCENT = 0.8f; 
+  public static final String TEZ_RUNTIME_SORT_SPILL_PERCENT = 
+      "tez.runtime.sort.spill.percent";
+  public static float DEFAULT_TEZ_RUNTIME_SORT_SPILL_PERCENT = 0.8f; 
 
   /**
    * 
    */
-  public static final String TEZ_ENGINE_IO_SORT_MB = "tez.engine.io.sort.mb";
-  public static final int DEFAULT_TEZ_ENGINE_IO_SORT_MB = 100;
+  public static final String TEZ_RUNTIME_IO_SORT_MB = "tez.runtime.io.sort.mb";
+  public static final int DEFAULT_TEZ_RUNTIME_IO_SORT_MB = 100;
 
   /**
    * 
    */
-  public static final String TEZ_ENGINE_INDEX_CACHE_MEMORY_LIMIT_BYTES = 
-      "tez.engine.index.cache.memory.limit.bytes";
-  public static final int DEFAULT_TEZ_ENGINE_INDEX_CACHE_MEMORY_LIMIT_BYTES = 
+  public static final String TEZ_RUNTIME_INDEX_CACHE_MEMORY_LIMIT_BYTES = 
+      "tez.runtime.index.cache.memory.limit.bytes";
+  public static final int DEFAULT_TEZ_RUNTIME_INDEX_CACHE_MEMORY_LIMIT_BYTES = 
       1024 * 1024;
 
   /**
    * 
    */
-  public static final String TEZ_ENGINE_COMBINE_MIN_SPILLS = 
-      "tez.engine.combine.min.spills";
-  public static final int  DEFAULT_TEZ_ENGINE_COMBINE_MIN_SPILLS = 3;
+  public static final String TEZ_RUNTIME_COMBINE_MIN_SPILLS = 
+      "tez.runtime.combine.min.spills";
+  public static final int  DEFAULT_TEZ_RUNTIME_COMBINE_MIN_SPILLS = 3;
   
   /**
    * 
    */
-  public static final String TEZ_ENGINE_SORT_THREADS = 
-	      "tez.engine.sort.threads";
-  public static final int DEFAULT_TEZ_ENGINE_SORT_THREADS = 1;
+  public static final String TEZ_RUNTIME_SORT_THREADS = 
+	      "tez.runtime.sort.threads";
+  public static final int DEFAULT_TEZ_RUNTIME_SORT_THREADS = 1;
 
   /**
-   * Specifies a partitioner class, which is used in Tez engine components like OnFileSortedOutput
+   * Specifies a partitioner class, which is used in Tez Runtime components
+   * like OnFileSortedOutput
    */
-  public static final String TEZ_ENGINE_PARTITIONER_CLASS = "tez.engine.partitioner.class";
+  public static final String TEZ_RUNTIME_PARTITIONER_CLASS = "tez.runtime.partitioner.class";
   
   /**
    * Specifies a combiner class (primarily for Shuffle)
    */
-  public static final String TEZ_ENGINE_COMBINER_CLASS = "tez.engine.combiner.class";
+  public static final String TEZ_RUNTIME_COMBINER_CLASS = "tez.runtime.combiner.class";
   
-  public static final String TEZ_ENGINE_NUM_EXPECTED_PARTITIONS = "tez.engine.num.expected.partitions";
+  public static final String TEZ_RUNTIME_NUM_EXPECTED_PARTITIONS = "tez.runtime.num.expected.partitions";
   
   /**
    * 
    */
-  public static final String COUNTERS_MAX_KEY = "tez.engine.job.counters.max";
+  public static final String COUNTERS_MAX_KEY = "tez.runtime.job.counters.max";
   public static final int COUNTERS_MAX_DEFAULT = 120;
 
   /**
    * 
    */
-  public static final String COUNTER_GROUP_NAME_MAX_KEY = "tez.engine.job.counters.group.name.max";
+  public static final String COUNTER_GROUP_NAME_MAX_KEY = "tez.runtime.job.counters.group.name.max";
   public static final int COUNTER_GROUP_NAME_MAX_DEFAULT = 128;
 
   /**
    * 
    */
-  public static final String COUNTER_NAME_MAX_KEY = "tez.engine.job.counters.counter.name.max";
+  public static final String COUNTER_NAME_MAX_KEY = "tez.runtime.job.counters.counter.name.max";
   public static final int COUNTER_NAME_MAX_DEFAULT = 64;
 
   /**
    * 
    */
-  public static final String COUNTER_GROUPS_MAX_KEY = "tez.engine.job.counters.groups.max";
+  public static final String COUNTER_GROUPS_MAX_KEY = "tez.runtime.job.counters.groups.max";
   public static final int COUNTER_GROUPS_MAX_DEFAULT = 50;
 
   
@@ -166,145 +167,145 @@ public class TezJobConfig {
    * in-memory shuffle should be used.
    */
   @Private
-  public static final String TEZ_ENGINE_SHUFFLE_USE_IN_MEMORY =
-      "tez.engine.shuffle.use.in-memory";
-  public static final boolean DEFAULT_TEZ_ENGINE_SHUFFLE_USE_IN_MEMORY = false;
+  public static final String TEZ_RUNTIME_SHUFFLE_USE_IN_MEMORY =
+      "tez.runtime.shuffle.use.in-memory";
+  public static final boolean DEFAULT_TEZ_RUNTIME_SHUFFLE_USE_IN_MEMORY = false;
 
   // TODO NEWTEZ Remove these config parameters. Will be part of an event.
   @Private
-  public static final String TEZ_ENGINE_SHUFFLE_PARTITION_RANGE = 
-      "tez.engine.shuffle.partition-range";
-  public static int TEZ_ENGINE_SHUFFLE_PARTITION_RANGE_DEFAULT = 1;
+  public static final String TEZ_RUNTIME_SHUFFLE_PARTITION_RANGE = 
+      "tez.runtime.shuffle.partition-range";
+  public static int TEZ_RUNTIME_SHUFFLE_PARTITION_RANGE_DEFAULT = 1;
 
   /**
    * 
    */
-  public static final String TEZ_ENGINE_SHUFFLE_PARALLEL_COPIES = 
-      "tez.engine.shuffle.parallel.copies";
-  public static final int DEFAULT_TEZ_ENGINE_SHUFFLE_PARALLEL_COPIES = 20;
+  public static final String TEZ_RUNTIME_SHUFFLE_PARALLEL_COPIES = 
+      "tez.runtime.shuffle.parallel.copies";
+  public static final int DEFAULT_TEZ_RUNTIME_SHUFFLE_PARALLEL_COPIES = 20;
 
   /**
    * TODO Is this user configurable.
    */
-  public static final String TEZ_ENGINE_METRICS_SESSION_ID = 
-      "tez.engine.metrics.session.id";
-  public static final String DEFAULT_TEZ_ENGINE_METRICS_SESSION_ID = "";
+  public static final String TEZ_RUNTIME_METRICS_SESSION_ID = 
+      "tez.runtime.metrics.session.id";
+  public static final String DEFAULT_TEZ_RUNTIME_METRICS_SESSION_ID = "";
 
   /**
    * 
    */
-  public static final String TEZ_ENGINE_SHUFFLE_FETCH_FAILURES = 
-      "tez.engine.shuffle.fetch.failures.limit";
-  public final static int DEFAULT_TEZ_ENGINE_SHUFFLE_FETCH_FAILURES_LIMIT = 10;
+  public static final String TEZ_RUNTIME_SHUFFLE_FETCH_FAILURES = 
+      "tez.runtime.shuffle.fetch.failures.limit";
+  public final static int DEFAULT_TEZ_RUNTIME_SHUFFLE_FETCH_FAILURES_LIMIT = 10;
 
   /**
    * 
    */
-  public static final String TEZ_ENGINE_SHUFFLE_NOTIFY_READERROR = 
-      "tez.engine.shuffle.notify.readerror";
-  public static final boolean DEFAULT_TEZ_ENGINE_SHUFFLE_NOTIFY_READERROR = true;
+  public static final String TEZ_RUNTIME_SHUFFLE_NOTIFY_READERROR = 
+      "tez.runtime.shuffle.notify.readerror";
+  public static final boolean DEFAULT_TEZ_RUNTIME_SHUFFLE_NOTIFY_READERROR = true;
 
   /**
    * 
    */
-  public static final String TEZ_ENGINE_SHUFFLE_CONNECT_TIMEOUT = 
-      "tez.engine.shuffle.connect.timeout";
-  public static final int DEFAULT_TEZ_ENGINE_SHUFFLE_STALLED_COPY_TIMEOUT = 
+  public static final String TEZ_RUNTIME_SHUFFLE_CONNECT_TIMEOUT = 
+      "tez.runtime.shuffle.connect.timeout";
+  public static final int DEFAULT_TEZ_RUNTIME_SHUFFLE_STALLED_COPY_TIMEOUT = 
       3 * 60 * 1000;
 
   /**
    * 
    */
-  public static final String TEZ_ENGINE_SHUFFLE_READ_TIMEOUT = "tez.engine.shuffle.read.timeout";
-  public final static int DEFAULT_TEZ_ENGINE_SHUFFLE_READ_TIMEOUT = 
+  public static final String TEZ_RUNTIME_SHUFFLE_READ_TIMEOUT = "tez.runtime.shuffle.read.timeout";
+  public final static int DEFAULT_TEZ_RUNTIME_SHUFFLE_READ_TIMEOUT = 
       3 * 60 * 1000;
 
   /**
    * 
    */
-  public static final String TEZ_ENGINE_SHUFFLE_ENABLE_SSL = 
-      "tez.engine.shuffle.ssl.enable";
-  public static final boolean DEFAULT_TEZ_ENGINE_SHUFFLE_ENABLE_SSL = false;
+  public static final String TEZ_RUNTIME_SHUFFLE_ENABLE_SSL = 
+      "tez.runtime.shuffle.ssl.enable";
+  public static final boolean DEFAULT_TEZ_RUNTIME_SHUFFLE_ENABLE_SSL = false;
 
   /**
    * 
    */
-  public static final String TEZ_ENGINE_SHUFFLE_INPUT_BUFFER_PERCENT = 
-      "tez.engine.shuffle.input.buffer.percent";
-  public static final float DEFAULT_TEZ_ENGINE_SHUFFLE_INPUT_BUFFER_PERCENT =
+  public static final String TEZ_RUNTIME_SHUFFLE_INPUT_BUFFER_PERCENT = 
+      "tez.runtime.shuffle.input.buffer.percent";
+  public static final float DEFAULT_TEZ_RUNTIME_SHUFFLE_INPUT_BUFFER_PERCENT =
       0.90f;
 
   /**
    * 
    */
-  public static final String TEZ_ENGINE_SHUFFLE_MEMORY_LIMIT_PERCENT = 
-      "tez.engine.shuffle.memory.limit.percent";
-  public static final float DEFAULT_TEZ_ENGINE_SHUFFLE_MEMORY_LIMIT_PERCENT = 
+  public static final String TEZ_RUNTIME_SHUFFLE_MEMORY_LIMIT_PERCENT = 
+      "tez.runtime.shuffle.memory.limit.percent";
+  public static final float DEFAULT_TEZ_RUNTIME_SHUFFLE_MEMORY_LIMIT_PERCENT = 
       0.25f;
 
   /**
    * 
    */
-  public static final String TEZ_ENGINE_SHUFFLE_MERGE_PERCENT = 
-      "tez.engine.shuffle.merge.percent";
-  public static final float DEFAULT_TEZ_ENGINE_SHUFFLE_MERGE_PERCENT = 0.90f;
+  public static final String TEZ_RUNTIME_SHUFFLE_MERGE_PERCENT = 
+      "tez.runtime.shuffle.merge.percent";
+  public static final float DEFAULT_TEZ_RUNTIME_SHUFFLE_MERGE_PERCENT = 0.90f;
   
   /**
    * TODO TEZAM3 default value ?
    */
-  public static final String TEZ_ENGINE_SHUFFLE_MEMTOMEM_SEGMENTS = 
-      "tez.engine.shuffle.memory-to-memory.segments";
+  public static final String TEZ_RUNTIME_SHUFFLE_MEMTOMEM_SEGMENTS = 
+      "tez.runtime.shuffle.memory-to-memory.segments";
 
   /**
    * 
    */
-  public static final String TEZ_ENGINE_SHUFFLE_ENABLE_MEMTOMEM = 
-      "tez.engine.shuffle.memory-to-memory.enable";
-  public static final boolean DEFAULT_TEZ_ENGINE_SHUFFLE_ENABLE_MEMTOMEM = 
+  public static final String TEZ_RUNTIME_SHUFFLE_ENABLE_MEMTOMEM = 
+      "tez.runtime.shuffle.memory-to-memory.enable";
+  public static final boolean DEFAULT_TEZ_RUNTIME_SHUFFLE_ENABLE_MEMTOMEM = 
       false;
 
   /**
    * 
    */
-  public static final String TEZ_ENGINE_INPUT_BUFFER_PERCENT = 
-      "tez.engine.task.input.buffer.percent";
-  public static final float DEFAULT_TEZ_ENGINE_INPUT_BUFFER_PERCENT = 0.0f;
+  public static final String TEZ_RUNTIME_INPUT_BUFFER_PERCENT = 
+      "tez.runtime.task.input.buffer.percent";
+  public static final float DEFAULT_TEZ_RUNTIME_INPUT_BUFFER_PERCENT = 0.0f;
 
   // TODO Rename. 
-  public static final String TEZ_ENGINE_GROUP_COMPARATOR_CLASS = 
-      "tez.engine.group.comparator.class";
+  public static final String TEZ_RUNTIME_GROUP_COMPARATOR_CLASS = 
+      "tez.runtime.group.comparator.class";
   
   // TODO Better name.
-  public static final String TEZ_ENGINE_INTERNAL_SORTER_CLASS = 
-      "tez.engine.internal.sorter.class";
+  public static final String TEZ_RUNTIME_INTERNAL_SORTER_CLASS = 
+      "tez.runtime.internal.sorter.class";
   
-  public static final String TEZ_ENGINE_INTERMEDIATE_OUTPUT_KEY_COMPARATOR_CLASS = 
-      "tez.engine.intermediate-output.key.comparator.class";
-  public static final String TEZ_ENGINE_INTERMEDIATE_INPUT_KEY_COMPARATOR_CLASS = 
-      "tez.engine.intermediate-input.key.comparator.class";
-
-  public static final String TEZ_ENGINE_INTERMEDIATE_OUTPUT_KEY_CLASS = 
-      "tez.engine.intermediate-output.key.class";
-  public static final String TEZ_ENGINE_INTERMEDIATE_INPUT_KEY_CLASS = 
-      "tez.engine.intermediate-input.key.class";
+  public static final String TEZ_RUNTIME_INTERMEDIATE_OUTPUT_KEY_COMPARATOR_CLASS = 
+      "tez.runtime.intermediate-output.key.comparator.class";
+  public static final String TEZ_RUNTIME_INTERMEDIATE_INPUT_KEY_COMPARATOR_CLASS = 
+      "tez.runtime.intermediate-input.key.comparator.class";
+
+  public static final String TEZ_RUNTIME_INTERMEDIATE_OUTPUT_KEY_CLASS = 
+      "tez.runtime.intermediate-output.key.class";
+  public static final String TEZ_RUNTIME_INTERMEDIATE_INPUT_KEY_CLASS = 
+      "tez.runtime.intermediate-input.key.class";
   
-  public static final String TEZ_ENGINE_INTERMEDIATE_OUTPUT_VALUE_CLASS = 
-      "tez.engine.intermediate-output.value.class";
-  public static final String TEZ_ENGINE_INTERMEDIATE_INPUT_VALUE_CLASS = 
-      "tez.engine.intermediate-input.value.class";
+  public static final String TEZ_RUNTIME_INTERMEDIATE_OUTPUT_VALUE_CLASS = 
+      "tez.runtime.intermediate-output.value.class";
+  public static final String TEZ_RUNTIME_INTERMEDIATE_INPUT_VALUE_CLASS = 
+      "tez.runtime.intermediate-input.value.class";
   
-  public static final String TEZ_ENGINE_INTERMEDIATE_OUTPUT_SHOULD_COMPRESS = 
-      "tez.engine.intermediate-output.should-compress";
-  public static final String TEZ_ENGINE_INTERMEDIATE_INPUT_IS_COMPRESSED = 
-      "tez.engine.intermdiate-input.is-compressed";
+  public static final String TEZ_RUNTIME_INTERMEDIATE_OUTPUT_SHOULD_COMPRESS = 
+      "tez.runtime.intermediate-output.should-compress";
+  public static final String TEZ_RUNTIME_INTERMEDIATE_INPUT_IS_COMPRESSED = 
+      "tez.runtime.intermdiate-input.is-compressed";
   
-  public static final String TEZ_ENGINE_INTERMEDIATE_OUTPUT_COMPRESS_CODEC = 
-      "tez.engine.intermediate-output.compress.codec";
-  public static final String TEZ_ENGINE_INTERMEDIATE_INPUT_COMPRESS_CODEC = 
-      "tez.engine.intermediate-input.compress.codec";
+  public static final String TEZ_RUNTIME_INTERMEDIATE_OUTPUT_COMPRESS_CODEC = 
+      "tez.runtime.intermediate-output.compress.codec";
+  public static final String TEZ_RUNTIME_INTERMEDIATE_INPUT_COMPRESS_CODEC = 
+      "tez.runtime.intermediate-input.compress.codec";
 
-  public static final String TEZ_ENGINE_INTERMEDIATE_INPUT_KEY_SECONDARY_COMPARATOR_CLASS = 
-      "tez.engine.intermediate-input.key.secondary.comparator.class";
+  public static final String TEZ_RUNTIME_INTERMEDIATE_INPUT_KEY_SECONDARY_COMPARATOR_CLASS = 
+      "tez.runtime.intermediate-input.key.secondary.comparator.class";
   
   // TODO This should be in DAGConfiguration
   /* config for tracking the local file where all the credentials for the job

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-api/src/main/java/org/apache/tez/engine/api/Event.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/engine/api/Event.java b/tez-api/src/main/java/org/apache/tez/engine/api/Event.java
deleted file mode 100644
index 80da655..0000000
--- a/tez-api/src/main/java/org/apache/tez/engine/api/Event.java
+++ /dev/null
@@ -1,28 +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.engine.api;
-
-/**
- * Base class for all events generated within the Tez execution engine.
- * Used as the primary mode of communication between the AM, Inputs, Processors
- * and Outputs.
- */
-public abstract class Event {
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-api/src/main/java/org/apache/tez/engine/api/Input.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/engine/api/Input.java b/tez-api/src/main/java/org/apache/tez/engine/api/Input.java
deleted file mode 100644
index e333075..0000000
--- a/tez-api/src/main/java/org/apache/tez/engine/api/Input.java
+++ /dev/null
@@ -1,71 +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.engine.api;
-
-import java.util.List;
-
-/**
- * Represents an input through which a TezProcessor receives data on an edge.
- * </p>
- *
- * <code>Input</code> classes must have a 0 argument public constructor for Tez
- * to construct the <code>Input</code>. Tez will take care of initializing and
- * closing the Input after a {@link Processor} completes. </p>
- */
-public interface Input {
-
-  /**
-   * Initializes the <code>Input</code>.
-   *
-   * @param inputContext
-   *          the {@link TezInputContext}
-   * @return
-   * @throws Exception
-   *           if an error occurs
-   */
-  public List<Event> initialize(TezInputContext inputContext)
-      throws Exception;
-
-  /**
-   * Gets an instance of the {@link Reader} for this <code>Output</code>
-   *
-   * @return
-   * @throws Exception
-   *           if an error occurs
-   */
-  public Reader getReader() throws Exception;
-
-  /**
-   * Handles user and system generated {@link Events}s, which typically carry
-   * information such as an output being available on the previous vertex.
-   *
-   * @param inputEvents
-   *          the list of {@link Event}s
-   */
-  public void handleEvents(List<Event> inputEvents);
-
-  /**
-   * Closes the <code>Input</code>
-   *
-   * @return
-   * @throws Exception
-   *           if an error occurs
-   */
-  public List<Event> close() throws Exception;
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-api/src/main/java/org/apache/tez/engine/api/LogicalIOProcessor.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/engine/api/LogicalIOProcessor.java b/tez-api/src/main/java/org/apache/tez/engine/api/LogicalIOProcessor.java
deleted file mode 100644
index 90be09e..0000000
--- a/tez-api/src/main/java/org/apache/tez/engine/api/LogicalIOProcessor.java
+++ /dev/null
@@ -1,43 +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.engine.api;
-
-import java.util.Map;
-
-/**
- * Represents a processor which consumes {@link LogicalInput}s and produces
- * {@link LogicalOutput}s
- */
-public interface LogicalIOProcessor extends Processor {
-
-  /**
-   * Runs the {@link LogicalProcessor}
-   * 
-   * @param inputs
-   *          a map of the source vertex name to {@link LogicalInput} - one per
-   *          incoming edge.
-   * @param outputs
-   *          a map of the destination vertex name to {@link LogicalOutput} -
-   *          one per outgoing edge
-   * @throws Exception TODO
-   */
-  public void run(Map<String, LogicalInput> inputs,
-      Map<String, LogicalOutput> outputs) throws Exception;
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-api/src/main/java/org/apache/tez/engine/api/LogicalInput.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/engine/api/LogicalInput.java b/tez-api/src/main/java/org/apache/tez/engine/api/LogicalInput.java
deleted file mode 100644
index 4a47ccf..0000000
--- a/tez-api/src/main/java/org/apache/tez/engine/api/LogicalInput.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tez.engine.api;
-
-/**
- * An @link {@link Input} which handles all incoming physical connections on an
- * edge. A {@link LogicalIOProcessor} sees a single Logical Input per incoming
- * edge.
- */
-public interface LogicalInput extends Input {
-
-  /**
-   * Sets the number of physical inputs that this <code>LogicalInput</code> will
-   * receive. This will be called by the Tez framework before initializing the
-   * <code>LogicalInput</code>
-   * 
-   * @param numInputs
-   *          the number of physical inputs.
-   */
-  public void setNumPhysicalInputs(int numInputs);
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-api/src/main/java/org/apache/tez/engine/api/LogicalOutput.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/engine/api/LogicalOutput.java b/tez-api/src/main/java/org/apache/tez/engine/api/LogicalOutput.java
deleted file mode 100644
index 4626fbd..0000000
--- a/tez-api/src/main/java/org/apache/tez/engine/api/LogicalOutput.java
+++ /dev/null
@@ -1,36 +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.engine.api;
-
-/**
- * An @link {@link Output} which handles all outgoing physical connections on an
- * edge. A {@link LogicalIOProcessor} sees a single Logical Output per outgoing
- * edge.
- */
-public interface LogicalOutput extends Output {
-  /**
-   * Sets the number of physical ouputs that this <code>LogicalOutput</code>
-   * will receive. This will be called by the Tez framework before initializing
-   * the <code>LogicalOutput</code>
-   * 
-   * @param numOutputs
-   *          the number of physical outputs
-   */
-  public void setNumPhysicalOutputs(int numOutputs);
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-api/src/main/java/org/apache/tez/engine/api/Output.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/engine/api/Output.java b/tez-api/src/main/java/org/apache/tez/engine/api/Output.java
deleted file mode 100644
index ec679ed..0000000
--- a/tez-api/src/main/java/org/apache/tez/engine/api/Output.java
+++ /dev/null
@@ -1,71 +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.engine.api;
-
-import java.util.List;
-
-/**
- * Represents an Output through which a TezProcessor writes information on an
- * edge. </p>
- *
- * <code>Output</code> implementations must have a 0 argument public constructor
- * for Tez to construct the <code>Output</code>. Tez will take care of
- * initializing and closing the Input after a {@link Processor} completes. </p>
- */
-public interface Output {
-
-  /**
-   * Initializes the <code>Output</code>
-   *
-   * @param outputContext
-   *          the {@link TezOutputContext}
-   * @return
-   * @throws Exception
-   *           if an error occurs
-   */
-  public List<Event> initialize(TezOutputContext outputContext)
-      throws Exception;
-
-  /**
-   * Gets an instance of the {@link Writer} in an <code>Output</code>
-   *
-   * @return
-   * @throws Exception
-   *           if an error occurs
-   */
-  public Writer getWriter() throws Exception;
-
-  /**
-   * Handles user and system generated {@link Events}s, which typically carry
-   * information such as a downstream vertex being ready to consume input.
-   *
-   * @param outputEvents
-   *          the list of {@link Event}s
-   */
-  public void handleEvents(List<Event> outputEvents);
-
-  /**
-   * Closes the <code>Output</code>
-   *
-   * @return
-   * @throws Exception
-   *           if an error occurs
-   */
-  public List<Event> close() throws Exception;
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-api/src/main/java/org/apache/tez/engine/api/Processor.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/engine/api/Processor.java b/tez-api/src/main/java/org/apache/tez/engine/api/Processor.java
deleted file mode 100644
index 05e6b84..0000000
--- a/tez-api/src/main/java/org/apache/tez/engine/api/Processor.java
+++ /dev/null
@@ -1,55 +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.engine.api;
-
-import java.io.IOException;
-import java.util.List;
-
-/**
- * {@link Processor} represents the <em>Tez</em> entity responsible for
- * consuming {@link Input} and producing {@link Output}.
- */
-public interface Processor {
-
-  /**
-   * Initializes the <code>Processor</code>
-   *
-   * @param processorContext
-   * @throws IOException
-   *           if an error occurs
-   */
-  public void initialize(TezProcessorContext processorContext)
-      throws Exception;
-
-  /**
-   * Handles user and system generated {@link Events}s.
-   *
-   * @param processorEvents
-   *          the list of {@link Event}s
-   */
-  public void handleEvents(List<Event> processorEvents);
-
-  /**
-   * Closes the <code>Processor</code>
-   *
-   * @throws IOException
-   *           if an error occurs
-   */
-  public void close() throws Exception;
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-api/src/main/java/org/apache/tez/engine/api/Reader.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/engine/api/Reader.java b/tez-api/src/main/java/org/apache/tez/engine/api/Reader.java
deleted file mode 100644
index 502c5f2..0000000
--- a/tez-api/src/main/java/org/apache/tez/engine/api/Reader.java
+++ /dev/null
@@ -1,26 +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.engine.api;
-
-/**
- * A <code>Reader</code> represents the data being read in an {@link Input}
- */
-public interface Reader {
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-api/src/main/java/org/apache/tez/engine/api/TezInputContext.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/engine/api/TezInputContext.java b/tez-api/src/main/java/org/apache/tez/engine/api/TezInputContext.java
deleted file mode 100644
index ddf1ff8..0000000
--- a/tez-api/src/main/java/org/apache/tez/engine/api/TezInputContext.java
+++ /dev/null
@@ -1,32 +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.engine.api;
-
-/**
- * Context handle for the Input to initialize itself.
- */
-public interface TezInputContext extends TezTaskContext {
-
-  /**
-   * Get the Vertex Name of the Source that generated data for this Input
-   * @return Name of the Source Vertex
-   */
-  public String getSourceVertexName();
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-api/src/main/java/org/apache/tez/engine/api/TezOutputContext.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/engine/api/TezOutputContext.java b/tez-api/src/main/java/org/apache/tez/engine/api/TezOutputContext.java
deleted file mode 100644
index 791a0f0..0000000
--- a/tez-api/src/main/java/org/apache/tez/engine/api/TezOutputContext.java
+++ /dev/null
@@ -1,33 +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.engine.api;
-
-/**
- * Context handle for the Output to initialize itself.
- */
-public interface TezOutputContext extends TezTaskContext {
-
-  /**
-   * Get the Vertex Name of the Destination that is the recipient of this
-   * Output's data
-   * @return Name of the Destination Vertex
-   */
-  public String getDestinationVertexName();
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-api/src/main/java/org/apache/tez/engine/api/TezProcessorContext.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/engine/api/TezProcessorContext.java b/tez-api/src/main/java/org/apache/tez/engine/api/TezProcessorContext.java
deleted file mode 100644
index 2bbbe81..0000000
--- a/tez-api/src/main/java/org/apache/tez/engine/api/TezProcessorContext.java
+++ /dev/null
@@ -1,41 +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.engine.api;
-
-import java.io.IOException;
-
-/**
- * Context handle for the Processor to initialize itself.
- */
-public interface TezProcessorContext extends TezTaskContext {
-
-  /**
-   * Set the overall progress of this Task Attempt
-   * @param progress Progress in the range from [0.0 - 1.0f]
-   */
-  public void setProgress(float progress);
-
-  /**
-   * Check whether this attempt can commit its output
-   * @return true if commit allowed
-   * @throws IOException
-   */
-  public boolean canCommit() throws IOException;
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-api/src/main/java/org/apache/tez/engine/api/TezTaskContext.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/engine/api/TezTaskContext.java b/tez-api/src/main/java/org/apache/tez/engine/api/TezTaskContext.java
deleted file mode 100644
index 706e646..0000000
--- a/tez-api/src/main/java/org/apache/tez/engine/api/TezTaskContext.java
+++ /dev/null
@@ -1,130 +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.engine.api;
-
-import java.nio.ByteBuffer;
-import java.util.List;
-
-import org.apache.hadoop.yarn.api.records.ApplicationId;
-import org.apache.tez.common.counters.TezCounters;
-
-/**
- * Base interface for Context classes used to initialize the Input, Output
- * and Processor instances.
- */
-public interface TezTaskContext {
-
-  // TODO NEWTEZ
-  // Scale the maximum events we fetch per RPC call to mitigate OOM issues
-  // on the ApplicationMaster when a thundering herd of reducers fetch events
-  // This should not be necessary after HADOOP-8942
-
-  /**
-   * Get the {@link ApplicationId} for the running app
-   * @return the {@link ApplicationId}
-   */
-  public ApplicationId getApplicationId();
-
-  /**
-   * Get the current DAG Attempt Number
-   * @return DAG Attempt Number
-   */
-  public int getDAGAttemptNumber();
-
-  /**
-   * Get the index of this Task
-   * @return Task Index
-   */
-  public int getTaskIndex();
-
-  /**
-   * Get the current Task Attempt Number
-   * @return Task Attempt Number
-   */
-  public int getTaskAttemptNumber();
-
-  /**
-   * Get the name of the DAG
-   * @return the DAG name
-   */
-  public String getDAGName();
-
-  /**
-   * Get the name of the Vertex in which the task is running
-   * @return Vertex Name
-   */
-  public String getTaskVertexName();
-
-  public TezCounters getCounters();
-
-  /**
-   * Send Events to the AM and/or dependent Vertices
-   * @param events Events to be sent
-   */
-  public void sendEvents(List<Event> events);
-
-  /**
-   * Get the User Payload for the Input/Output/Processor
-   * @return User Payload
-   */
-  public byte[] getUserPayload();
-
-  /**
-   * Get the work diectories for the Input/Output/Processor
-   * @return an array of work dirs
-   */
-  public String[] getWorkDirs();
-
-  /**
-   * Returns an identifier which is unique to the specific Input, Processor or
-   * Output
-   *
-   * @return
-   */
-  public String getUniqueIdentifier();
-
-  /**
-   * Report a fatal error to the framework. This will cause the entire task to
-   * fail and should not be used for reporting temporary or recoverable errors
-   *
-   * @param exception an exception representing the error
-   */
-  public void fatalError(Throwable exception, String message);
-
-  /**
-   * Returns meta-data for the specified service. As an example, when the MR
-   * ShuffleHandler is used - this would return the jobToken serialized as bytes
-   *
-   * @param serviceName
-   *          the name of the service for which meta-data is required
-   * @return a ByteBuffer representing the meta-data
-   */
-  public ByteBuffer getServiceConsumerMetaData(String serviceName);
-
-  /**
-   * Return Provider meta-data for the specified service As an example, when the
-   * MR ShuffleHandler is used - this would return the shuffle port serialized
-   * as bytes
-   *
-   * @param serviceName
-   *          the name of the service for which provider meta-data is required
-   * @return a ByteBuffer representing the meta-data
-   */
-  public ByteBuffer getServiceProviderMetaData(String serviceName);
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-api/src/main/java/org/apache/tez/engine/api/Writer.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/engine/api/Writer.java b/tez-api/src/main/java/org/apache/tez/engine/api/Writer.java
deleted file mode 100644
index c9503a3..0000000
--- a/tez-api/src/main/java/org/apache/tez/engine/api/Writer.java
+++ /dev/null
@@ -1,26 +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.engine.api;
-
-/**
- * A <code>Writer</code> represents the data being written by an {@link Output}
- */
-public interface Writer {
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-api/src/main/java/org/apache/tez/engine/api/events/DataMovementEvent.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/engine/api/events/DataMovementEvent.java b/tez-api/src/main/java/org/apache/tez/engine/api/events/DataMovementEvent.java
deleted file mode 100644
index 3f35555..0000000
--- a/tez-api/src/main/java/org/apache/tez/engine/api/events/DataMovementEvent.java
+++ /dev/null
@@ -1,109 +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.engine.api.events;
-
-import org.apache.hadoop.classification.InterfaceAudience.Private;
-import org.apache.tez.engine.api.Event;
-
-/**
- * Event used by user code to send information between tasks. An output can
- * generate an Event of this type to sending information regarding output data
- * ( such as URI for file-based output data, port info in case of
- * streaming-based data transfers ) to the Input on the destination vertex.
- */
-public final class DataMovementEvent extends Event {
-
-  /**
-   * Index(i) of the i-th (physical) Input or Output that generated an Event.
-   * For a Processor-generated event, this is ignored.
-   */
-  private final int sourceIndex;
-
-  /**
-   * Index(i) of the i-th (physical) Input or Output that is meant to receive
-   * this Event. For a Processor event, this is ignored.
-   */
-  private int targetIndex;
-
-  /**
-   * User Payload for this Event
-   */
-  private final byte[] userPayload;
-
-  /**
-   * Version number to indicate what attempt generated this Event
-   */
-  private int version;
-
-  /**
-   * User Event constructor
-   * @param sourceIndex Index to identify the physical edge of the input/output
-   * that generated the event
-   * @param userPayload User Payload of the User Event
-   */
-  public DataMovementEvent(int sourceIndex,
-      byte[] userPayload) {
-    this.userPayload = userPayload;
-    this.sourceIndex = sourceIndex;
-  }
-
-  @Private
-  public DataMovementEvent(int sourceIndex,
-      int targetIndex,
-      byte[] userPayload) {
-    this.userPayload = userPayload;
-    this.sourceIndex = sourceIndex;
-    this.targetIndex = targetIndex;
-  }
-
-  /**
-   * Constructor for Processor-generated User Events
-   * @param userPayload
-   */
-  public DataMovementEvent(byte[] userPayload) {
-    this(-1, userPayload);
-  }
-
-  public byte[] getUserPayload() {
-    return userPayload;
-  }
-
-  public int getSourceIndex() {
-    return sourceIndex;
-  }
-
-  public int getTargetIndex() {
-    return targetIndex;
-  }
-
-  @Private
-  public void setTargetIndex(int targetIndex) {
-    this.targetIndex = targetIndex;
-  }
-
-  public int getVersion() {
-    return version;
-  }
-
-  @Private
-  public void setVersion(int version) {
-    this.version = version;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-api/src/main/java/org/apache/tez/engine/api/events/InputFailedEvent.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/engine/api/events/InputFailedEvent.java b/tez-api/src/main/java/org/apache/tez/engine/api/events/InputFailedEvent.java
deleted file mode 100644
index 57de09b..0000000
--- a/tez-api/src/main/java/org/apache/tez/engine/api/events/InputFailedEvent.java
+++ /dev/null
@@ -1,89 +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.engine.api.events;
-
-import org.apache.hadoop.classification.InterfaceAudience.Private;
-import org.apache.tez.engine.api.Event;
-
-/**
- * Event sent from the AM to an Input to indicate that one of it's sources has
- * failed - effectively the input is no longer available from the particular
- * source.
- * Users are not expected to send this event.
- */
-public class InputFailedEvent extends Event{
-
-  /**
-   * Index(i) of the i-th (physical) Input or Output that generated the data.
-   * For a Processor-generated event, this is ignored.
-   */
-  private final int sourceIndex;
-
-  /**
-   * Index(i) of the i-th (physical) Input or Output that is meant to receive
-   * this Event. For a Processor event, this is ignored.
-   */
-  private int targetIndex;
-
-  /**
-   * Version number to indicate what attempt generated this Event
-   */
-  private int version;
-
-  /**
-   * User Event constructor
-   * @param sourceIndex Index to identify the physical edge of the input/output
-   * that generated the event
-   * @param userPayload User Payload of the User Event
-   */
-  public InputFailedEvent(int sourceIndex) {
-    this.sourceIndex = sourceIndex;
-  }
-
-  @Private
-  public InputFailedEvent(int sourceIndex,
-      int targetIndex,
-      int version) {
-    this.sourceIndex = sourceIndex;
-    this.targetIndex = targetIndex;
-    this.version = version;
-  }
-
-  public int getSourceIndex() {
-    return sourceIndex;
-  }
-
-  public int getTargetIndex() {
-    return targetIndex;
-  }
-
-  @Private
-  public void setTargetIndex(int targetIndex) {
-    this.targetIndex = targetIndex;
-  }
-
-  public int getVersion() {
-    return version;
-  }
-
-  @Private
-  public void setVersion(int version) {
-    this.version = version;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-api/src/main/java/org/apache/tez/engine/api/events/InputInformationEvent.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/engine/api/events/InputInformationEvent.java b/tez-api/src/main/java/org/apache/tez/engine/api/events/InputInformationEvent.java
deleted file mode 100644
index 3656d45..0000000
--- a/tez-api/src/main/java/org/apache/tez/engine/api/events/InputInformationEvent.java
+++ /dev/null
@@ -1,41 +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.engine.api.events;
-
-import org.apache.tez.engine.api.Event;
-
-/**
- * Event used to send user specific data from the user 
- * code in the AM to the task input
- */
-public class InputInformationEvent extends Event {
-
-  /**
-   * User Payload for this Event
-   */
-  private final byte[] userPayload;
-  public InputInformationEvent(byte[] userPayload) {
-    this.userPayload = userPayload;
-  }
-
-  public byte[] getUserPayload() {
-    return userPayload;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-api/src/main/java/org/apache/tez/engine/api/events/InputReadErrorEvent.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/engine/api/events/InputReadErrorEvent.java b/tez-api/src/main/java/org/apache/tez/engine/api/events/InputReadErrorEvent.java
deleted file mode 100644
index fa49b79..0000000
--- a/tez-api/src/main/java/org/apache/tez/engine/api/events/InputReadErrorEvent.java
+++ /dev/null
@@ -1,65 +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.engine.api.events;
-
-import org.apache.tez.engine.api.Event;
-
-/**
- * Event generated by an Input to indicate error when trying to retrieve data.
- * This is not necessarily a fatal event - it's an indication to the AM to retry
- * source data generation.
- */
-public final class InputReadErrorEvent extends Event {
-
-  /**
-   * Diagnostics/trace of the error that occurred on the Input's edge.
-   */
-  private final String diagnostics;
-
-  /**
-   * Index of the physical edge on which the error occurred.
-   */
-  private final int index;
-
-  /**
-   * Version of the data on which the error occurred.
-   */
-  private final int version;
-
-  public InputReadErrorEvent(String diagnostics, int index,
-      int version) {
-    super();
-    this.diagnostics = diagnostics;
-    this.index = index;
-    this.version = version;
-  }
-
-  public String getDiagnostics() {
-    return diagnostics;
-  }
-
-  public int getIndex() {
-    return index;
-  }
-
-  public int getVersion() {
-    return version;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-api/src/main/java/org/apache/tez/engine/common/objectregistry/ObjectLifeCycle.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/engine/common/objectregistry/ObjectLifeCycle.java b/tez-api/src/main/java/org/apache/tez/engine/common/objectregistry/ObjectLifeCycle.java
deleted file mode 100644
index 7099299..0000000
--- a/tez-api/src/main/java/org/apache/tez/engine/common/objectregistry/ObjectLifeCycle.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tez.engine.common.objectregistry;
-
-/**
- * Defines the valid lifecycle and scope of Objects stored in ObjectRegistry.
- * Objects are guaranteed to not be valid outside of their defined life-cycle
- * period. Objects are not guaranteed to be retained through the defined period
- * as they may be evicted for various reasons.
- */
-public enum ObjectLifeCycle {
-  /** Objects are valid for the lifetime of the Tez JVM/Session
-   */
-  SESSION,
-  /** Objects are valid for the lifetime of the DAG.
-   */
-  DAG,
-  /** Objects are valid for the lifetime of the Vertex.
-   */
-  VERTEX,
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-api/src/main/java/org/apache/tez/engine/common/objectregistry/ObjectRegistry.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/engine/common/objectregistry/ObjectRegistry.java b/tez-api/src/main/java/org/apache/tez/engine/common/objectregistry/ObjectRegistry.java
deleted file mode 100644
index a27903d..0000000
--- a/tez-api/src/main/java/org/apache/tez/engine/common/objectregistry/ObjectRegistry.java
+++ /dev/null
@@ -1,56 +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.engine.common.objectregistry;
-
-/**
- * Preliminary version of a simple shared object cache to re-use
- * objects across multiple tasks within the same container/JVM.
- */
-public interface ObjectRegistry {
-
-  /**
-   * Insert or update object into the registry. This will remove an object
-   * associated with the same key with a different life-cycle as there is only
-   * one instance of an Object stored for a given key irrespective of the
-   * life-cycle attached to the Object.
-   * @param lifeCycle What life-cycle is the Object valid for
-   * @param key Key to identify the Object
-   * @param value Object to be inserted
-   * @return Previous Object associated with the key attached if present
-   * else null. Could return the same object if the object was associated with
-   * the same key for a different life-cycle.
-   */
-  public Object add(ObjectLifeCycle lifeCycle, String key, Object value);
-
-  /**
-   * Return the object associated with the provided key
-   * @param key Key to find object
-   * @return Object if found else null
-   */
-  public Object get(String key);
-
-  /**
-   * Delete the object associated with the provided key
-   * @param lifeCycle What life-cycle is the Object valid for
-   * @param key Key to find object
-   * @return True if an object was found and removed
-   */
-  public boolean delete(String key);
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-api/src/main/java/org/apache/tez/engine/common/objectregistry/ObjectRegistryFactory.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/engine/common/objectregistry/ObjectRegistryFactory.java b/tez-api/src/main/java/org/apache/tez/engine/common/objectregistry/ObjectRegistryFactory.java
deleted file mode 100644
index 94352b3..0000000
--- a/tez-api/src/main/java/org/apache/tez/engine/common/objectregistry/ObjectRegistryFactory.java
+++ /dev/null
@@ -1,32 +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.engine.common.objectregistry;
-
-import com.google.inject.Inject;
-
-public class ObjectRegistryFactory {
-
-  @Inject
-  private static ObjectRegistry objectRegistry;
-
-  public static ObjectRegistry getObjectRegistry() {
-    return objectRegistry;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-api/src/main/java/org/apache/tez/runtime/api/Event.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/runtime/api/Event.java b/tez-api/src/main/java/org/apache/tez/runtime/api/Event.java
new file mode 100644
index 0000000..6f3d667
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/runtime/api/Event.java
@@ -0,0 +1,28 @@
+/**
+ * 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.api;
+
+/**
+ * Base class for all events generated within the Tez execution engine.
+ * Used as the primary mode of communication between the AM, Inputs, Processors
+ * and Outputs.
+ */
+public abstract class Event {
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-api/src/main/java/org/apache/tez/runtime/api/Input.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/runtime/api/Input.java b/tez-api/src/main/java/org/apache/tez/runtime/api/Input.java
new file mode 100644
index 0000000..a15c106
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/runtime/api/Input.java
@@ -0,0 +1,71 @@
+/**
+ * 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.api;
+
+import java.util.List;
+
+/**
+ * Represents an input through which a TezProcessor receives data on an edge.
+ * </p>
+ *
+ * <code>Input</code> classes must have a 0 argument public constructor for Tez
+ * to construct the <code>Input</code>. Tez will take care of initializing and
+ * closing the Input after a {@link Processor} completes. </p>
+ */
+public interface Input {
+
+  /**
+   * Initializes the <code>Input</code>.
+   *
+   * @param inputContext
+   *          the {@link TezInputContext}
+   * @return
+   * @throws Exception
+   *           if an error occurs
+   */
+  public List<Event> initialize(TezInputContext inputContext)
+      throws Exception;
+
+  /**
+   * Gets an instance of the {@link Reader} for this <code>Output</code>
+   *
+   * @return
+   * @throws Exception
+   *           if an error occurs
+   */
+  public Reader getReader() throws Exception;
+
+  /**
+   * Handles user and system generated {@link Events}s, which typically carry
+   * information such as an output being available on the previous vertex.
+   *
+   * @param inputEvents
+   *          the list of {@link Event}s
+   */
+  public void handleEvents(List<Event> inputEvents);
+
+  /**
+   * Closes the <code>Input</code>
+   *
+   * @return
+   * @throws Exception
+   *           if an error occurs
+   */
+  public List<Event> close() throws Exception;
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-api/src/main/java/org/apache/tez/runtime/api/LogicalIOProcessor.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/runtime/api/LogicalIOProcessor.java b/tez-api/src/main/java/org/apache/tez/runtime/api/LogicalIOProcessor.java
new file mode 100644
index 0000000..9372c70
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/runtime/api/LogicalIOProcessor.java
@@ -0,0 +1,43 @@
+/**
+ * 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.api;
+
+import java.util.Map;
+
+/**
+ * Represents a processor which consumes {@link LogicalInput}s and produces
+ * {@link LogicalOutput}s
+ */
+public interface LogicalIOProcessor extends Processor {
+
+  /**
+   * Runs the {@link LogicalProcessor}
+   * 
+   * @param inputs
+   *          a map of the source vertex name to {@link LogicalInput} - one per
+   *          incoming edge.
+   * @param outputs
+   *          a map of the destination vertex name to {@link LogicalOutput} -
+   *          one per outgoing edge
+   * @throws Exception TODO
+   */
+  public void run(Map<String, LogicalInput> inputs,
+      Map<String, LogicalOutput> outputs) throws Exception;
+
+}


[45/50] [abbrv] Rename tez-engine-api to tez-runtime-api and tez-engine is split into 2: - tez-engine-library for user-visible Input/Output/Processor implementations - tez-engine-internals for framework internals

Posted by ss...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-dag/src/main/java/org/apache/tez/engine/records/TezDependentTaskCompletionEvent.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/engine/records/TezDependentTaskCompletionEvent.java b/tez-dag/src/main/java/org/apache/tez/engine/records/TezDependentTaskCompletionEvent.java
deleted file mode 100644
index 7a4dd13..0000000
--- a/tez-dag/src/main/java/org/apache/tez/engine/records/TezDependentTaskCompletionEvent.java
+++ /dev/null
@@ -1,228 +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.engine.records;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.io.WritableUtils;
-import org.apache.tez.dag.records.TezTaskAttemptID;
-
-/**
- * This is used to track task completion events on 
- * job tracker. 
- */
-@InterfaceAudience.Public
-@InterfaceStability.Stable
-// TODO TEZAM3 This needs to be more generic. Maybe some kind of a serialized
-// blob - which can be interpretted by the Input plugin.
-public class TezDependentTaskCompletionEvent implements Writable {
-  @InterfaceAudience.Public
-  @InterfaceStability.Evolving
-  // TODO EVENTUALLY - Remove TIPFAILED state ?
-  static public enum Status {FAILED, KILLED, SUCCEEDED, OBSOLETE, TIPFAILED};
-    
-  private int eventId;
-  private int taskRunTime; // using int since runtime is the time difference
-  private TezTaskAttemptID taskAttemptId;
-  private long dataSize;
-  Status status;
-  byte[] userPayload;
-  // TODO TEZAM2 Get rid of the isMap field. Job specific type information can be determined from TaskAttemptId.getTaskType
-//  boolean isMap = false;
-  public static final TezDependentTaskCompletionEvent[] EMPTY_ARRAY = 
-    new TezDependentTaskCompletionEvent[0];
-
-  public TezDependentTaskCompletionEvent() {
-    taskAttemptId = new TezTaskAttemptID();
-  }
-  
-  /**
-   * Constructor. eventId should be created externally and incremented
-   * per event for each job. 
-   * @param eventId event id, event id should be unique and assigned in
-   *  incrementally, starting from 0. 
-   * @param taskAttemptId task id
-   * @param status task's status 
-   * @param taskTrackerHttp task tracker's host:port for http. 
-   */
-  public TezDependentTaskCompletionEvent(int eventId, 
-                             TezTaskAttemptID taskAttemptId,
-//                             boolean isMap,
-                             Status status, 
-                             int runTime,
-                             long dataSize){
-      
-    this.taskAttemptId = taskAttemptId;
-//    this.isMap = isMap;
-    this.eventId = eventId; 
-    this.status =status; 
-    this.taskRunTime = runTime;
-    this.dataSize = dataSize;
-  }
-  
-  public TezDependentTaskCompletionEvent clone() {
-    TezDependentTaskCompletionEvent clone = new TezDependentTaskCompletionEvent(
-        this.eventId, this.taskAttemptId, this.status, 
-        this.taskRunTime, this.dataSize);
-    
-    return clone;
-  }
-  
-  /**
-   * Returns event Id. 
-   * @return event id
-   */
-  public int getEventId() {
-    return eventId;
-  }
-
-  /**
-   * Returns task id. 
-   * @return task id
-   */
-  public TezTaskAttemptID getTaskAttemptID() {
-    return taskAttemptId;
-  }
-  
-  /**
-   * Returns enum Status.SUCESS or Status.FAILURE.
-   * @return task tracker status
-   */
-  public Status getStatus() {
-    return status;
-  }
-  
-  /**
-   * Returns time (in millisec) the task took to complete. 
-   */
-  public int getTaskRunTime() {
-    return taskRunTime;
-  }
-  
-  /**
-   * Return size of output produced by the task
-   */
-  public long getDataSize() {
-    return dataSize;
-  }
-  
-  /**
-   * @return user payload. Maybe null
-   */
-  public byte[] getUserPayload() {
-    return userPayload;
-  }
-
-  /**
-   * Set the task completion time
-   * @param taskCompletionTime time (in millisec) the task took to complete
-   */
-  protected void setTaskRunTime(int taskCompletionTime) {
-    this.taskRunTime = taskCompletionTime;
-  }
-
-  /**
-   * set event Id. should be assigned incrementally starting from 0. 
-   * @param eventId
-   */
-  public void setEventId(int eventId) {
-    this.eventId = eventId;
-  }
-
-  /**
-   * Sets task id. 
-   * @param taskId
-   */
-  public void setTaskAttemptID(TezTaskAttemptID taskId) {
-    this.taskAttemptId = taskId;
-  }
-  
-  /**
-   * Set task status. 
-   * @param status
-   */
-  public void setTaskStatus(Status status) {
-    this.status = status;
-  }
-  
-  /**
-   * Set the user payload
-   * @param userPayload
-   */
-  public void setUserPayload(byte[] userPayload) {
-    this.userPayload = userPayload;
-  }
-    
-  @Override
-  public String toString(){
-    StringBuffer buf = new StringBuffer(); 
-    buf.append("Task Id : "); 
-    buf.append(taskAttemptId); 
-    buf.append(", Status : ");  
-    buf.append(status.name());
-    return buf.toString();
-  }
-    
-  @Override
-  public boolean equals(Object o) {
-    // not counting userPayload as that is a piggyback mechanism
-    if(o == null)
-      return false;
-    if(o.getClass().equals(this.getClass())) {
-      TezDependentTaskCompletionEvent event = (TezDependentTaskCompletionEvent) o;
-      return this.eventId == event.getEventId()
-             && this.status.equals(event.getStatus())
-             && this.taskAttemptId.equals(event.getTaskAttemptID()) 
-             && this.taskRunTime == event.getTaskRunTime()
-             && this.dataSize == event.getDataSize();
-    }
-    return false;
-  }
-
-  @Override
-  public int hashCode() {
-    return toString().hashCode(); 
-  }
-
-  @Override
-  public void write(DataOutput out) throws IOException {
-    taskAttemptId.write(out);
-//    out.writeBoolean(isMap);
-    WritableUtils.writeEnum(out, status);
-    WritableUtils.writeVInt(out, taskRunTime);
-    WritableUtils.writeVInt(out, eventId);
-    WritableUtils.writeCompressedByteArray(out, userPayload);
-  }
-
-  @Override
-  public void readFields(DataInput in) throws IOException {
-    taskAttemptId.readFields(in);
-//    isMap = in.readBoolean();
-    status = WritableUtils.readEnum(in, Status.class);
-    taskRunTime = WritableUtils.readVInt(in);
-    eventId = WritableUtils.readVInt(in);
-    userPayload = WritableUtils.readCompressedByteArray(in);
-  }
-  
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-dag/src/main/java/org/apache/tez/engine/records/TezTaskDependencyCompletionEventsUpdate.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/engine/records/TezTaskDependencyCompletionEventsUpdate.java b/tez-dag/src/main/java/org/apache/tez/engine/records/TezTaskDependencyCompletionEventsUpdate.java
deleted file mode 100644
index 13c9088..0000000
--- a/tez-dag/src/main/java/org/apache/tez/engine/records/TezTaskDependencyCompletionEventsUpdate.java
+++ /dev/null
@@ -1,64 +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.engine.records;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-
-import org.apache.hadoop.io.Writable;
-
-
-public class TezTaskDependencyCompletionEventsUpdate implements Writable {
-  TezDependentTaskCompletionEvent[] events;
-  boolean reset;
-
-  public TezTaskDependencyCompletionEventsUpdate() { }
-
-  public TezTaskDependencyCompletionEventsUpdate(
-      TezDependentTaskCompletionEvent[] events, boolean reset) {
-    this.events = events;
-    this.reset = reset;
-  }
-
-  public boolean shouldReset() {
-    return reset;
-  }
-
-  public TezDependentTaskCompletionEvent[] getDependentTaskCompletionEvents() {
-    return events;
-  }
-  
-  public void write(DataOutput out) throws IOException {
-    out.writeBoolean(reset);
-    out.writeInt(events.length);
-    for (TezDependentTaskCompletionEvent event : events) {
-      event.write(out);
-    }
-  }
-
-  public void readFields(DataInput in) throws IOException {
-    reset = in.readBoolean();
-    events = new TezDependentTaskCompletionEvent[in.readInt()];
-    for (int i = 0; i < events.length; ++i) {
-      events[i] = new TezDependentTaskCompletionEvent();
-      events[i].readFields(in);
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-dag/src/main/java/org/apache/tez/runtime/records/TezDependentTaskCompletionEvent.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/runtime/records/TezDependentTaskCompletionEvent.java b/tez-dag/src/main/java/org/apache/tez/runtime/records/TezDependentTaskCompletionEvent.java
new file mode 100644
index 0000000..fd4c1ee
--- /dev/null
+++ b/tez-dag/src/main/java/org/apache/tez/runtime/records/TezDependentTaskCompletionEvent.java
@@ -0,0 +1,228 @@
+/**
+ * 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.records;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableUtils;
+import org.apache.tez.dag.records.TezTaskAttemptID;
+
+/**
+ * This is used to track task completion events on 
+ * job tracker. 
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+// TODO TEZAM3 This needs to be more generic. Maybe some kind of a serialized
+// blob - which can be interpretted by the Input plugin.
+public class TezDependentTaskCompletionEvent implements Writable {
+  @InterfaceAudience.Public
+  @InterfaceStability.Evolving
+  // TODO EVENTUALLY - Remove TIPFAILED state ?
+  static public enum Status {FAILED, KILLED, SUCCEEDED, OBSOLETE, TIPFAILED};
+    
+  private int eventId;
+  private int taskRunTime; // using int since runtime is the time difference
+  private TezTaskAttemptID taskAttemptId;
+  private long dataSize;
+  Status status;
+  byte[] userPayload;
+  // TODO TEZAM2 Get rid of the isMap field. Job specific type information can be determined from TaskAttemptId.getTaskType
+//  boolean isMap = false;
+  public static final TezDependentTaskCompletionEvent[] EMPTY_ARRAY = 
+    new TezDependentTaskCompletionEvent[0];
+
+  public TezDependentTaskCompletionEvent() {
+    taskAttemptId = new TezTaskAttemptID();
+  }
+  
+  /**
+   * Constructor. eventId should be created externally and incremented
+   * per event for each job. 
+   * @param eventId event id, event id should be unique and assigned in
+   *  incrementally, starting from 0. 
+   * @param taskAttemptId task id
+   * @param status task's status 
+   * @param taskTrackerHttp task tracker's host:port for http. 
+   */
+  public TezDependentTaskCompletionEvent(int eventId, 
+                             TezTaskAttemptID taskAttemptId,
+//                             boolean isMap,
+                             Status status, 
+                             int runTime,
+                             long dataSize){
+      
+    this.taskAttemptId = taskAttemptId;
+//    this.isMap = isMap;
+    this.eventId = eventId; 
+    this.status =status; 
+    this.taskRunTime = runTime;
+    this.dataSize = dataSize;
+  }
+  
+  public TezDependentTaskCompletionEvent clone() {
+    TezDependentTaskCompletionEvent clone = new TezDependentTaskCompletionEvent(
+        this.eventId, this.taskAttemptId, this.status, 
+        this.taskRunTime, this.dataSize);
+    
+    return clone;
+  }
+  
+  /**
+   * Returns event Id. 
+   * @return event id
+   */
+  public int getEventId() {
+    return eventId;
+  }
+
+  /**
+   * Returns task id. 
+   * @return task id
+   */
+  public TezTaskAttemptID getTaskAttemptID() {
+    return taskAttemptId;
+  }
+  
+  /**
+   * Returns enum Status.SUCESS or Status.FAILURE.
+   * @return task tracker status
+   */
+  public Status getStatus() {
+    return status;
+  }
+  
+  /**
+   * Returns time (in millisec) the task took to complete. 
+   */
+  public int getTaskRunTime() {
+    return taskRunTime;
+  }
+  
+  /**
+   * Return size of output produced by the task
+   */
+  public long getDataSize() {
+    return dataSize;
+  }
+  
+  /**
+   * @return user payload. Maybe null
+   */
+  public byte[] getUserPayload() {
+    return userPayload;
+  }
+
+  /**
+   * Set the task completion time
+   * @param taskCompletionTime time (in millisec) the task took to complete
+   */
+  protected void setTaskRunTime(int taskCompletionTime) {
+    this.taskRunTime = taskCompletionTime;
+  }
+
+  /**
+   * set event Id. should be assigned incrementally starting from 0. 
+   * @param eventId
+   */
+  public void setEventId(int eventId) {
+    this.eventId = eventId;
+  }
+
+  /**
+   * Sets task id. 
+   * @param taskId
+   */
+  public void setTaskAttemptID(TezTaskAttemptID taskId) {
+    this.taskAttemptId = taskId;
+  }
+  
+  /**
+   * Set task status. 
+   * @param status
+   */
+  public void setTaskStatus(Status status) {
+    this.status = status;
+  }
+  
+  /**
+   * Set the user payload
+   * @param userPayload
+   */
+  public void setUserPayload(byte[] userPayload) {
+    this.userPayload = userPayload;
+  }
+    
+  @Override
+  public String toString(){
+    StringBuffer buf = new StringBuffer(); 
+    buf.append("Task Id : "); 
+    buf.append(taskAttemptId); 
+    buf.append(", Status : ");  
+    buf.append(status.name());
+    return buf.toString();
+  }
+    
+  @Override
+  public boolean equals(Object o) {
+    // not counting userPayload as that is a piggyback mechanism
+    if(o == null)
+      return false;
+    if(o.getClass().equals(this.getClass())) {
+      TezDependentTaskCompletionEvent event = (TezDependentTaskCompletionEvent) o;
+      return this.eventId == event.getEventId()
+             && this.status.equals(event.getStatus())
+             && this.taskAttemptId.equals(event.getTaskAttemptID()) 
+             && this.taskRunTime == event.getTaskRunTime()
+             && this.dataSize == event.getDataSize();
+    }
+    return false;
+  }
+
+  @Override
+  public int hashCode() {
+    return toString().hashCode(); 
+  }
+
+  @Override
+  public void write(DataOutput out) throws IOException {
+    taskAttemptId.write(out);
+//    out.writeBoolean(isMap);
+    WritableUtils.writeEnum(out, status);
+    WritableUtils.writeVInt(out, taskRunTime);
+    WritableUtils.writeVInt(out, eventId);
+    WritableUtils.writeCompressedByteArray(out, userPayload);
+  }
+
+  @Override
+  public void readFields(DataInput in) throws IOException {
+    taskAttemptId.readFields(in);
+//    isMap = in.readBoolean();
+    status = WritableUtils.readEnum(in, Status.class);
+    taskRunTime = WritableUtils.readVInt(in);
+    eventId = WritableUtils.readVInt(in);
+    userPayload = WritableUtils.readCompressedByteArray(in);
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-dag/src/main/java/org/apache/tez/runtime/records/TezTaskDependencyCompletionEventsUpdate.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/runtime/records/TezTaskDependencyCompletionEventsUpdate.java b/tez-dag/src/main/java/org/apache/tez/runtime/records/TezTaskDependencyCompletionEventsUpdate.java
new file mode 100644
index 0000000..ff4f267
--- /dev/null
+++ b/tez-dag/src/main/java/org/apache/tez/runtime/records/TezTaskDependencyCompletionEventsUpdate.java
@@ -0,0 +1,64 @@
+/**
+ * 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.records;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hadoop.io.Writable;
+
+
+public class TezTaskDependencyCompletionEventsUpdate implements Writable {
+  TezDependentTaskCompletionEvent[] events;
+  boolean reset;
+
+  public TezTaskDependencyCompletionEventsUpdate() { }
+
+  public TezTaskDependencyCompletionEventsUpdate(
+      TezDependentTaskCompletionEvent[] events, boolean reset) {
+    this.events = events;
+    this.reset = reset;
+  }
+
+  public boolean shouldReset() {
+    return reset;
+  }
+
+  public TezDependentTaskCompletionEvent[] getDependentTaskCompletionEvents() {
+    return events;
+  }
+  
+  public void write(DataOutput out) throws IOException {
+    out.writeBoolean(reset);
+    out.writeInt(events.length);
+    for (TezDependentTaskCompletionEvent event : events) {
+      event.write(out);
+    }
+  }
+
+  public void readFields(DataInput in) throws IOException {
+    reset = in.readBoolean();
+    events = new TezDependentTaskCompletionEvent[in.readInt()];
+    for (int i = 0; i < events.length; ++i) {
+      events[i] = new TezDependentTaskCompletionEvent();
+      events[i].readFields(in);
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/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 6e42673..31513c5 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
@@ -70,7 +70,7 @@ import org.apache.tez.dag.history.avro.HistoryEventType;
 import org.apache.tez.dag.records.TezDAGID;
 import org.apache.tez.dag.records.TezTaskID;
 import org.apache.tez.dag.records.TezVertexID;
-import org.apache.tez.engine.common.security.JobTokenSecretManager;
+import org.apache.tez.runtime.library.common.security.JobTokenSecretManager;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/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 676e747..f2717be 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
@@ -82,7 +82,7 @@ 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.engine.api.impl.TaskSpec;
+import org.apache.tez.runtime.api.impl.TaskSpec;
 import org.junit.Test;
 import org.mockito.ArgumentCaptor;
 

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/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 3d00cb7..b524f6a 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
@@ -82,8 +82,8 @@ 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.engine.records.TezDependentTaskCompletionEvent;
-import org.apache.tez.engine.records.TezDependentTaskCompletionEvent.Status;
+import org.apache.tez.runtime.records.TezDependentTaskCompletionEvent;
+import org.apache.tez.runtime.records.TezDependentTaskCompletionEvent.Status;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexScheduler.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexScheduler.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexScheduler.java
index 1a07b5b..81715bd 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexScheduler.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexScheduler.java
@@ -40,8 +40,8 @@ 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.engine.records.TezDependentTaskCompletionEvent;
 import org.apache.tez.mapreduce.hadoop.MRHelpers;
+import org.apache.tez.runtime.records.TezDependentTaskCompletionEvent;
 import org.junit.Assert;
 import org.junit.Test;
 import org.mockito.invocation.InvocationOnMock;

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestContainerReuse.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestContainerReuse.java b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestContainerReuse.java
index 3a6e008..68ee532 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestContainerReuse.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestContainerReuse.java
@@ -56,9 +56,9 @@ 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.engine.api.impl.InputSpec;
-import org.apache.tez.engine.api.impl.OutputSpec;
-import org.apache.tez.engine.api.impl.TaskSpec;
+import org.apache.tez.runtime.api.impl.InputSpec;
+import org.apache.tez.runtime.api.impl.OutputSpec;
+import org.apache.tez.runtime.api.impl.TaskSpec;
 import org.junit.Test;
 
 import com.google.common.collect.Lists;

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-dag/src/test/java/org/apache/tez/dag/app/rm/container/TestAMContainer.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/container/TestAMContainer.java b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/container/TestAMContainer.java
index c2457e1..fc89e82 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/container/TestAMContainer.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/container/TestAMContainer.java
@@ -65,9 +65,9 @@ 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.engine.api.impl.TaskSpec;
-import org.apache.tez.engine.common.security.JobTokenIdentifier;
-import org.apache.tez.engine.common.security.TokenCache;
+import org.apache.tez.runtime.api.impl.TaskSpec;
+import org.apache.tez.runtime.library.common.security.JobTokenIdentifier;
+import org.apache.tez.runtime.library.common.security.TokenCache;
 import org.junit.Test;
 import org.mockito.ArgumentCaptor;
 

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/findbugs-exclude.xml
----------------------------------------------------------------------
diff --git a/tez-engine/findbugs-exclude.xml b/tez-engine/findbugs-exclude.xml
deleted file mode 100644
index 5b11308..0000000
--- a/tez-engine/findbugs-exclude.xml
+++ /dev/null
@@ -1,16 +0,0 @@
-<!--
-  Licensed 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. See accompanying LICENSE file.
--->
-<FindBugsFilter>
-
-</FindBugsFilter>

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/pom.xml
----------------------------------------------------------------------
diff --git a/tez-engine/pom.xml b/tez-engine/pom.xml
deleted file mode 100644
index 498f2f2..0000000
--- a/tez-engine/pom.xml
+++ /dev/null
@@ -1,92 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-  Licensed 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. See accompanying LICENSE file.
--->
-
-<project xmlns="http://maven.apache.org/POM/4.0.0"
-         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
-  <modelVersion>4.0.0</modelVersion>
-  <parent>
-    <groupId>org.apache.tez</groupId>
-    <artifactId>tez</artifactId>
-    <version>0.2.0-SNAPSHOT</version>
-  </parent>
-  <artifactId>tez-engine</artifactId>
-
-  <dependencies>
-    <dependency>
-      <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-yarn-api</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.tez</groupId>
-      <artifactId>tez-common</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.tez</groupId>
-      <artifactId>tez-api</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-yarn-common</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>com.google.inject</groupId>
-      <artifactId>guice</artifactId>
-    </dependency>
-    <dependency>
-     <groupId>com.google.protobuf</groupId>
-     <artifactId>protobuf-java</artifactId>
-    </dependency>
-  </dependencies>
-
-  <build>
-    <plugins>
-      <plugin>
-        <groupId>org.apache.rat</groupId>
-        <artifactId>apache-rat-plugin</artifactId>
-        <configuration>
-        </configuration>
-      </plugin>
-      <plugin>
-        <groupId>org.apache.hadoop</groupId>
-        <artifactId>hadoop-maven-plugins</artifactId>
-        <executions>
-          <execution>
-            <id>compile-protoc</id>
-            <phase>generate-sources</phase>
-            <goals>
-              <goal>protoc</goal>
-            </goals>
-            <configuration>
-              <protocVersion>${protobuf.version}</protocVersion>
-              <protocCommand>${protoc.path}</protocCommand>
-              <imports>
-                <param>${basedir}/src/main/proto</param>
-              </imports>
-              <source>
-                <directory>${basedir}/src/main/proto</directory>
-                <includes>
-                  <include>Events.proto</include>
-                  <include>ShufflePayloads.proto</include>
-                </includes>
-              </source>
-              <output>${project.build.directory}/generated-sources/java</output>
-            </configuration>
-          </execution>
-        </executions>
-      </plugin>
-    </plugins>
-  </build>
-</project>

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/hadoop/io/BufferUtils.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/hadoop/io/BufferUtils.java b/tez-engine/src/main/java/org/apache/hadoop/io/BufferUtils.java
deleted file mode 100644
index 16f7a8f..0000000
--- a/tez-engine/src/main/java/org/apache/hadoop/io/BufferUtils.java
+++ /dev/null
@@ -1,81 +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.hadoop.io;
-
-import java.io.IOException;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-
-@InterfaceAudience.LimitedPrivate({"HDFS", "MapReduce"})
-@InterfaceStability.Unstable
-public class BufferUtils {
-  public static int compare(DataInputBuffer buf1, DataInputBuffer buf2) {
-    byte[] b1 = buf1.getData();
-    byte[] b2 = buf2.getData();
-    int s1 = buf1.getPosition();
-    int s2 = buf2.getPosition();
-    int l1 = buf1.getLength();
-    int l2 = buf2.getLength();
-    return FastByteComparisons.compareTo(b1, s1, l1, b2, s2, l2);
-  }
-
-  public static int compare(DataOutputBuffer buf1, DataOutputBuffer buf2) {
-    byte[] b1 = buf1.getData();
-    byte[] b2 = buf2.getData();
-    int s1 = 0;
-    int s2 = 0;
-    int l1 = buf1.getLength();
-    int l2 = buf2.getLength();
-    return FastByteComparisons.compareTo(b1, s1, l1, b2, s2, l2);
-  }
-
-  public static int compare(DataInputBuffer buf1, DataOutputBuffer buf2) {
-    byte[] b1 = buf1.getData();
-    byte[] b2 = buf2.getData();
-    int s1 = buf1.getPosition();    
-    int s2 = 0;
-    int l1 = buf1.getLength();
-    int l2 = buf2.getLength();
-    return FastByteComparisons.compareTo(b1, s1, l1, b2, s2, l2);
-  }
-
-  public static int compare(DataOutputBuffer buf1, DataInputBuffer buf2) {
-    return compare(buf2, buf1);
-  }
-
-  public static void copy(DataInputBuffer src, DataOutputBuffer dst) 
-                              throws IOException {
-    byte[] b1 = src.getData();
-    int s1 = src.getPosition();    
-    int l1 = src.getLength();
-    dst.reset();
-    dst.write(b1, s1, l1 - s1);
-  }
-
-  public static void copy(DataOutputBuffer src, DataOutputBuffer dst) 
-                              throws IOException {
-    byte[] b1 = src.getData();
-    int s1 = 0;
-    int l1 = src.getLength();
-    dst.reset();
-    dst.write(b1, s1, l1);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/hadoop/io/HashComparator.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/hadoop/io/HashComparator.java b/tez-engine/src/main/java/org/apache/hadoop/io/HashComparator.java
deleted file mode 100644
index a372e01..0000000
--- a/tez-engine/src/main/java/org/apache/hadoop/io/HashComparator.java
+++ /dev/null
@@ -1,24 +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.hadoop.io;
-
-public interface HashComparator<KEY> {
-
-  int getHashCode(KEY key);
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/common/Constants.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/common/Constants.java b/tez-engine/src/main/java/org/apache/tez/common/Constants.java
deleted file mode 100644
index 8ea2909..0000000
--- a/tez-engine/src/main/java/org/apache/tez/common/Constants.java
+++ /dev/null
@@ -1,57 +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.common;
-
-import org.apache.hadoop.classification.InterfaceAudience.Private;
-
-
-public class Constants {
-
-  // TODO NEWTEZ Check which of these constants are expecting specific pieces of information which are being removed - like taskAttemptId
-  
-  public static final String TEZ = "tez";
-
-  public static final String MAP_OUTPUT_FILENAME_STRING = "file.out";
-  public static final String MAP_OUTPUT_INDEX_SUFFIX_STRING = ".index";
-  public static final String REDUCE_INPUT_FILE_FORMAT_STRING = "%s/map_%d.out";
-
-  public static final int MAP_OUTPUT_INDEX_RECORD_LENGTH = 24;
-  public static String MERGED_OUTPUT_PREFIX = ".merged";
-  
-  // TODO NEWTEZ Remove this constant once the old code is removed.
-  public static final String TEZ_ENGINE_TASK_ATTEMPT_ID = 
-      "tez.engine.task.attempt.id";
-
-  public static final String TEZ_ENGINE_TASK_OUTPUT_FILENAME_STRING = "file.out";
-
-  public static final String TEZ_ENGINE_TASK_OUTPUT_INDEX_SUFFIX_STRING = ".index";
-
-  public static final String TEZ_ENGINE_TASK_INPUT_FILE_FORMAT_STRING = "%s/task_%d.out"; 
-
-  public static final String TEZ_ENGINE_JOB_CREDENTIALS = 
-      "tez.engine.job.credentials";
-  
-  @Private
-  public static final String TEZ_ENGINE_TASK_MEMORY =  "tez.engine.task.memory";
-  
-  public static final String TASK_OUTPUT_DIR = "output";
-  
-  public static final String TEZ_ENGINE_TASK_OUTPUT_MANAGER = 
-      "tez.engine.task.local.output.manager";
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/common/ContainerContext.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/common/ContainerContext.java b/tez-engine/src/main/java/org/apache/tez/common/ContainerContext.java
deleted file mode 100644
index df92bdc..0000000
--- a/tez-engine/src/main/java/org/apache/tez/common/ContainerContext.java
+++ /dev/null
@@ -1,64 +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.common;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.Writable;
-
-// TODO EVENTUALLY move this over to PB. Fix package/module.
-// TODO EVENTUALLY unit tests for functionality.
-public class ContainerContext implements Writable {
-
-  String containerIdentifier;
-  String pid;
-
-  public ContainerContext() {
-    containerIdentifier = "";
-    pid = "";
-  }
-
-  public ContainerContext(String containerIdStr, String pid) {
-    this.containerIdentifier = containerIdStr;
-    this.pid = pid;
-  }
-
-  public String getContainerIdentifier() {
-    return containerIdentifier;
-  }
-
-  public String getPid() {
-    return pid;
-  }
-
-  @Override
-  public void readFields(DataInput in) throws IOException {
-    this.containerIdentifier = Text.readString(in);
-    this.pid = Text.readString(in);
-  }
-
-  @Override
-  public void write(DataOutput out) throws IOException {
-    Text.writeString(out, containerIdentifier);
-    Text.writeString(out, pid);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/common/ContainerTask.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/common/ContainerTask.java b/tez-engine/src/main/java/org/apache/tez/common/ContainerTask.java
deleted file mode 100644
index e90f7fa..0000000
--- a/tez-engine/src/main/java/org/apache/tez/common/ContainerTask.java
+++ /dev/null
@@ -1,74 +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.common;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-
-import org.apache.hadoop.io.Writable;
-import org.apache.tez.engine.api.impl.TaskSpec;
-
-public class ContainerTask implements Writable {
-
-  TaskSpec taskSpec;
-  boolean shouldDie;
-
-  public ContainerTask() {
-  }
-
-  public ContainerTask(TaskSpec taskSpec, boolean shouldDie) {
-    this.taskSpec = taskSpec;
-    this.shouldDie = shouldDie;
-  }
-
-  public TaskSpec getTaskSpec() {
-    return taskSpec;
-  }
-
-  public boolean shouldDie() {
-    return shouldDie;
-  }
-
-  @Override
-  public void write(DataOutput out) throws IOException {
-    out.writeBoolean(shouldDie);
-    if (taskSpec != null) {
-      out.writeBoolean(true);
-      taskSpec.write(out);
-    } else {
-      out.writeBoolean(false);
-    }
-  }
-
-  @Override
-  public void readFields(DataInput in) throws IOException {
-    shouldDie = in.readBoolean();
-    boolean taskComing = in.readBoolean();
-    if (taskComing) {
-      taskSpec = new TaskSpec();
-      taskSpec.readFields(in);
-    }
-  }
-
-  @Override
-  public String toString() {
-    return "shouldDie: " + shouldDie + ", TaskSpec: "
-        + taskSpec;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/common/TezTaskUmbilicalProtocol.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/common/TezTaskUmbilicalProtocol.java b/tez-engine/src/main/java/org/apache/tez/common/TezTaskUmbilicalProtocol.java
deleted file mode 100644
index 9e4129f..0000000
--- a/tez-engine/src/main/java/org/apache/tez/common/TezTaskUmbilicalProtocol.java
+++ /dev/null
@@ -1,57 +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.common;
-
-import java.io.IOException;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.ipc.ProtocolInfo;
-import org.apache.hadoop.ipc.VersionedProtocol;
-import org.apache.tez.common.records.ProceedToCompletionResponse;
-import org.apache.tez.dag.api.TezException;
-import org.apache.tez.dag.records.TezTaskAttemptID;
-import org.apache.tez.engine.api.impl.TezHeartbeatRequest;
-import org.apache.tez.engine.api.impl.TezHeartbeatResponse;
-
-/** Protocol that task child process uses to contact its parent process.  The
- * parent is a daemon which which polls the central master for a new map or
- * reduce task and runs it as a child process.  All communication between child
- * and parent is via this protocol. */
-@InterfaceAudience.Private
-@InterfaceStability.Stable
-@ProtocolInfo(protocolName = "TezTaskUmbilicalProtocol", protocolVersion = 1)
-public interface TezTaskUmbilicalProtocol extends VersionedProtocol {
-
-  public static final long versionID = 19L;
-
-  ContainerTask getTask(ContainerContext containerContext) throws IOException;
-
-  boolean canCommit(TezTaskAttemptID taskid) throws IOException;
-
-  ProceedToCompletionResponse
-      proceedToCompletion(TezTaskAttemptID taskAttemptId) throws IOException;
-
-  /// Copies from TezUmbilical until complete re-factor is done
-  // TODONEWTEZ
-
-  public TezHeartbeatResponse heartbeat(TezHeartbeatRequest request)
-      throws IOException, TezException;
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/engine/api/KVReader.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/api/KVReader.java b/tez-engine/src/main/java/org/apache/tez/engine/api/KVReader.java
deleted file mode 100644
index 0178b3a..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/api/KVReader.java
+++ /dev/null
@@ -1,81 +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.engine.api;
-
-import java.io.IOException;
-
-import org.apache.tez.engine.api.Reader;
-
-/**
- * A key/value(s) pair based {@link Reader}.
- * 
- * Example usage
- * <code>
- * while (kvReader.next()) {
- *   KVRecord kvRecord = getCurrentKV();
- *   Object key =  kvRecord.getKey();
- *   Iterable values = kvRecord.getValues();
- * </code>
- *
- */
-public interface KVReader extends Reader {
-
-  /**
-   * Moves to the next key/values(s) pair
-   * 
-   * @return true if another key/value(s) pair exists, false if there are no more.
-   * @throws IOException
-   *           if an error occurs
-   */
-  public boolean next() throws IOException;
-
-  /**
-   * Return the current key/value(s) pair. Use moveToNext() to advance.
-   * @return
-   * @throws IOException
-   */
-  public KVRecord getCurrentKV() throws IOException;
-  
-  // TODO NEWTEZ Move this to getCurrentKey and getCurrentValue independently. Otherwise usage pattern seems to be getCurrentKV.getKey, getCurrentKV.getValue
-  
-  // TODO NEWTEZ KVRecord which does not need to return a list!
-  // TODO NEWTEZ Parameterize this
-  /**
-   * Represents a key and an associated set of values
-   *
-   */
-  public static class KVRecord {
-
-    private Object key;
-    private Iterable<Object> values;
-
-    public KVRecord(Object key, Iterable<Object> values) {
-      this.key = key;
-      this.values = values;
-    }
-
-    public Object getKey() {
-      return this.key;
-    }
-
-    public Iterable<Object> getValues() {
-      return this.values;
-    }
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/engine/api/KVWriter.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/api/KVWriter.java b/tez-engine/src/main/java/org/apache/tez/engine/api/KVWriter.java
deleted file mode 100644
index 970831b..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/api/KVWriter.java
+++ /dev/null
@@ -1,40 +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.engine.api;
-
-import java.io.IOException;
-
-import org.apache.tez.engine.api.Writer;
-
-/**
- * A key/value(s) pair based {@link Writer}
- */
-public interface KVWriter extends Writer {
-  /**
-   * Writes a key/value pair.
-   * 
-   * @param key
-   *          the key to write
-   * @param value
-   *          the value to write
-   * @throws IOException
-   *           if an error occurs
-   */
-  public void write(Object key, Object value) throws IOException;
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/engine/api/Partitioner.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/api/Partitioner.java b/tez-engine/src/main/java/org/apache/tez/engine/api/Partitioner.java
deleted file mode 100644
index ccf3cb8..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/api/Partitioner.java
+++ /dev/null
@@ -1,52 +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.engine.api;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.tez.common.TezJobConfig;
-
-/**
- * {@link Partitioner} is used by the TEZ framework to partition output
- * key/value pairs.
- * 
- * <b>Partitioner Initialization</b></p> The Partitioner class is picked up
- * using the TEZ_ENGINE_PARTITIONER_CLASS attribute in {@link TezJobConfig}
- * 
- * TODO NEWTEZ Change construction to first check for a Constructor with a bytep[] payload
- * 
- * Partitioners need to provide a single argument ({@link Configuration})
- * constructor or a 0 argument constructor. If both exist, preference is given
- * to the single argument constructor. This is primarily for MR support.
- * 
- * If using the configuration constructor, TEZ_ENGINE_NUM_EXPECTED_PARTITIONS
- * will be set in the configuration, to indicate the max number of expected
- * partitions.
- * 
- */
-public interface Partitioner {
-  
-  /**
-   * Get partition for given key/value.
-   * @param key key
-   * @param value value
-   * @param numPartitions number of partitions
-   * @return
-   */
-  int getPartition(Object key, Object value, int numPartitions);
-  
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/engine/api/events/TaskAttemptCompletedEvent.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/api/events/TaskAttemptCompletedEvent.java b/tez-engine/src/main/java/org/apache/tez/engine/api/events/TaskAttemptCompletedEvent.java
deleted file mode 100644
index a9f2c98..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/api/events/TaskAttemptCompletedEvent.java
+++ /dev/null
@@ -1,28 +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.engine.api.events;
-
-import org.apache.tez.engine.api.Event;
-
-public class TaskAttemptCompletedEvent extends Event {
-
-  public TaskAttemptCompletedEvent() {
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/engine/api/events/TaskAttemptFailedEvent.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/api/events/TaskAttemptFailedEvent.java b/tez-engine/src/main/java/org/apache/tez/engine/api/events/TaskAttemptFailedEvent.java
deleted file mode 100644
index fc67472..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/api/events/TaskAttemptFailedEvent.java
+++ /dev/null
@@ -1,35 +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.engine.api.events;
-
-import org.apache.tez.engine.api.Event;
-
-public class TaskAttemptFailedEvent extends Event {
-
-  private final String diagnostics;
-
-  public TaskAttemptFailedEvent(String diagnostics) {
-    this.diagnostics = diagnostics;
-  }
-
-  public String getDiagnostics() {
-    return diagnostics;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/engine/api/events/TaskStatusUpdateEvent.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/api/events/TaskStatusUpdateEvent.java b/tez-engine/src/main/java/org/apache/tez/engine/api/events/TaskStatusUpdateEvent.java
deleted file mode 100644
index c0d1ee6..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/api/events/TaskStatusUpdateEvent.java
+++ /dev/null
@@ -1,70 +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.engine.api.events;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-
-import org.apache.hadoop.io.Writable;
-import org.apache.tez.common.counters.TezCounters;
-import org.apache.tez.engine.api.Event;
-
-public class TaskStatusUpdateEvent extends Event implements Writable {
-
-  private TezCounters tezCounters;
-  private float progress;
-
-  public TaskStatusUpdateEvent() {
-  }
-
-  public TaskStatusUpdateEvent(TezCounters tezCounters, float progress) {
-    this.tezCounters = tezCounters;
-    this.progress = progress;
-  }
-
-  public TezCounters getCounters() {
-    return tezCounters;
-  }
-
-  public float getProgress() {
-    return progress;
-  }
-
-  @Override
-  public void write(DataOutput out) throws IOException {
-    out.writeFloat(progress);
-    if (tezCounters != null) {
-      out.writeBoolean(true);
-      tezCounters.write(out);
-    } else {
-      out.writeBoolean(false);
-    }
-  }
-
-  @Override
-  public void readFields(DataInput in) throws IOException {
-    progress = in.readFloat();
-    if (in.readBoolean()) {
-      tezCounters = new TezCounters();
-      tezCounters.readFields(in);
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/engine/api/impl/EventMetaData.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/api/impl/EventMetaData.java b/tez-engine/src/main/java/org/apache/tez/engine/api/impl/EventMetaData.java
deleted file mode 100644
index 64df7bb..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/api/impl/EventMetaData.java
+++ /dev/null
@@ -1,152 +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.engine.api.impl;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-
-import org.apache.hadoop.io.Writable;
-import org.apache.tez.dag.records.TezTaskAttemptID;
-
-/**
- * Class that encapsulates all the information to identify the unique
- * object that either generated an Event or is the recipient of an Event.
- */
-public class EventMetaData implements Writable {
-
-  public static enum EventProducerConsumerType {
-    INPUT,
-    PROCESSOR,
-    OUTPUT,
-    SYSTEM
-  }
-
-  /**
-   * Producer Type ( one of Input/Output/Processor ) that generated the Event
-   * or Consumer Type that will consume the Event.
-   */
-  private EventProducerConsumerType producerConsumerType;
-
-  /**
-   * Name of the vertex where the event was generated.
-   */
-  private String taskVertexName;
-
-  /**
-   * Name of the vertex to which the Input or Output is connected to.
-   */
-  private String edgeVertexName;
-
-  /**
-   * i'th physical input/output that this event maps to.
-   */
-  private int index;
-
-  /**
-   * Task Attempt ID
-   */
-  private TezTaskAttemptID taskAttemptID;
-
-  public EventMetaData() {
-  }
-
-  public EventMetaData(EventProducerConsumerType generator,
-      String taskVertexName, String edgeVertexName,
-      TezTaskAttemptID taskAttemptID) {
-    this.producerConsumerType = generator;
-    this.taskVertexName = taskVertexName;
-    this.edgeVertexName = edgeVertexName;
-    this.taskAttemptID = taskAttemptID;
-  }
-
-  public EventProducerConsumerType getEventGenerator() {
-    return producerConsumerType;
-  }
-
-  public TezTaskAttemptID getTaskAttemptID() {
-    return taskAttemptID;
-  }
-
-  public String getTaskVertexName() {
-    return taskVertexName;
-  }
-
-  public String getEdgeVertexName() {
-    return edgeVertexName;
-  }
-
-  @Override
-  public void write(DataOutput out) throws IOException {
-    out.writeInt(producerConsumerType.ordinal());
-    if (taskVertexName != null) {
-      out.writeBoolean(true);
-      out.writeUTF(taskVertexName);
-    } else {
-      out.writeBoolean(false);
-    }
-    if (edgeVertexName != null) {
-      out.writeBoolean(true);
-      out.writeUTF(edgeVertexName);
-    } else {
-      out.writeBoolean(false);
-    }
-    if(taskAttemptID != null) {
-      out.writeBoolean(true);
-      taskAttemptID.write(out);
-    } else {
-      out.writeBoolean(false);
-    }
-    
-    out.writeInt(index);
-  }
-
-  @Override
-  public void readFields(DataInput in) throws IOException {
-    producerConsumerType = EventProducerConsumerType.values()[in.readInt()];
-    if (in.readBoolean()) {
-      taskVertexName = in.readUTF();
-    }
-    if (in.readBoolean()) {
-      edgeVertexName = in.readUTF();
-    }
-    if (in.readBoolean()) {
-      taskAttemptID = new TezTaskAttemptID();
-      taskAttemptID.readFields(in);
-    }
-    index = in.readInt();
-  }
-
-  public int getIndex() {
-    return index;
-  }
-
-  public void setIndex(int index) {
-    this.index = index;
-  }
-
-  @Override
-  public String toString() {
-    return "{ producerConsumerType=" + producerConsumerType
-        + ", taskVertexName=" + taskVertexName
-        + ", edgeVertexName=" + edgeVertexName
-        + ", taskAttemptId=" + taskAttemptID
-        + ", index=" + index + " }";
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/engine/api/impl/EventType.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/api/impl/EventType.java b/tez-engine/src/main/java/org/apache/tez/engine/api/impl/EventType.java
deleted file mode 100644
index 52fc10d..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/api/impl/EventType.java
+++ /dev/null
@@ -1,29 +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.engine.api.impl;
-
-public enum EventType {
-  TASK_ATTEMPT_COMPLETED_EVENT,
-  TASK_ATTEMPT_FAILED_EVENT,
-  DATA_MOVEMENT_EVENT,
-  INPUT_READ_ERROR_EVENT,
-  INPUT_FAILED_EVENT,
-  INTPUT_INFORMATION_EVENT,
-  TASK_STATUS_UPDATE_EVENT
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/engine/api/impl/InputSpec.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/api/impl/InputSpec.java b/tez-engine/src/main/java/org/apache/tez/engine/api/impl/InputSpec.java
deleted file mode 100644
index a9ef333..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/api/impl/InputSpec.java
+++ /dev/null
@@ -1,88 +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.engine.api.impl;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-
-import org.apache.hadoop.io.Writable;
-import org.apache.tez.dag.api.DagTypeConverters;
-import org.apache.tez.dag.api.InputDescriptor;
-import org.apache.tez.dag.api.records.DAGProtos.TezEntityDescriptorProto;
-
-public class InputSpec implements Writable {
-
-  private String sourceVertexName;
-  private InputDescriptor inputDescriptor;
-  private int physicalEdgeCount;
-
-  public InputSpec() {
-  }
-
-  public InputSpec(String sourceVertexName, InputDescriptor inputDescriptor,
-      int physicalEdgeCount) {
-    this.sourceVertexName = sourceVertexName;
-    this.inputDescriptor = inputDescriptor;
-    this.physicalEdgeCount = physicalEdgeCount;
-  }
-
-  public String getSourceVertexName() {
-    return sourceVertexName;
-  }
-
-  public InputDescriptor getInputDescriptor() {
-    return inputDescriptor;
-  }
-
-  public int getPhysicalEdgeCount() {
-    return physicalEdgeCount;
-  }
-
-  @Override
-  public void write(DataOutput out) throws IOException {
-    // TODONEWTEZ convert to PB
-    out.writeUTF(sourceVertexName);
-    out.writeInt(physicalEdgeCount);
-    byte[] inputDescBytes =
-        DagTypeConverters.convertToDAGPlan(inputDescriptor).toByteArray();
-    out.writeInt(inputDescBytes.length);
-    out.write(inputDescBytes);
-  }
-
-  @Override
-  public void readFields(DataInput in) throws IOException {
-    sourceVertexName = in.readUTF();
-    physicalEdgeCount = in.readInt();
-    int inputDescLen = in.readInt();
-    byte[] inputDescBytes = new byte[inputDescLen];
-    in.readFully(inputDescBytes);
-    inputDescriptor =
-        DagTypeConverters.convertInputDescriptorFromDAGPlan(
-            TezEntityDescriptorProto.parseFrom(inputDescBytes));
-  }
-
-  public String toString() {
-    return "{ sourceVertexName=" + sourceVertexName
-        + ", physicalEdgeCount" + physicalEdgeCount
-        + ", inputClassName=" + inputDescriptor.getClassName()
-        + " }";
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/engine/api/impl/OutputSpec.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/api/impl/OutputSpec.java b/tez-engine/src/main/java/org/apache/tez/engine/api/impl/OutputSpec.java
deleted file mode 100644
index 3a1d5d8..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/api/impl/OutputSpec.java
+++ /dev/null
@@ -1,87 +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.engine.api.impl;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-
-import org.apache.hadoop.io.Writable;
-import org.apache.tez.dag.api.DagTypeConverters;
-import org.apache.tez.dag.api.OutputDescriptor;
-import org.apache.tez.dag.api.records.DAGProtos.TezEntityDescriptorProto;
-
-public class OutputSpec implements Writable {
-
-  private String destinationVertexName;
-  private OutputDescriptor outputDescriptor;
-  private int physicalEdgeCount;
-
-  public OutputSpec() {
-  }
-
-  public OutputSpec(String destinationVertexName,
-      OutputDescriptor outputDescriptor, int physicalEdgeCount) {
-    this.destinationVertexName = destinationVertexName;
-    this.outputDescriptor = outputDescriptor;
-    this.physicalEdgeCount = physicalEdgeCount;
-  }
-
-  public String getDestinationVertexName() {
-    return destinationVertexName;
-  }
-
-  public OutputDescriptor getOutputDescriptor() {
-    return outputDescriptor;
-  }
-
-  public int getPhysicalEdgeCount() {
-    return physicalEdgeCount;
-  }
-
-  @Override
-  public void write(DataOutput out) throws IOException {
-    // TODONEWTEZ convert to PB
-    out.writeUTF(destinationVertexName);
-    out.writeInt(physicalEdgeCount);
-    byte[] inputDescBytes =
-        DagTypeConverters.convertToDAGPlan(outputDescriptor).toByteArray();
-    out.writeInt(inputDescBytes.length);
-    out.write(inputDescBytes);
-  }
-
-  @Override
-  public void readFields(DataInput in) throws IOException {
-    destinationVertexName = in.readUTF();
-    physicalEdgeCount = in.readInt();
-    int inputDescLen = in.readInt();
-    byte[] inputDescBytes = new byte[inputDescLen];
-    in.readFully(inputDescBytes);
-    outputDescriptor =
-        DagTypeConverters.convertOutputDescriptorFromDAGPlan(
-            TezEntityDescriptorProto.parseFrom(inputDescBytes));
-  }
-
-  public String toString() {
-    return "{ destinationVertexName=" + destinationVertexName
-        + ", physicalEdgeCount" + physicalEdgeCount
-        + ", outputClassName=" + outputDescriptor.getClassName()
-        + " }";
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TaskSpec.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TaskSpec.java b/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TaskSpec.java
deleted file mode 100644
index 6527777..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TaskSpec.java
+++ /dev/null
@@ -1,146 +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.engine.api.impl;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.hadoop.io.Writable;
-import org.apache.tez.dag.api.DagTypeConverters;
-import org.apache.tez.dag.api.ProcessorDescriptor;
-import org.apache.tez.dag.api.records.DAGProtos.TezEntityDescriptorProto;
-import org.apache.tez.dag.records.TezTaskAttemptID;
-
-public class TaskSpec implements Writable {
-
-  private TezTaskAttemptID taskAttemptId;
-  private String vertexName;
-  private String user;
-  private ProcessorDescriptor processorDescriptor;
-  private List<InputSpec> inputSpecList;
-  private List<OutputSpec> outputSpecList;
-
-  public TaskSpec() {
-  }
-
-  // TODO NEWTEZ Remove user
-  public TaskSpec(TezTaskAttemptID taskAttemptID, String user,
-      String vertexName, ProcessorDescriptor processorDescriptor,
-      List<InputSpec> inputSpecList, List<OutputSpec> outputSpecList) {
-    this.taskAttemptId = taskAttemptID;
-    this.vertexName = vertexName;
-    this.user = user;
-    this.processorDescriptor = processorDescriptor;
-    this.inputSpecList = inputSpecList;
-    this.outputSpecList = outputSpecList;
-  }
-
-  public String getVertexName() {
-    return vertexName;
-  }
-
-  public TezTaskAttemptID getTaskAttemptID() {
-    return taskAttemptId;
-  }
-
-  public String getUser() {
-    return user;
-  }
-
-  public ProcessorDescriptor getProcessorDescriptor() {
-    return processorDescriptor;
-  }
-
-  public List<InputSpec> getInputs() {
-    return inputSpecList;
-  }
-
-  public List<OutputSpec> getOutputs() {
-    return outputSpecList;
-  }
-
-  @Override
-  public void write(DataOutput out) throws IOException {
-    taskAttemptId.write(out);
-    out.writeUTF(vertexName);
-    byte[] procDesc =
-        DagTypeConverters.convertToDAGPlan(processorDescriptor).toByteArray();
-    out.writeInt(procDesc.length);
-    out.write(procDesc);
-    out.writeInt(inputSpecList.size());
-    for (InputSpec inputSpec : inputSpecList) {
-      inputSpec.write(out);
-    }
-    out.writeInt(outputSpecList.size());
-    for (OutputSpec outputSpec : outputSpecList) {
-      outputSpec.write(out);
-    }
-  }
-
-  @Override
-  public void readFields(DataInput in) throws IOException {
-    taskAttemptId = new TezTaskAttemptID();
-    taskAttemptId.readFields(in);
-    vertexName = in.readUTF();
-    int procDescLength = in.readInt();
-    // TODO at least 3 buffer copies here. Need to convert this to full PB
-    // TEZ-305
-    byte[] procDescBytes = new byte[procDescLength];
-    in.readFully(procDescBytes);
-    processorDescriptor =
-        DagTypeConverters.convertProcessorDescriptorFromDAGPlan(
-            TezEntityDescriptorProto.parseFrom(procDescBytes));
-    int numInputSpecs = in.readInt();
-    inputSpecList = new ArrayList<InputSpec>(numInputSpecs);
-    for (int i = 0; i < numInputSpecs; i++) {
-      InputSpec inputSpec = new InputSpec();
-      inputSpec.readFields(in);
-      inputSpecList.add(inputSpec);
-    }
-    int numOutputSpecs = in.readInt();
-    outputSpecList = new ArrayList<OutputSpec>(numOutputSpecs);
-    for (int i = 0; i < numOutputSpecs; i++) {
-      OutputSpec outputSpec = new OutputSpec();
-      outputSpec.readFields(in);
-      outputSpecList.add(outputSpec);
-    }
-  }
-
-  @Override
-  public String toString() {
-    StringBuffer sb = new StringBuffer();
-    sb.append("TaskAttemptID:" + taskAttemptId);
-    sb.append("processorName=" + processorDescriptor.getClassName()
-        + ", inputSpecListSize=" + inputSpecList.size()
-        + ", outputSpecListSize=" + outputSpecList.size());
-    sb.append(", inputSpecList=[");
-    for (InputSpec i : inputSpecList) {
-      sb.append("{" + i.toString() + "}, ");
-    }
-    sb.append("], outputSpecList=[");
-    for (OutputSpec i : outputSpecList) {
-      sb.append("{" + i.toString() + "}, ");
-    }
-    sb.append("]");
-    return sb.toString();
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezEvent.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezEvent.java b/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezEvent.java
deleted file mode 100644
index 6841d72..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezEvent.java
+++ /dev/null
@@ -1,248 +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.engine.api.impl;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-
-import org.apache.hadoop.io.Writable;
-import org.apache.tez.dag.api.TezUncheckedException;
-import org.apache.tez.engine.api.Event;
-import org.apache.tez.engine.api.events.DataMovementEvent;
-import org.apache.tez.engine.api.events.InputFailedEvent;
-import org.apache.tez.engine.api.events.InputInformationEvent;
-import org.apache.tez.engine.api.events.InputReadErrorEvent;
-import org.apache.tez.engine.api.events.TaskAttemptCompletedEvent;
-import org.apache.tez.engine.api.events.TaskAttemptFailedEvent;
-import org.apache.tez.engine.api.events.TaskStatusUpdateEvent;
-import org.apache.tez.engine.api.events.EventProtos.DataMovementEventProto;
-import org.apache.tez.engine.api.events.EventProtos.InputFailedEventProto;
-import org.apache.tez.engine.api.events.EventProtos.InputInformationEventProto;
-import org.apache.tez.engine.api.events.EventProtos.InputReadErrorEventProto;
-import org.apache.tez.engine.api.events.SystemEventProtos.TaskAttemptCompletedEventProto;
-import org.apache.tez.engine.api.events.SystemEventProtos.TaskAttemptFailedEventProto;
-
-import com.google.protobuf.ByteString;
-
-public class TezEvent implements Writable {
-
-  private EventType eventType;
-
-  private Event event;
-
-  private EventMetaData sourceInfo;
-
-  private EventMetaData destinationInfo;
-
-  public TezEvent() {
-  }
-
-  public TezEvent(Event event, EventMetaData sourceInfo) {
-    this.event = event;
-    this.setSourceInfo(sourceInfo);
-    if (event instanceof DataMovementEvent) {
-      eventType = EventType.DATA_MOVEMENT_EVENT;
-    } else if (event instanceof InputReadErrorEvent) {
-      eventType = EventType.INPUT_READ_ERROR_EVENT;
-    } else if (event instanceof TaskAttemptFailedEvent) {
-      eventType = EventType.TASK_ATTEMPT_FAILED_EVENT;
-    } else if (event instanceof TaskAttemptCompletedEvent) {
-      eventType = EventType.TASK_ATTEMPT_COMPLETED_EVENT;
-    } else if (event instanceof InputInformationEvent) {
-      eventType = EventType.INTPUT_INFORMATION_EVENT;
-    } else if (event instanceof InputFailedEvent) {
-      eventType = EventType.INPUT_FAILED_EVENT;
-    } else if (event instanceof TaskStatusUpdateEvent) {
-      eventType = EventType.TASK_STATUS_UPDATE_EVENT;
-    } else {
-      throw new TezUncheckedException("Unknown event, event="
-          + event.getClass().getName());
-    }
-  }
-
-  public Event getEvent() {
-    return event;
-  }
-
-  public EventMetaData getSourceInfo() {
-    return sourceInfo;
-  }
-
-  public void setSourceInfo(EventMetaData sourceInfo) {
-    this.sourceInfo = sourceInfo;
-  }
-
-  public EventMetaData getDestinationInfo() {
-    return destinationInfo;
-  }
-
-  public void setDestinationInfo(EventMetaData destinationInfo) {
-    this.destinationInfo = destinationInfo;
-  }
-
-  public EventType getEventType() {
-    return eventType;
-  }
-
-  private void serializeEvent(DataOutput out) throws IOException {
-    if (event == null) {
-      out.writeBoolean(false);
-      return;
-    }
-    out.writeBoolean(true);
-    out.writeInt(eventType.ordinal());
-    if (eventType.equals(EventType.TASK_STATUS_UPDATE_EVENT)) {
-      // TODO NEWTEZ convert to PB
-      TaskStatusUpdateEvent sEvt = (TaskStatusUpdateEvent) event;
-      sEvt.write(out);
-    } else {
-      byte[] eventBytes = null;
-      switch (eventType) {
-      case DATA_MOVEMENT_EVENT:
-        DataMovementEvent dmEvt = (DataMovementEvent) event;
-        eventBytes = DataMovementEventProto.newBuilder()
-          .setSourceIndex(dmEvt.getSourceIndex())
-          .setTargetIndex(dmEvt.getTargetIndex())
-          .setUserPayload(ByteString.copyFrom(dmEvt.getUserPayload()))
-          .build().toByteArray();
-        break;
-      case INPUT_READ_ERROR_EVENT:
-        InputReadErrorEvent ideEvt = (InputReadErrorEvent) event;
-        eventBytes = InputReadErrorEventProto.newBuilder()
-            .setIndex(ideEvt.getIndex())
-            .setDiagnostics(ideEvt.getDiagnostics())
-            .build().toByteArray();
-        break;
-      case TASK_ATTEMPT_FAILED_EVENT:
-        TaskAttemptFailedEvent tfEvt = (TaskAttemptFailedEvent) event;
-        eventBytes = TaskAttemptFailedEventProto.newBuilder()
-            .setDiagnostics(tfEvt.getDiagnostics())
-            .build().toByteArray();
-        break;
-      case TASK_ATTEMPT_COMPLETED_EVENT:
-        eventBytes = TaskAttemptCompletedEventProto.newBuilder()
-            .build().toByteArray();
-        break;
-      case INPUT_FAILED_EVENT:
-        InputFailedEvent ifEvt = (InputFailedEvent) event;
-        eventBytes = InputFailedEventProto.newBuilder()
-            .setSourceIndex(ifEvt.getSourceIndex())
-            .setTargetIndex(ifEvt.getTargetIndex())
-            .setVersion(ifEvt.getVersion()).build().toByteArray();
-      case INTPUT_INFORMATION_EVENT:
-        InputInformationEvent iEvt = (InputInformationEvent) event;
-        eventBytes = InputInformationEventProto.newBuilder()
-            .setUserPayload(ByteString.copyFrom(iEvt.getUserPayload()))
-            .build().toByteArray();
-      default:
-        throw new TezUncheckedException("Unknown TezEvent"
-           + ", type=" + eventType);
-      }
-      out.writeInt(eventBytes.length);
-      out.write(eventBytes);
-    }
-  }
-
-  private void deserializeEvent(DataInput in) throws IOException {
-    if (!in.readBoolean()) {
-      event = null;
-      return;
-    }
-    eventType = EventType.values()[in.readInt()];
-    if (eventType.equals(EventType.TASK_STATUS_UPDATE_EVENT)) {
-      // TODO NEWTEZ convert to PB
-      event = new TaskStatusUpdateEvent();
-      ((TaskStatusUpdateEvent)event).readFields(in);
-    } else {
-      int eventBytesLen = in.readInt();
-      byte[] eventBytes = new byte[eventBytesLen];
-      in.readFully(eventBytes);
-      switch (eventType) {
-      case DATA_MOVEMENT_EVENT:
-        DataMovementEventProto dmProto =
-            DataMovementEventProto.parseFrom(eventBytes);
-        event = new DataMovementEvent(dmProto.getSourceIndex(),
-            dmProto.getTargetIndex(),
-            dmProto.getUserPayload().toByteArray());
-        break;
-      case INPUT_READ_ERROR_EVENT:
-        InputReadErrorEventProto ideProto =
-            InputReadErrorEventProto.parseFrom(eventBytes);
-        event = new InputReadErrorEvent(ideProto.getDiagnostics(),
-            ideProto.getIndex(), ideProto.getVersion());
-        break;
-      case TASK_ATTEMPT_FAILED_EVENT:
-        TaskAttemptFailedEventProto tfProto =
-            TaskAttemptFailedEventProto.parseFrom(eventBytes);
-        event = new TaskAttemptFailedEvent(tfProto.getDiagnostics());
-        break;
-      case TASK_ATTEMPT_COMPLETED_EVENT:
-        event = new TaskAttemptCompletedEvent();
-        break;
-      case INPUT_FAILED_EVENT:
-        InputFailedEventProto ifProto =
-            InputFailedEventProto.parseFrom(eventBytes);
-        event = new InputFailedEvent(ifProto.getSourceIndex(),
-            ifProto.getTargetIndex(), ifProto.getVersion());
-        break;
-      case INTPUT_INFORMATION_EVENT:
-        InputInformationEventProto infoProto =
-            InputInformationEventProto.parseFrom(eventBytes);
-        event = new InputInformationEvent(
-            infoProto.getUserPayload().toByteArray());
-        break;
-      default:
-        throw new TezUncheckedException("Unknown TezEvent"
-           + ", type=" + eventType);
-      }
-    }
-  }
-
-  @Override
-  public void write(DataOutput out) throws IOException {
-    serializeEvent(out);
-    if (sourceInfo != null) {
-      out.writeBoolean(true);
-      sourceInfo.write(out);
-    } else {
-      out.writeBoolean(false);
-    }
-    if (destinationInfo != null) {
-      out.writeBoolean(true);
-      destinationInfo.write(out);
-    } else {
-      out.writeBoolean(false);
-    }
-  }
-
-  @Override
-  public void readFields(DataInput in) throws IOException {
-    deserializeEvent(in);
-    if (in.readBoolean()) {
-      sourceInfo = new EventMetaData();
-      sourceInfo.readFields(in);
-    }
-    if (in.readBoolean()) {
-      destinationInfo = new EventMetaData();
-      destinationInfo.readFields(in);
-    }
-  }
-
-}


[24/50] [abbrv] TEZ-443. Merge tez-dag-api and tez-engine-api into a single module - tez-api (part of TEZ-398). (sseth)

Posted by ss...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/src/main/java/org/apache/tez/common/counters/AbstractCounters.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/common/counters/AbstractCounters.java b/tez-api/src/main/java/org/apache/tez/common/counters/AbstractCounters.java
new file mode 100644
index 0000000..fd4fdee
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/common/counters/AbstractCounters.java
@@ -0,0 +1,385 @@
+/*
+ * 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.common.counters;
+
+import static org.apache.tez.common.counters.CounterGroupFactory.getFrameworkGroupId;
+import static org.apache.tez.common.counters.CounterGroupFactory.isFrameworkGroup;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.concurrent.ConcurrentSkipListMap;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableUtils;
+
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Iterators;
+import com.google.common.collect.Maps;
+
+/**
+ * An abstract class to provide common implementation for the Counters
+ * container in both mapred and mapreduce packages.
+ *
+ * @param <C> type of counter inside the counters
+ * @param <G> type of group inside the counters
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public abstract class AbstractCounters<C extends TezCounter,
+                                       G extends CounterGroupBase<C>>
+    implements Writable, Iterable<G> {
+
+  protected static final Log LOG = LogFactory.getLog("mapreduce.Counters");
+
+  /**
+   * A cache from enum values to the associated counter.
+   */
+  private Map<Enum<?>, C> cache = Maps.newIdentityHashMap();
+  //framework & fs groups
+  private Map<String, G> fgroups = new ConcurrentSkipListMap<String, G>();
+  // other groups
+  private Map<String, G> groups = new ConcurrentSkipListMap<String, G>();
+  private final CounterGroupFactory<C, G> groupFactory;
+
+  // For framework counter serialization without strings
+  enum GroupType { FRAMEWORK, FILESYSTEM };
+
+  // Writes only framework and fs counters if false.
+  private boolean writeAllCounters = true;
+
+  private static final Map<String, String> legacyMap = Maps.newHashMap();
+  static {
+    legacyMap.put("org.apache.hadoop.mapred.Task$Counter",
+                  TaskCounter.class.getName());
+    legacyMap.put("org.apache.hadoop.mapred.JobInProgress$Counter",
+                  JobCounter.class.getName());
+    legacyMap.put("FileSystemCounters", FileSystemCounter.class.getName());
+  }
+
+  private final Limits limits = new Limits();
+
+  @InterfaceAudience.Private
+  public AbstractCounters(CounterGroupFactory<C, G> gf) {
+    groupFactory = gf;
+  }
+
+  /**
+   * Construct from another counters object.
+   * @param <C1> type of the other counter
+   * @param <G1> type of the other counter group
+   * @param counters the counters object to copy
+   * @param groupFactory the factory for new groups
+   */
+  @InterfaceAudience.Private
+  public <C1 extends TezCounter, G1 extends CounterGroupBase<C1>>
+  AbstractCounters(AbstractCounters<C1, G1> counters,
+                   CounterGroupFactory<C, G> groupFactory) {
+    this.groupFactory = groupFactory;
+    for(G1 group: counters) {
+      String name = group.getName();
+      G newGroup = groupFactory.newGroup(name, group.getDisplayName(), limits);
+      (isFrameworkGroup(name) ? fgroups : groups).put(name, newGroup);
+      for(TezCounter counter: group) {
+        newGroup.addCounter(counter.getName(), counter.getDisplayName(),
+                            counter.getValue());
+      }
+    }
+  }
+
+  /** Add a group.
+   * @param group object to add
+   * @return the group
+   */
+  @InterfaceAudience.Private
+  public synchronized G addGroup(G group) {
+    String name = group.getName();
+    if (isFrameworkGroup(name)) {
+      fgroups.put(name, group);
+    } else {
+      limits.checkGroups(groups.size() + 1);
+      groups.put(name, group);
+    }
+    return group;
+  }
+
+  /**
+   * Add a new group
+   * @param name of the group
+   * @param displayName of the group
+   * @return the group
+   */
+  @InterfaceAudience.Private
+  public G addGroup(String name, String displayName) {
+    return addGroup(groupFactory.newGroup(name, displayName, limits));
+  }
+
+  /**
+   * Find a counter, create one if necessary
+   * @param groupName of the counter
+   * @param counterName name of the counter
+   * @return the matching counter
+   */
+  public C findCounter(String groupName, String counterName) {
+    G grp = getGroup(groupName);
+    return grp.findCounter(counterName);
+  }
+
+  /**
+   * Find the counter for the given enum. The same enum will always return the
+   * same counter.
+   * @param key the counter key
+   * @return the matching counter object
+   */
+  public synchronized C findCounter(Enum<?> key) {
+    C counter = cache.get(key);
+    if (counter == null) {
+      counter = findCounter(key.getDeclaringClass().getName(), key.name());
+      cache.put(key, counter);
+    }
+    return counter;
+  }
+
+  /**
+   * Find the file system counter for the given scheme and enum.
+   * @param scheme of the file system
+   * @param key the enum of the counter
+   * @return the file system counter
+   */
+  @InterfaceAudience.Private
+  public synchronized C findCounter(String scheme, FileSystemCounter key) {
+    return ((FileSystemCounterGroup<C>) getGroup(
+        FileSystemCounter.class.getName()).getUnderlyingGroup()).
+        findCounter(scheme, key);
+  }
+
+  /**
+   * Returns the names of all counter classes.
+   * @return Set of counter names.
+   */
+  public synchronized Iterable<String> getGroupNames() {
+    HashSet<String> deprecated = new HashSet<String>();
+    for(Map.Entry<String, String> entry : legacyMap.entrySet()) {
+      String newGroup = entry.getValue();
+      boolean isFGroup = isFrameworkGroup(newGroup);
+      if(isFGroup ? fgroups.containsKey(newGroup) : groups.containsKey(newGroup)) {
+        deprecated.add(entry.getKey());
+      }
+    }
+    return Iterables.concat(fgroups.keySet(), groups.keySet(), deprecated);
+  }
+
+  @Override
+  public Iterator<G> iterator() {
+    return Iterators.concat(fgroups.values().iterator(),
+                            groups.values().iterator());
+  }
+
+  /**
+   * Returns the named counter group, or an empty group if there is none
+   * with the specified name.
+   * @param groupName name of the group
+   * @return the group
+   */
+  public synchronized G getGroup(String groupName) {
+
+    // filterGroupName
+    boolean groupNameInLegacyMap = true;
+    String newGroupName = legacyMap.get(groupName);
+    if (newGroupName == null) {
+      groupNameInLegacyMap = false;
+      newGroupName = Limits.filterGroupName(groupName);
+    }
+
+    boolean isFGroup = isFrameworkGroup(newGroupName);
+    G group = isFGroup ? fgroups.get(newGroupName) : groups.get(newGroupName);
+    if (group == null) {
+      group = groupFactory.newGroup(newGroupName, limits);
+      if (isFGroup) {
+        fgroups.put(newGroupName, group);
+      } else {
+        limits.checkGroups(groups.size() + 1);
+        groups.put(newGroupName, group);
+      }
+      if (groupNameInLegacyMap) {
+        LOG.warn("Group " + groupName + " is deprecated. Use " + newGroupName
+            + " instead");
+      }
+    }
+    return group;
+  }
+
+  /**
+   * Returns the total number of counters, by summing the number of counters
+   * in each group.
+   * @return the total number of counters
+   */
+  public synchronized int countCounters() {
+    int result = 0;
+    for (G group : this) {
+      result += group.size();
+    }
+    return result;
+  }
+
+  /**
+   * Write the set of groups.
+   * Counters ::= version #fgroups (groupId, group)* #groups (group)*
+   */
+  @Override
+  public synchronized void write(DataOutput out) throws IOException {
+    WritableUtils.writeVInt(out, groupFactory.version());
+    WritableUtils.writeVInt(out, fgroups.size());  // framework groups first
+    for (G group : fgroups.values()) {
+      if (group.getUnderlyingGroup() instanceof FrameworkCounterGroup<?, ?>) {
+        WritableUtils.writeVInt(out, GroupType.FRAMEWORK.ordinal());
+        WritableUtils.writeVInt(out, getFrameworkGroupId(group.getName()));
+        group.write(out);
+      } else if (group.getUnderlyingGroup() instanceof FileSystemCounterGroup<?>) {
+        WritableUtils.writeVInt(out, GroupType.FILESYSTEM.ordinal());
+        group.write(out);
+      }
+    }
+    if (writeAllCounters) {
+      WritableUtils.writeVInt(out, groups.size());
+      for (G group : groups.values()) {
+        Text.writeString(out, group.getName());
+        group.write(out);
+      }
+    } else {
+      WritableUtils.writeVInt(out, 0);
+    }
+  }
+
+  @Override
+  public synchronized void readFields(DataInput in) throws IOException {
+    int version = WritableUtils.readVInt(in);
+    if (version != groupFactory.version()) {
+      throw new IOException("Counters version mismatch, expected "+
+          groupFactory.version() +" got "+ version);
+    }
+    int numFGroups = WritableUtils.readVInt(in);
+    fgroups.clear();
+    GroupType[] groupTypes = GroupType.values();
+    while (numFGroups-- > 0) {
+      GroupType groupType = groupTypes[WritableUtils.readVInt(in)];
+      G group;
+      switch (groupType) {
+        case FILESYSTEM: // with nothing
+          group = groupFactory.newFileSystemGroup();
+          break;
+        case FRAMEWORK:  // with group id
+          group = groupFactory.newFrameworkGroup(WritableUtils.readVInt(in));
+          break;
+        default: // Silence dumb compiler, as it would've thrown earlier
+          throw new IOException("Unexpected counter group type: "+ groupType);
+      }
+      group.readFields(in);
+      fgroups.put(group.getName(), group);
+    }
+    int numGroups = WritableUtils.readVInt(in);
+    while (numGroups-- > 0) {
+      limits.checkGroups(groups.size() + 1);
+      G group = groupFactory.newGenericGroup(Text.readString(in), null, limits);
+      group.readFields(in);
+      groups.put(group.getName(), group);
+    }
+  }
+
+  /**
+   * Return textual representation of the counter values.
+   * @return the string
+   */
+  @Override
+  public synchronized String toString() {
+    StringBuilder sb = new StringBuilder("Counters: " + countCounters());
+    for (G group: this) {
+      sb.append("\n\t").append(group.getDisplayName());
+      for (TezCounter counter: group) {
+        sb.append("\n\t\t").append(counter.getDisplayName()).append("=")
+          .append(counter.getValue());
+      }
+    }
+    return sb.toString();
+  }
+
+  /**
+   * Increments multiple counters by their amounts in another Counters
+   * instance.
+   * @param other the other Counters instance
+   */
+  public synchronized void incrAllCounters(AbstractCounters<C, G> other) {
+    for(G right : other) {
+      String groupName = right.getName();
+      G left = (isFrameworkGroup(groupName) ? fgroups : groups).get(groupName);
+      if (left == null) {
+        left = addGroup(groupName, right.getDisplayName());
+      }
+      left.incrAllCounters(right);
+    }
+  }
+
+  @Override
+  @SuppressWarnings("unchecked")
+  public boolean equals(Object genericRight) {
+    if (genericRight instanceof AbstractCounters<?, ?>) {
+      return Iterators.elementsEqual(iterator(),
+          ((AbstractCounters<C, G>)genericRight).iterator());
+    }
+    return false;
+  }
+
+  @Override
+  public int hashCode() {
+    return groups.hashCode();
+  }
+
+  /**
+   * Set the "writeAllCounters" option to true or false
+   * @param send  if true all counters would be serialized, otherwise only
+   *              framework counters would be serialized in
+   *              {@link #write(DataOutput)}
+   */
+  @InterfaceAudience.Private
+  public void setWriteAllCounters(boolean send) {
+    writeAllCounters = send;
+  }
+
+  /**
+   * Get the "writeAllCounters" option
+   * @return true of all counters would serialized
+   */
+  @InterfaceAudience.Private
+  public boolean getWriteAllCounters() {
+    return writeAllCounters;
+  }
+
+  @InterfaceAudience.Private
+  public Limits limits() {
+    return limits;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/src/main/java/org/apache/tez/common/counters/CounterGroup.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/common/counters/CounterGroup.java b/tez-api/src/main/java/org/apache/tez/common/counters/CounterGroup.java
new file mode 100644
index 0000000..bc7986d
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/common/counters/CounterGroup.java
@@ -0,0 +1,32 @@
+/**
+ * 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.common.counters;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * A group of {@link TezCounter}s that logically belong together. Typically,
+ * it is an {@link Enum} subclass and the counters are the values.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public interface CounterGroup extends CounterGroupBase<TezCounter> {
+  // essentially a typedef so user doesn't have to use generic syntax
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/src/main/java/org/apache/tez/common/counters/CounterGroupBase.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/common/counters/CounterGroupBase.java b/tez-api/src/main/java/org/apache/tez/common/counters/CounterGroupBase.java
new file mode 100644
index 0000000..3b702ba
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/common/counters/CounterGroupBase.java
@@ -0,0 +1,108 @@
+/*
+ * 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.common.counters;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.io.Writable;
+
+/**
+ * The common counter group interface.
+ *
+ * @param <T> type of the counter for the group
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public interface CounterGroupBase<T extends TezCounter>
+    extends Writable, Iterable<T> {
+
+  /**
+   * Get the internal name of the group
+   * @return the internal name
+   */
+  String getName();
+
+  /**
+   * Get the display name of the group.
+   * @return the human readable name
+   */
+  String getDisplayName();
+
+  /**
+   * Set the display name of the group
+   * @param displayName of the group
+   */
+  void setDisplayName(String displayName);
+
+  /** Add a counter to this group.
+   * @param counter to add
+   */
+  void addCounter(T counter);
+
+  /**
+   * Add a counter to this group
+   * @param name  of the counter
+   * @param displayName of the counter
+   * @param value of the counter
+   * @return the counter
+   */
+  T addCounter(String name, String displayName, long value);
+
+  /**
+   * Find a counter in the group.
+   * @param counterName the name of the counter
+   * @param displayName the display name of the counter
+   * @return the counter that was found or added
+   */
+  T findCounter(String counterName, String displayName);
+
+  /**
+   * Find a counter in the group
+   * @param counterName the name of the counter
+   * @param create create the counter if not found if true
+   * @return the counter that was found or added or null if create is false
+   */
+  T findCounter(String counterName, boolean create);
+
+  /**
+   * Find a counter in the group.
+   * @param counterName the name of the counter
+   * @return the counter that was found or added
+   */
+  T findCounter(String counterName);
+
+  /**
+   * @return the number of counters in this group.
+   */
+  int size();
+
+  /**
+   * Increment all counters by a group of counters
+   * @param rightGroup  the group to be added to this group
+   */
+  void incrAllCounters(CounterGroupBase<T> rightGroup);
+  
+  @Private
+  /**
+   * Exposes the underlying group type if a facade.
+   * @return the underlying object that this object is wrapping up.
+   */
+  CounterGroupBase<T> getUnderlyingGroup();
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/src/main/java/org/apache/tez/common/counters/CounterGroupFactory.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/common/counters/CounterGroupFactory.java b/tez-api/src/main/java/org/apache/tez/common/counters/CounterGroupFactory.java
new file mode 100644
index 0000000..45da0dd
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/common/counters/CounterGroupFactory.java
@@ -0,0 +1,180 @@
+/*
+ * 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.common.counters;
+
+import java.util.List;
+import java.util.Map;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+
+/**
+ * An abstract class to provide common implementation of the
+ * group factory in both mapred and mapreduce packages.
+ *
+ * @param <C> type of the counter
+ * @param <G> type of the group
+ */
+@InterfaceAudience.Private
+public abstract class CounterGroupFactory<C extends TezCounter,
+                                          G extends CounterGroupBase<C>> {
+
+  public interface FrameworkGroupFactory<F> {
+    F newGroup(String name);
+  }
+
+  // Integer mapping (for serialization) for framework groups
+  private static final Map<String, Integer> s2i = Maps.newHashMap();
+  private static final List<String> i2s = Lists.newArrayList();
+  private static final int VERSION = 1;
+  private static final String FS_GROUP_NAME = FileSystemCounter.class.getName();
+
+  private final Map<String, FrameworkGroupFactory<G>> fmap = Maps.newHashMap();
+  {
+    // Add builtin counter class here and the version when changed.
+    addFrameworkGroup(TaskCounter.class);
+    addFrameworkGroup(JobCounter.class);
+    addFrameworkGroup(DAGCounter.class);
+  }
+
+  // Initialize the framework counter group mapping
+  private synchronized <T extends Enum<T>>
+  void addFrameworkGroup(final Class<T> cls) {
+    updateFrameworkGroupMapping(cls);
+    fmap.put(cls.getName(), newFrameworkGroupFactory(cls));
+  }
+
+  // Update static mappings (c2i, i2s) of framework groups
+  private static synchronized void updateFrameworkGroupMapping(Class<?> cls) {
+    String name = cls.getName();
+    Integer i = s2i.get(name);
+    if (i != null) return;
+    i2s.add(name);
+    s2i.put(name, i2s.size() - 1);
+  }
+
+  /**
+   * Required override to return a new framework group factory
+   * @param <T> type of the counter enum class
+   * @param cls the counter enum class
+   * @return a new framework group factory
+   */
+  protected abstract <T extends Enum<T>>
+  FrameworkGroupFactory<G> newFrameworkGroupFactory(Class<T> cls);
+
+  /**
+   * Create a new counter group
+   * @param name of the group
+   * @param limits the counters limits policy object
+   * @return a new counter group
+   */
+  public G newGroup(String name, Limits limits) {
+    return newGroup(name, ResourceBundles.getCounterGroupName(name, name),
+                    limits);
+  }
+
+  /**
+   * Create a new counter group
+   * @param name of the group
+   * @param displayName of the group
+   * @param limits the counters limits policy object
+   * @return a new counter group
+   */
+  public G newGroup(String name, String displayName, Limits limits) {
+    FrameworkGroupFactory<G> gf = fmap.get(name);
+    if (gf != null) return gf.newGroup(name);
+    if (name.equals(FS_GROUP_NAME)) {
+      return newFileSystemGroup();
+    } else if (s2i.get(name) != null) {
+      return newFrameworkGroup(s2i.get(name));
+    }
+    return newGenericGroup(name, displayName, limits);
+  }
+
+  /**
+   * Create a new framework group
+   * @param id of the group
+   * @return a new framework group
+   */
+  public G newFrameworkGroup(int id) {
+    String name;
+    synchronized(CounterGroupFactory.class) {
+      if (id < 0 || id >= i2s.size()) throwBadFrameGroupIdException(id);
+      name = i2s.get(id); // should not throw here.
+    }
+    FrameworkGroupFactory<G> gf = fmap.get(name);
+    if (gf == null) throwBadFrameGroupIdException(id);
+    return gf.newGroup(name);
+  }
+
+  /**
+   * Get the id of a framework group
+   * @param name of the group
+   * @return the framework group id
+   */
+  public static synchronized int getFrameworkGroupId(String name) {
+    Integer i = s2i.get(name);
+    if (i == null) throwBadFrameworkGroupNameException(name);
+    return i;
+  }
+
+  /**
+   * @return the counter factory version
+   */
+  public int version() {
+    return VERSION;
+  }
+
+  /**
+   * Check whether a group name is a name of a framework group (including
+   * the filesystem group).
+   *
+   * @param name  to check
+   * @return true for framework group names
+   */
+  public static synchronized boolean isFrameworkGroup(String name) {
+    return s2i.get(name) != null || name.equals(FS_GROUP_NAME);
+  }
+
+  private static void throwBadFrameGroupIdException(int id) {
+    throw new IllegalArgumentException("bad framework group id: "+ id);
+  }
+
+  private static void throwBadFrameworkGroupNameException(String name) {
+    throw new IllegalArgumentException("bad framework group name: "+ name);
+  }
+
+  /**
+   * Abstract factory method to create a generic (vs framework) counter group
+   * @param name  of the group
+   * @param displayName of the group
+   * @param limits limits of the counters
+   * @return a new generic counter group
+   */
+  protected abstract G newGenericGroup(String name, String displayName,
+                                       Limits limits);
+
+  /**
+   * Abstract factory method to create a file system counter group
+   * @return a new file system counter group
+   */
+  protected abstract G newFileSystemGroup();
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/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
new file mode 100644
index 0000000..3598572
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/common/counters/DAGCounter.java
@@ -0,0 +1,39 @@
+/**
+ * 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.common.counters;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+// Per-job counters
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public enum DAGCounter {
+  NUM_FAILED_TASKS, 
+  NUM_KILLED_TASKS,
+  TOTAL_LAUNCHED_TASKS,
+  OTHER_LOCAL_TASKS,
+  DATA_LOCAL_TASKS,
+  RACK_LOCAL_TASKS,
+  SLOTS_MILLIS_TASKS,
+  FALLOW_SLOTS_MILLIS_TASKS,
+  TOTAL_LAUNCHED_UBERTASKS,
+  NUM_UBER_SUBTASKS,
+  NUM_FAILED_UBERTASKS
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/src/main/java/org/apache/tez/common/counters/FileSystemCounter.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/common/counters/FileSystemCounter.java b/tez-api/src/main/java/org/apache/tez/common/counters/FileSystemCounter.java
new file mode 100644
index 0000000..08f4c5d
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/common/counters/FileSystemCounter.java
@@ -0,0 +1,30 @@
+/*
+ * 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.common.counters;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+
+@InterfaceAudience.Private
+public enum FileSystemCounter {
+  BYTES_READ,
+  BYTES_WRITTEN,
+  READ_OPS,
+  LARGE_READ_OPS,
+  WRITE_OPS,
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/src/main/java/org/apache/tez/common/counters/FileSystemCounterGroup.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/common/counters/FileSystemCounterGroup.java b/tez-api/src/main/java/org/apache/tez/common/counters/FileSystemCounterGroup.java
new file mode 100644
index 0000000..d4b167a
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/common/counters/FileSystemCounterGroup.java
@@ -0,0 +1,327 @@
+/*
+ * 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.common.counters;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.Iterator;
+import java.util.Locale;
+import java.util.Map;
+
+import com.google.common.base.Joiner;
+import static com.google.common.base.Preconditions.*;
+import com.google.common.collect.AbstractIterator;
+import com.google.common.collect.Iterators;
+import com.google.common.collect.Maps;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.io.WritableUtils;
+
+/**
+ * An abstract class to provide common implementation of the filesystem
+ * counter group in both mapred and mapreduce packages.
+ *
+ * @param <C> the type of the Counter for the group
+ */
+@InterfaceAudience.Private
+public abstract class FileSystemCounterGroup<C extends TezCounter>
+    implements CounterGroupBase<C> {
+
+  static final int MAX_NUM_SCHEMES = 100; // intern/sanity check
+  static final ConcurrentMap<String, String> schemes = Maps.newConcurrentMap();
+
+  // C[] would need Array.newInstance which requires a Class<C> reference.
+  // Just a few local casts probably worth not having to carry it around.
+  private final Map<String, Object[]> map =
+    new ConcurrentSkipListMap<String, Object[]>();
+  private String displayName;
+
+  private static final Joiner NAME_JOINER = Joiner.on('_');
+  private static final Joiner DISP_JOINER = Joiner.on(": ");
+
+  @InterfaceAudience.Private
+  public static class FSCounter extends AbstractCounter {
+    final String scheme;
+    final FileSystemCounter key;
+    private long value;
+
+    public FSCounter(String scheme, FileSystemCounter ref) {
+      this.scheme = scheme;
+      key = ref;
+    }
+
+    @Override
+    public String getName() {
+      return NAME_JOINER.join(scheme, key.name());
+    }
+
+    @Override
+    public String getDisplayName() {
+      return DISP_JOINER.join(scheme, localizeCounterName(key.name()));
+    }
+
+    protected String localizeCounterName(String counterName) {
+      return ResourceBundles.getCounterName(FileSystemCounter.class.getName(),
+                                            counterName, counterName);
+    }
+
+    @Override
+    public long getValue() {
+      return value;
+    }
+
+    @Override
+    public void setValue(long value) {
+      this.value = value;
+    }
+
+    @Override
+    public void increment(long incr) {
+      value += incr;
+    }
+
+    @Override
+    public void write(DataOutput out) throws IOException {
+      assert false : "shouldn't be called";
+    }
+
+    @Override
+    public void readFields(DataInput in) throws IOException {
+      assert false : "shouldn't be called";
+    }
+
+    @Override
+    public TezCounter getUnderlyingCounter() {
+      return this;
+    }
+  }
+
+  @Override
+  public String getName() {
+    return FileSystemCounter.class.getName();
+  }
+
+  @Override
+  public String getDisplayName() {
+    if (displayName == null) {
+      displayName = ResourceBundles.getCounterGroupName(getName(),
+          "File System Counters");
+    }
+    return displayName;
+  }
+
+  @Override
+  public void setDisplayName(String displayName) {
+    this.displayName = displayName;
+  }
+
+  @Override
+  public void addCounter(C counter) {
+    C ours;
+    if (counter instanceof FileSystemCounterGroup.FSCounter) {
+      FSCounter c = (FSCounter) counter;
+      ours = findCounter(c.scheme, c.key);
+    }
+    else {
+      ours = findCounter(counter.getName());
+    }
+    ours.setValue(counter.getValue());
+  }
+
+  @Override
+  public C addCounter(String name, String displayName, long value) {
+    C counter = findCounter(name);
+    counter.setValue(value);
+    return counter;
+  }
+
+  // Parse generic counter name into [scheme, key]
+  private String[] parseCounterName(String counterName) {
+    int schemeEnd = counterName.indexOf('_');
+    if (schemeEnd < 0) {
+      throw new IllegalArgumentException("bad fs counter name");
+    }
+    return new String[]{counterName.substring(0, schemeEnd),
+                        counterName.substring(schemeEnd + 1)};
+  }
+
+  @Override
+  public C findCounter(String counterName, String displayName) {
+    return findCounter(counterName);
+  }
+
+  @Override
+  public C findCounter(String counterName, boolean create) {
+    try {
+      String[] pair = parseCounterName(counterName);
+      return findCounter(pair[0], FileSystemCounter.valueOf(pair[1]));
+    }
+    catch (Exception e) {
+      if (create) throw new IllegalArgumentException(e);
+      return null;
+    }
+  }
+
+  @Override
+  public C findCounter(String counterName) {
+    return findCounter(counterName, true);
+  }
+
+  @SuppressWarnings("unchecked")
+  public synchronized C findCounter(String scheme, FileSystemCounter key) {
+    final String canonicalScheme = checkScheme(scheme);
+    Object[] counters = map.get(canonicalScheme);
+    int ord = key.ordinal();
+    if (counters == null) {
+      counters = new Object[FileSystemCounter.values().length];
+      map.put(canonicalScheme, counters);
+      counters[ord] = newCounter(canonicalScheme, key);
+    }
+    else if (counters[ord] == null) {
+      counters[ord] = newCounter(canonicalScheme, key);
+    }
+    return (C) counters[ord];
+  }
+
+  private String checkScheme(String scheme) {
+    String fixed = scheme.toUpperCase(Locale.US);
+    String interned = schemes.putIfAbsent(fixed, fixed);
+    if (schemes.size() > MAX_NUM_SCHEMES) {
+      // mistakes or abuses
+      throw new IllegalArgumentException("too many schemes? "+ schemes.size() +
+                                         " when process scheme: "+ scheme);
+    }
+    return interned == null ? fixed : interned;
+  }
+
+  /**
+   * Abstract factory method to create a file system counter
+   * @param scheme of the file system
+   * @param key the enum of the file system counter
+   * @return a new file system counter
+   */
+  protected abstract C newCounter(String scheme, FileSystemCounter key);
+
+  @Override
+  public int size() {
+    int n = 0;
+    for (Object[] counters : map.values()) {
+      n += numSetCounters(counters);
+    }
+    return n;
+  }
+
+  @Override
+  @SuppressWarnings("unchecked")
+  public void incrAllCounters(CounterGroupBase<C> other) {
+    if (checkNotNull(other.getUnderlyingGroup(), "other group")
+        instanceof FileSystemCounterGroup<?>) {
+      for (TezCounter counter : other) {
+        FSCounter c = (FSCounter) ((TezCounter)counter).getUnderlyingCounter();
+        findCounter(c.scheme, c.key) .increment(counter.getValue());
+      }
+    }
+  }
+
+  /**
+   * FileSystemGroup ::= #scheme (scheme #counter (key value)*)*
+   */
+  @Override
+  public void write(DataOutput out) throws IOException {
+    WritableUtils.writeVInt(out, map.size()); // #scheme
+    for (Map.Entry<String, Object[]> entry : map.entrySet()) {
+      WritableUtils.writeString(out, entry.getKey()); // scheme
+      // #counter for the above scheme
+      WritableUtils.writeVInt(out, numSetCounters(entry.getValue()));
+      for (Object counter : entry.getValue()) {
+        if (counter == null) continue;
+        @SuppressWarnings("unchecked")
+        FSCounter c = (FSCounter) ((TezCounter)counter).getUnderlyingCounter();
+        WritableUtils.writeVInt(out, c.key.ordinal());  // key
+        WritableUtils.writeVLong(out, c.getValue());    // value
+      }
+    }
+  }
+
+  private int numSetCounters(Object[] counters) {
+    int n = 0;
+    for (Object counter : counters) if (counter != null) ++n;
+    return n;
+  }
+
+  @Override
+  public void readFields(DataInput in) throws IOException {
+    int numSchemes = WritableUtils.readVInt(in);    // #scheme
+    FileSystemCounter[] enums = FileSystemCounter.values();
+    for (int i = 0; i < numSchemes; ++i) {
+      String scheme = WritableUtils.readString(in); // scheme
+      int numCounters = WritableUtils.readVInt(in); // #counter
+      for (int j = 0; j < numCounters; ++j) {
+        findCounter(scheme, enums[WritableUtils.readVInt(in)])  // key
+            .setValue(WritableUtils.readVLong(in)); // value
+      }
+    }
+  }
+
+  @Override
+  public Iterator<C> iterator() {
+    return new AbstractIterator<C>() {
+      Iterator<Object[]> it = map.values().iterator();
+      Object[] counters = it.hasNext() ? it.next() : null;
+      int i = 0;
+      @Override
+      protected C computeNext() {
+        while (counters != null) {
+          while (i < counters.length) {
+            @SuppressWarnings("unchecked")
+            C counter = (C) counters[i++];
+            if (counter != null) return counter;
+          }
+          i = 0;
+          counters = it.hasNext() ? it.next() : null;
+        }
+        return endOfData();
+      }
+    };
+  }
+
+  @Override
+  public synchronized boolean equals(Object genericRight) {
+    if (genericRight instanceof CounterGroupBase<?>) {
+      @SuppressWarnings("unchecked")
+      CounterGroupBase<C> right = (CounterGroupBase<C>) genericRight;
+      return Iterators.elementsEqual(iterator(), right.iterator());
+    }
+    return false;
+  }
+
+  @Override
+  public synchronized int hashCode() {
+    // need to be deep as counters is an array
+    int hash = FileSystemCounter.class.hashCode();
+    for (Object[] counters : map.values()) {
+      if (counters != null) hash ^= Arrays.hashCode(counters);
+    }
+    return hash;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/src/main/java/org/apache/tez/common/counters/FrameworkCounterGroup.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/common/counters/FrameworkCounterGroup.java b/tez-api/src/main/java/org/apache/tez/common/counters/FrameworkCounterGroup.java
new file mode 100644
index 0000000..42fb636
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/common/counters/FrameworkCounterGroup.java
@@ -0,0 +1,275 @@
+/*
+ * 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.common.counters;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Iterator;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.io.WritableUtils;
+
+import com.google.common.collect.AbstractIterator;
+import com.google.common.collect.Iterators;
+
+/**
+ * An abstract class to provide common implementation for the framework
+ * counter group in both mapred and mapreduce packages.
+ *
+ * @param <T> type of the counter enum class
+ * @param <C> type of the counter
+ */
+@InterfaceAudience.Private
+public abstract class FrameworkCounterGroup<T extends Enum<T>,
+    C extends TezCounter> implements CounterGroupBase<C> {
+  private static final Log LOG = LogFactory.getLog(FrameworkCounterGroup.class);
+  
+  private final Class<T> enumClass; // for Enum.valueOf
+  private final Object[] counters;  // local casts are OK and save a class ref
+  private String displayName = null;
+
+  /**
+   * A counter facade for framework counters.
+   * Use old (which extends new) interface to make compatibility easier.
+   */
+  @InterfaceAudience.Private
+  public static class FrameworkCounter<T extends Enum<T>> extends AbstractCounter {
+    final T key;
+    final String groupName;
+    private long value;
+
+    public FrameworkCounter(T ref, String groupName) {
+      key = ref;
+      this.groupName = groupName;
+    }
+
+    @Override
+    public String getName() {
+      return key.name();
+    }
+
+    @Override
+    public String getDisplayName() {
+      return ResourceBundles.getCounterName(groupName, getName(), getName());
+    }
+
+    @Override
+    public long getValue() {
+      return value;
+    }
+
+    @Override
+    public void setValue(long value) {
+      this.value = value;
+    }
+
+    @Override
+    public void increment(long incr) {
+      value += incr;
+    }
+
+    @Override
+    public void write(DataOutput out) throws IOException {
+      assert false : "shouldn't be called";
+    }
+
+    @Override
+    public void readFields(DataInput in) throws IOException {
+      assert false : "shouldn't be called";
+    }
+
+    @Override
+    public TezCounter getUnderlyingCounter() {
+      return this;
+    }
+  }
+
+  @SuppressWarnings("unchecked")
+  public FrameworkCounterGroup(Class<T> enumClass) {
+    this.enumClass = enumClass;
+    T[] enums = enumClass.getEnumConstants();
+    counters = new Object[enums.length];
+  }
+
+  @Override
+  public String getName() {
+    return enumClass.getName();
+  }
+
+  @Override
+  public String getDisplayName() {
+    if (displayName == null) {
+      displayName = ResourceBundles.getCounterGroupName(getName(), getName());
+    }
+    return displayName;
+  }
+
+  @Override
+  public void setDisplayName(String displayName) {
+    this.displayName = displayName;
+  }
+
+  private T valueOf(String name) {
+    return Enum.valueOf(enumClass, name);
+  }
+
+  @Override
+  public void addCounter(C counter) {
+    C ours = findCounter(counter.getName());
+    ours.setValue(counter.getValue());
+  }
+
+  @Override
+  public C addCounter(String name, String displayName, long value) {
+    C counter = findCounter(name);
+    counter.setValue(value);
+    return counter;
+  }
+
+  @Override
+  public C findCounter(String counterName, String displayName) {
+    return findCounter(counterName);
+  }
+
+  @Override
+  public C findCounter(String counterName, boolean create) {
+    try {
+      return findCounter(valueOf(counterName));
+    }
+    catch (Exception e) {
+      if (create) throw new IllegalArgumentException(e);
+      return null;
+    }
+  }
+
+  @Override
+  public C findCounter(String counterName) {
+    return findCounter(valueOf(counterName));
+  }
+
+  @SuppressWarnings("unchecked")
+  private C findCounter(T key) {
+    int i = key.ordinal();
+    if (counters[i] == null) {
+      counters[i] = newCounter(key);
+    }
+    return (C) counters[i];
+  }
+
+  /**
+   * Abstract factory method for new framework counter
+   * @param key for the enum value of a counter
+   * @return a new counter for the key
+   */
+  protected abstract C newCounter(T key);
+
+  @Override
+  public int size() {
+    int n = 0;
+    for (int i = 0; i < counters.length; ++i) {
+      if (counters[i] != null) ++n;
+    }
+    return n;
+  }
+
+  @Override
+  @SuppressWarnings("unchecked")
+  public void incrAllCounters(CounterGroupBase<C> other) {
+    if (checkNotNull(other, "other counter group")
+        instanceof FrameworkCounterGroup<?, ?>) {
+      for (TezCounter counter : other) {
+        findCounter(((FrameworkCounter) counter).key.name())
+            .increment(counter.getValue());
+      }
+    }
+  }
+
+  /**
+   * FrameworkGroup ::= #counter (key value)*
+   */
+  @Override
+  @SuppressWarnings("unchecked")
+  public void write(DataOutput out) throws IOException {
+    WritableUtils.writeVInt(out, size());
+    for (int i = 0; i < counters.length; ++i) {
+      TezCounter counter = (C) counters[i];
+      if (counter != null) {
+        WritableUtils.writeVInt(out, i);
+        WritableUtils.writeVLong(out, counter.getValue());
+      }
+    }
+  }
+
+  @Override
+  public void readFields(DataInput in) throws IOException {
+    clear();
+    int len = WritableUtils.readVInt(in);
+    T[] enums = enumClass.getEnumConstants();
+    for (int i = 0; i < len; ++i) {
+      int ord = WritableUtils.readVInt(in);
+      TezCounter counter = newCounter(enums[ord]);
+      counter.setValue(WritableUtils.readVLong(in));
+      counters[ord] = counter;
+    }
+  }
+
+  private void clear() {
+    for (int i = 0; i < counters.length; ++i) {
+      counters[i] = null;
+    }
+  }
+
+  @Override
+  public Iterator<C> iterator() {
+    return new AbstractIterator<C>() {
+      int i = 0;
+      @Override
+      protected C computeNext() {
+        while (i < counters.length) {
+          @SuppressWarnings("unchecked")
+          C counter = (C) counters[i++];
+          if (counter != null) return counter;
+        }
+        return endOfData();
+      }
+    };
+  }
+
+  @Override
+  public boolean equals(Object genericRight) {
+    if (genericRight instanceof CounterGroupBase<?>) {
+      @SuppressWarnings("unchecked")
+      CounterGroupBase<C> right = (CounterGroupBase<C>) genericRight;
+      return Iterators.elementsEqual(iterator(), right.iterator());
+    }
+    return false;
+  }
+
+  @Override
+  public synchronized int hashCode() {
+    // need to be deep as counters is an array
+    return Arrays.deepHashCode(new Object[]{enumClass, counters, displayName});
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/src/main/java/org/apache/tez/common/counters/GenericCounter.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/common/counters/GenericCounter.java b/tez-api/src/main/java/org/apache/tez/common/counters/GenericCounter.java
new file mode 100644
index 0000000..5477606
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/common/counters/GenericCounter.java
@@ -0,0 +1,109 @@
+/*
+ * 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.common.counters;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.WritableUtils;
+
+/**
+ * A generic counter implementation
+ */
+@InterfaceAudience.Private
+public class GenericCounter extends AbstractCounter {
+
+  private String name;
+  private String displayName;
+  private long value = 0;
+
+  public GenericCounter() {
+    // mostly for readFields
+  }
+
+  public GenericCounter(String name, String displayName) {
+    this.name = name;
+    this.displayName = displayName;
+  }
+
+  public GenericCounter(String name, String displayName, long value) {
+    this.name = name;
+    this.displayName = displayName;
+    this.value = value;
+  }
+
+  @Override @Deprecated
+  public synchronized void setDisplayName(String displayName) {
+    this.displayName = displayName;
+  }
+
+  @Override
+  public synchronized void readFields(DataInput in) throws IOException {
+    name = Text.readString(in);
+    displayName = in.readBoolean() ? Text.readString(in) : name;
+    value = WritableUtils.readVLong(in);
+  }
+
+  /**
+   * GenericCounter ::= keyName isDistinctDisplayName [displayName] value
+   */
+  @Override
+  public synchronized void write(DataOutput out) throws IOException {
+    Text.writeString(out, name);
+    boolean distinctDisplayName = ! name.equals(displayName);
+    out.writeBoolean(distinctDisplayName);
+    if (distinctDisplayName) {
+      Text.writeString(out, displayName);
+    }
+    WritableUtils.writeVLong(out, value);
+  }
+
+  @Override
+  public synchronized String getName() {
+    return name;
+  }
+
+  @Override
+  public synchronized String getDisplayName() {
+    return displayName;
+  }
+
+  @Override
+  public synchronized long getValue() {
+    return value;
+  }
+
+  @Override
+  public synchronized void setValue(long value) {
+    this.value = value;
+  }
+
+  @Override
+  public synchronized void increment(long incr) {
+    value += incr;
+  }
+
+  @Override
+  public TezCounter getUnderlyingCounter() {
+    return this;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/src/main/java/org/apache/tez/common/counters/JobCounter.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/common/counters/JobCounter.java b/tez-api/src/main/java/org/apache/tez/common/counters/JobCounter.java
new file mode 100644
index 0000000..1eb2be8
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/common/counters/JobCounter.java
@@ -0,0 +1,45 @@
+/**
+ * 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.common.counters;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+// Per-job counters
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public enum JobCounter {
+  NUM_FAILED_MAPS, 
+  NUM_FAILED_REDUCES,
+  NUM_KILLED_MAPS,
+  NUM_KILLED_REDUCES,
+  TOTAL_LAUNCHED_MAPS,
+  TOTAL_LAUNCHED_REDUCES,
+  OTHER_LOCAL_MAPS,
+  DATA_LOCAL_MAPS,
+  RACK_LOCAL_MAPS,
+  SLOTS_MILLIS_MAPS,
+  SLOTS_MILLIS_REDUCES,
+  FALLOW_SLOTS_MILLIS_MAPS,
+  FALLOW_SLOTS_MILLIS_REDUCES,
+  TOTAL_LAUNCHED_UBERTASKS,
+  NUM_UBER_SUBMAPS,
+  NUM_UBER_SUBREDUCES,
+  NUM_FAILED_UBERTASKS
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/src/main/java/org/apache/tez/common/counters/LimitExceededException.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/common/counters/LimitExceededException.java b/tez-api/src/main/java/org/apache/tez/common/counters/LimitExceededException.java
new file mode 100644
index 0000000..e50bd81
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/common/counters/LimitExceededException.java
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.common.counters;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+
+@InterfaceAudience.Private
+public class LimitExceededException extends RuntimeException {
+
+  private static final long serialVersionUID = 1L;
+
+  public LimitExceededException(String msg) {
+    super(msg);
+  }
+
+  // Only allows chaining of related exceptions
+  public LimitExceededException(LimitExceededException cause) {
+    super(cause);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/src/main/java/org/apache/tez/common/counters/Limits.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/common/counters/Limits.java b/tez-api/src/main/java/org/apache/tez/common/counters/Limits.java
new file mode 100644
index 0000000..aacce87
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/common/counters/Limits.java
@@ -0,0 +1,112 @@
+/*
+ * 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.common.counters;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.tez.common.TezJobConfig;
+
+@InterfaceAudience.Private
+public class Limits {
+
+  private static Configuration conf = null;
+  private static int GROUP_NAME_MAX;
+  private static int COUNTER_NAME_MAX;
+  private static int GROUPS_MAX;
+  private static int COUNTERS_MAX;
+  private static boolean initialized = false;
+
+  private static synchronized void ensureInitialized() {
+    if (initialized) {
+      return;
+    }
+    if (conf == null) {
+      conf = new Configuration();
+    }
+    GROUP_NAME_MAX =
+        conf.getInt(TezJobConfig.COUNTER_GROUP_NAME_MAX_KEY,
+            TezJobConfig.COUNTER_GROUP_NAME_MAX_DEFAULT);
+    COUNTER_NAME_MAX =
+        conf.getInt(TezJobConfig.COUNTER_NAME_MAX_KEY,
+            TezJobConfig.COUNTER_NAME_MAX_DEFAULT);
+    GROUPS_MAX =
+        conf.getInt(TezJobConfig.COUNTER_GROUPS_MAX_KEY,
+            TezJobConfig.COUNTER_GROUPS_MAX_DEFAULT);
+    COUNTERS_MAX =
+        conf.getInt(TezJobConfig.COUNTERS_MAX_KEY, TezJobConfig.
+            COUNTERS_MAX_DEFAULT);
+    initialized = true;
+  }
+
+  private int totalCounters;
+  private LimitExceededException firstViolation;
+
+  public static String filterName(String name, int maxLen) {
+    return name.length() > maxLen ? name.substring(0, maxLen - 1) : name;
+  }
+
+  public static String filterCounterName(String name) {
+    ensureInitialized();
+    return filterName(name, COUNTER_NAME_MAX);
+  }
+
+  public static String filterGroupName(String name) {
+    ensureInitialized();
+    return filterName(name, GROUP_NAME_MAX);
+  }
+
+  public synchronized void checkCounters(int size) {
+    ensureInitialized();
+    if (firstViolation != null) {
+      throw new LimitExceededException(firstViolation);
+    }
+    if (size > COUNTERS_MAX) {
+      firstViolation = new LimitExceededException("Too many counters: "+ size +
+                                                  " max="+ COUNTERS_MAX);
+      throw firstViolation;
+    }
+  }
+
+  public synchronized void incrCounters() {
+    checkCounters(totalCounters + 1);
+    ++totalCounters;
+  }
+
+  public synchronized void checkGroups(int size) {
+    ensureInitialized();
+    if (firstViolation != null) {
+      throw new LimitExceededException(firstViolation);
+    }
+    if (size > GROUPS_MAX) {
+      firstViolation = new LimitExceededException("Too many counter groups: "+
+                                                  size +" max="+ GROUPS_MAX);
+    }
+  }
+
+  public synchronized LimitExceededException violation() {
+    return firstViolation;
+  }
+
+  public synchronized static void setConfiguration(Configuration conf) {
+    if (Limits.conf == null && conf != null) {
+      Limits.conf = conf;
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/src/main/java/org/apache/tez/common/counters/ResourceBundles.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/common/counters/ResourceBundles.java b/tez-api/src/main/java/org/apache/tez/common/counters/ResourceBundles.java
new file mode 100644
index 0000000..8113cab
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/common/counters/ResourceBundles.java
@@ -0,0 +1,94 @@
+/*
+ * 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.common.counters;
+
+import java.util.Locale;
+import java.util.ResourceBundle;
+import java.util.MissingResourceException;
+
+/**
+ * Helper class to handle resource bundles in a saner way
+ */
+public class ResourceBundles {
+
+  /**
+   * Get a resource bundle
+   * @param bundleName of the resource
+   * @return the resource bundle
+   * @throws MissingResourceException
+   */
+  public static ResourceBundle getBundle(String bundleName) {
+    return ResourceBundle.getBundle(bundleName.replace('$', '_'),
+        Locale.getDefault(), Thread.currentThread().getContextClassLoader());
+  }
+
+  /**
+   * Get a resource given bundle name and key
+   * @param <T> type of the resource
+   * @param bundleName name of the resource bundle
+   * @param key to lookup the resource
+   * @param suffix for the key to lookup
+   * @param defaultValue of the resource
+   * @return the resource or the defaultValue
+   * @throws ClassCastException if the resource found doesn't match T
+   */
+  @SuppressWarnings("unchecked")
+  public static synchronized <T> T getValue(String bundleName, String key,
+                                            String suffix, T defaultValue) {
+    T value;
+    try {
+      ResourceBundle bundle = getBundle(bundleName);
+      value = (T) bundle.getObject(getLookupKey(key, suffix));
+      if (value != null) {
+        return value;
+      }
+    }
+    catch (Exception e) {
+      // Ignore
+    }
+    return defaultValue;
+  }
+
+  private static String getLookupKey(String key, String suffix) {
+    if (suffix == null || suffix.isEmpty()) return key;
+    return key + suffix;
+  }
+
+  /**
+   * Get the counter group display name
+   * @param group the group name to lookup
+   * @param defaultValue of the group
+   * @return the group display name
+   */
+  public static String getCounterGroupName(String group, String defaultValue) {
+    return getValue(group, "CounterGroupName", "", defaultValue);
+  }
+
+  /**
+   * Get the counter display name
+   * @param group the counter group name for the counter
+   * @param counter the counter name to lookup
+   * @param defaultValue of the counter
+   * @return the counter display name
+   */
+  public static String getCounterName(String group, String counter,
+                                      String defaultValue) {
+    return getValue(group, counter, ".name", defaultValue);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/src/main/java/org/apache/tez/common/counters/TaskCounter.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/common/counters/TaskCounter.java b/tez-api/src/main/java/org/apache/tez/common/counters/TaskCounter.java
new file mode 100644
index 0000000..b6fca27
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/common/counters/TaskCounter.java
@@ -0,0 +1,66 @@
+/**
+ * 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.common.counters;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+// TODO TEZAM5 For MR compatibility, a conversion from tez.TaskCounters to
+// mapreduce.TaskCounters will likely be required somewhere.
+// Similarly for FileSystemCounters and others.
+
+// Counters used by Task classes
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public enum TaskCounter {
+  // TODO Eventually, rename counters to be non-MR specific and map them to MR equivalent.
+  MAP_INPUT_RECORDS, 
+  MAP_OUTPUT_RECORDS,
+  MAP_SKIPPED_RECORDS,
+  MAP_OUTPUT_BYTES,
+  MAP_OUTPUT_MATERIALIZED_BYTES,
+  SPLIT_RAW_BYTES,
+  COMBINE_INPUT_RECORDS,
+  COMBINE_OUTPUT_RECORDS,
+  REDUCE_INPUT_GROUPS,
+  REDUCE_SHUFFLE_BYTES,
+  REDUCE_INPUT_RECORDS,
+  REDUCE_OUTPUT_RECORDS,
+  REDUCE_SKIPPED_GROUPS,
+  REDUCE_SKIPPED_RECORDS,
+  SPILLED_RECORDS,
+  SHUFFLED_MAPS, 
+  FAILED_SHUFFLE,
+  MERGED_MAP_OUTPUTS,
+  GC_TIME_MILLIS,
+  CPU_MILLISECONDS,
+  PHYSICAL_MEMORY_BYTES,
+  VIRTUAL_MEMORY_BYTES,
+  COMMITTED_HEAP_BYTES,
+  
+  INPUT_RECORDS, 
+  OUTPUT_RECORDS,
+  SKIPPED_RECORDS,
+  OUTPUT_BYTES,
+  OUTPUT_MATERIALIZED_BYTES,
+  INPUT_GROUPS,
+  SHUFFLE_BYTES,
+  SHUFFLED_TASKS, 
+  MERGED_TASK_OUTPUTS,
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/src/main/java/org/apache/tez/common/counters/TezCounter.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/common/counters/TezCounter.java b/tez-api/src/main/java/org/apache/tez/common/counters/TezCounter.java
new file mode 100644
index 0000000..394c820
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/common/counters/TezCounter.java
@@ -0,0 +1,83 @@
+/**
+ * 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.common.counters;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.io.Writable;
+
+/**
+ * A named counter that tracks the progress of a map/reduce job.
+ *
+ * <p><code>Counters</code> represent global counters, defined either by the
+ * Map-Reduce framework or applications. Each <code>Counter</code> is named by
+ * an {@link Enum} and has a long for the value.</p>
+ *
+ * <p><code>Counters</code> are bunched into Groups, each comprising of
+ * counters from a particular <code>Enum</code> class.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public interface TezCounter extends Writable {
+
+  /**
+   * Set the display name of the counter
+   * @param displayName of the counter
+   * @deprecated (and no-op by default)
+   */
+  @Deprecated
+  void setDisplayName(String displayName);
+
+  /**
+   * @return the name of the counter
+   */
+  String getName();
+
+  /**
+   * Get the display name of the counter.
+   * @return the user facing name of the counter
+   */
+  String getDisplayName();
+
+  /**
+   * What is the current value of this counter?
+   * @return the current value
+   */
+  long getValue();
+
+  /**
+   * Set this counter by the given value
+   * @param value the value to set
+   */
+  void setValue(long value);
+
+  /**
+   * Increment this counter by the given value
+   * @param incr the value to increase this counter by
+   */
+  void increment(long incr);
+ 
+  /**
+   * Return the underlying object if this is a facade.
+   * @return the undelying object.
+   */
+  @Private
+  TezCounter getUnderlyingCounter();
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/src/main/java/org/apache/tez/common/counters/TezCounters.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/common/counters/TezCounters.java b/tez-api/src/main/java/org/apache/tez/common/counters/TezCounters.java
new file mode 100644
index 0000000..1c9521a
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/common/counters/TezCounters.java
@@ -0,0 +1,144 @@
+/**
+ * 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.common.counters;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * <p><code>Counters</code> holds per job/task counters, defined either by the
+ * Map-Reduce framework or applications. Each <code>Counter</code> can be of
+ * any {@link Enum} type.</p>
+ *
+ * <p><code>Counters</code> are bunched into {@link CounterGroup}s, each
+ * comprising of counters from a particular <code>Enum</code> class.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public class TezCounters extends AbstractCounters<TezCounter, CounterGroup> {
+
+  // Mix framework group implementation into CounterGroup interface
+  private static class FrameworkGroupImpl<T extends Enum<T>>
+      extends FrameworkCounterGroup<T, TezCounter> implements CounterGroup {
+
+    FrameworkGroupImpl(Class<T> cls) {
+      super(cls);
+    }
+
+    @Override
+    protected FrameworkCounter<T> newCounter(T key) {
+      return new FrameworkCounter<T>(key, getName());
+    }
+
+    @Override
+    public CounterGroupBase<TezCounter> getUnderlyingGroup() {
+      return this;
+    }
+  }
+
+  // Mix generic group implementation into CounterGroup interface
+  // and provide some mandatory group factory methods.
+  private static class GenericGroup extends AbstractCounterGroup<TezCounter>
+      implements CounterGroup {
+
+    GenericGroup(String name, String displayName, Limits limits) {
+      super(name, displayName, limits);
+    }
+
+    @Override
+    protected TezCounter newCounter(String name, String displayName, long value) {
+      return new GenericCounter(name, displayName, value);
+    }
+
+    @Override
+    protected TezCounter newCounter() {
+      return new GenericCounter();
+    }
+
+    @Override
+    public CounterGroupBase<TezCounter> getUnderlyingGroup() {
+      return this;
+    }
+  }
+
+  // Mix file system group implementation into the CounterGroup interface
+  private static class FileSystemGroup extends FileSystemCounterGroup<TezCounter>
+      implements CounterGroup {
+
+    @Override
+    protected TezCounter newCounter(String scheme, FileSystemCounter key) {
+      return new FSCounter(scheme, key);
+    }
+
+    @Override
+    public CounterGroupBase<TezCounter> getUnderlyingGroup() {
+      return this;
+    }
+  }
+
+  /**
+   * Provide factory methods for counter group factory implementation.
+   * See also the GroupFactory in
+   *  {@link org.apache.hadoop.TezCounters.Counters mapred.Counters}
+   */
+  private static class GroupFactory
+      extends CounterGroupFactory<TezCounter, CounterGroup> {
+
+    @Override
+    protected <T extends Enum<T>>
+    FrameworkGroupFactory<CounterGroup>
+        newFrameworkGroupFactory(final Class<T> cls) {
+      return new FrameworkGroupFactory<CounterGroup>() {
+        @Override public CounterGroup newGroup(String name) {
+          return new FrameworkGroupImpl<T>(cls); // impl in this package
+        }
+      };
+    }
+
+    @Override
+    protected CounterGroup newGenericGroup(String name, String displayName,
+                                           Limits limits) {
+      return new GenericGroup(name, displayName, limits);
+    }
+
+    @Override
+    protected CounterGroup newFileSystemGroup() {
+      return new FileSystemGroup();
+    }
+  }
+
+  private static final GroupFactory groupFactory = new GroupFactory();
+
+  /**
+   * Default constructor
+   */
+  public TezCounters() {
+    super(groupFactory);
+  }
+
+  /**
+   * Construct the Counters object from the another counters object
+   * @param <C> the type of counter
+   * @param <G> the type of counter group
+   * @param counters the old counters object
+   */
+  public <C extends TezCounter, G extends CounterGroupBase<C>>
+  TezCounters(AbstractCounters<C, G> counters) {
+    super(counters, groupFactory);
+  }
+}


[49/50] [abbrv] git commit: TEZ-431. Implement fault tolerance, retries and event flow for dealing with failed inputs (bikas)

Posted by ss...@apache.org.
TEZ-431. Implement fault tolerance, retries and event flow for dealing with failed inputs (bikas)


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

Branch: refs/heads/master
Commit: 3749a18fafad2f0ebe661ff8979d8c6f794e295f
Parents: b212ca1
Author: Bikas Saha <bi...@apache.org>
Authored: Tue Sep 24 18:42:03 2013 -0700
Committer: Bikas Saha <bi...@apache.org>
Committed: Tue Sep 24 18:42:03 2013 -0700

----------------------------------------------------------------------
 .../tez/dag/app/dag/DAGTerminationCause.java    |   4 +-
 .../org/apache/tez/dag/app/dag/EdgeManager.java |   5 +-
 .../tez/dag/app/dag/TaskTerminationCause.java   |   7 +-
 .../java/org/apache/tez/dag/app/dag/Vertex.java |   5 -
 .../apache/tez/dag/app/dag/VertexScheduler.java |   4 +-
 .../tez/dag/app/dag/VertexTerminationCause.java |   4 +-
 .../tez/dag/app/dag/event/DAGEventType.java     |   1 +
 .../app/dag/event/DAGEventVertexReRunning.java  |  37 +++
 .../dag/event/TaskAttemptEventNodeFailed.java   |   4 +-
 .../dag/event/TaskAttemptEventOutputFailed.java |  44 ++++
 .../dag/event/TaskAttemptEventStatusUpdate.java |   3 +-
 .../dag/app/dag/event/TaskAttemptEventType.java |   5 +-
 .../VertexEventSourceTaskAttemptCompleted.java  |   7 +-
 .../event/VertexEventTaskAttemptCompleted.java  |  25 +-
 .../VertexEventTaskAttemptFetchFailure.java     |  46 ----
 .../tez/dag/app/dag/event/VertexEventType.java  |   5 +-
 .../dag/app/dag/impl/BroadcastEdgeManager.java  |  15 +-
 .../apache/tez/dag/app/dag/impl/DAGImpl.java    |  93 +++++---
 .../org/apache/tez/dag/app/dag/impl/Edge.java   |  42 ++--
 .../dag/impl/ImmediateStartVertexScheduler.java |   4 +-
 .../dag/app/dag/impl/OneToOneEdgeManager.java   |  12 +-
 .../app/dag/impl/ScatterGatherEdgeManager.java  |  29 +--
 .../dag/app/dag/impl/ShuffleVertexManager.java  |  29 ++-
 .../tez/dag/app/dag/impl/TaskAttemptImpl.java   | 109 ++++++---
 .../apache/tez/dag/app/dag/impl/TaskImpl.java   |  78 +++----
 .../apache/tez/dag/app/dag/impl/VertexImpl.java | 201 ++++++----------
 .../TezDependentTaskCompletionEvent.java        | 228 -------------------
 ...TezTaskDependencyCompletionEventsUpdate.java |  64 ------
 .../tez/dag/app/dag/impl/TestTaskAttempt.java   |  20 +-
 .../tez/dag/app/dag/impl/TestVertexImpl.java    |  42 +---
 .../dag/app/dag/impl/TestVertexScheduler.java   |  67 +++---
 31 files changed, 485 insertions(+), 754 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/3749a18f/tez-dag/src/main/java/org/apache/tez/dag/app/dag/DAGTerminationCause.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/DAGTerminationCause.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/DAGTerminationCause.java
index 05f15f3..3b097eb 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/DAGTerminationCause.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/DAGTerminationCause.java
@@ -34,5 +34,7 @@ public enum DAGTerminationCause {
   ZERO_VERTICES, 
   
   /** DAG failed during init. */
-  INIT_FAILURE,   
+  INIT_FAILURE,
+  
+  INTERNAL_ERROR
 }

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/3749a18f/tez-dag/src/main/java/org/apache/tez/dag/app/dag/EdgeManager.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/EdgeManager.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/EdgeManager.java
index 674d18e..86d155f 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/EdgeManager.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/EdgeManager.java
@@ -26,10 +26,10 @@ import org.apache.tez.runtime.api.events.InputReadErrorEvent;
 
 public abstract class EdgeManager {
   
-  public abstract int getNumDestinationTaskInputs(Vertex sourceVertex,
+  public abstract int getNumDestinationTaskInputs(int numSourceTasks, 
       int destinationTaskIndex);
 
-  public abstract int getNumSourceTaskOutputs(Vertex destinationVertex,
+  public abstract int getNumSourceTaskOutputs(int numDestinationTasks, 
       int sourceTaskIndex);
   
   /**
@@ -41,6 +41,7 @@ public abstract class EdgeManager {
   public abstract void routeEventToDestinationTasks(InputFailedEvent event,
       int sourceTaskIndex, int numDestinationTasks, List<Integer> taskIndices);
 
+  public abstract int getDestinationConsumerTaskNumber(int sourceTaskIndex, int numDestTasks);
   
   /**
    * Return the source task index to which to send the event

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/3749a18f/tez-dag/src/main/java/org/apache/tez/dag/app/dag/TaskTerminationCause.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/TaskTerminationCause.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/TaskTerminationCause.java
index 73741f0..6736d2a 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/TaskTerminationCause.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/TaskTerminationCause.java
@@ -18,8 +18,6 @@
 
 package org.apache.tez.dag.app.dag;
 
-import org.apache.tez.dag.app.dag.event.TaskEventType;
-
 /**
  * Represents proximate cause of a Task transition to FAILED or KILLED.
  */
@@ -31,6 +29,9 @@ public enum TaskTerminationCause {
   /** Other vertex failed causing DAG to fail thus killing the parent vertex  */
   OTHER_VERTEX_FAILURE,
   
-  /** One of the tasks for the parent vertex failed.  */
+  /** One of the tasks for the source/destination vertex failed.  */
   OTHER_TASK_FAILURE, 
+  
+  /** One of the tasks of the destination vertex failed. */
+  OWN_TASK_FAILURE
 }

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/3749a18f/tez-dag/src/main/java/org/apache/tez/dag/app/dag/Vertex.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/Vertex.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/Vertex.java
index 76964a3..adaa27f 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/Vertex.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/Vertex.java
@@ -29,12 +29,10 @@ import org.apache.tez.dag.api.records.DAGProtos.VertexPlan;
 import org.apache.tez.dag.api.client.ProgressBuilder;
 import org.apache.tez.dag.api.client.VertexStatusBuilder;
 import org.apache.tez.dag.app.dag.impl.Edge;
-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.impl.InputSpec;
 import org.apache.tez.runtime.api.impl.OutputSpec;
-import org.apache.tez.runtime.records.TezDependentTaskCompletionEvent;
 
 
 /**
@@ -68,9 +66,6 @@ public interface Vertex extends Comparable<Vertex> {
 
   void setParallelism(int parallelism,Map<Vertex, EdgeManager> sourceEdgeManagers);
 
-  TezDependentTaskCompletionEvent[] getTaskAttemptCompletionEvents(
-      TezTaskAttemptID attemptId, int fromEventId, int maxEvents);
-  
   // CHANGE THESE TO LISTS AND MAINTAIN ORDER?
   void setInputVertices(Map<Vertex, Edge> inVertices);
   void setOutputVertices(Map<Vertex, Edge> outVertices);

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/3749a18f/tez-dag/src/main/java/org/apache/tez/dag/app/dag/VertexScheduler.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/VertexScheduler.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/VertexScheduler.java
index 4a1a7a6..3789702 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/VertexScheduler.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/VertexScheduler.java
@@ -20,12 +20,10 @@ package org.apache.tez.dag.app.dag;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.tez.dag.records.TezTaskAttemptID;
-import org.apache.tez.runtime.records.TezDependentTaskCompletionEvent;
 
 // Rename to VertexManager TEZ-364 and move to DAG API. Make abstract class.
 public interface VertexScheduler {
   void initialize(Configuration conf);
   void onVertexStarted();
-  void onSourceTaskCompleted(TezTaskAttemptID attemptId,
-      TezDependentTaskCompletionEvent event);
+  void onSourceTaskCompleted(TezTaskAttemptID attemptId);
 }

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/3749a18f/tez-dag/src/main/java/org/apache/tez/dag/app/dag/VertexTerminationCause.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/VertexTerminationCause.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/VertexTerminationCause.java
index 138ee70..f675ace 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/VertexTerminationCause.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/VertexTerminationCause.java
@@ -39,5 +39,7 @@ public enum VertexTerminationCause {
   ZERO_TASKS, 
 
   /** This vertex failed during init. */
-  INIT_FAILURE
+  INIT_FAILURE,
+  
+  INTERNAL_ERROR
 }

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/3749a18f/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/DAGEventType.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/DAGEventType.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/DAGEventType.java
index 14c2f30..476c688 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/DAGEventType.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/DAGEventType.java
@@ -32,6 +32,7 @@ public enum DAGEventType {
 
   //Producer: Vertex
   DAG_VERTEX_COMPLETED,
+  DAG_VERTEX_RERUNNING,
 
   //Producer: TaskImpl
   DAG_SCHEDULER_UPDATE,

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/3749a18f/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/DAGEventVertexReRunning.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/DAGEventVertexReRunning.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/DAGEventVertexReRunning.java
new file mode 100644
index 0000000..303d48d
--- /dev/null
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/DAGEventVertexReRunning.java
@@ -0,0 +1,37 @@
+/**
+* Licensed to the Apache Software Foundation (ASF) under one
+* or more contributor license agreements.  See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership.  The ASF licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License.  You may obtain a copy of the License at
+*
+*     http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package org.apache.tez.dag.app.dag.event;
+
+import org.apache.tez.dag.records.TezVertexID;
+
+public class DAGEventVertexReRunning extends DAGEvent {
+
+  private TezVertexID vertexId;
+
+  public DAGEventVertexReRunning(TezVertexID vertexId) {
+    super(vertexId.getDAGId(), DAGEventType.DAG_VERTEX_RERUNNING);
+    this.vertexId = vertexId;
+  }
+
+  public TezVertexID getVertexId() {
+    return vertexId;
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/3749a18f/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskAttemptEventNodeFailed.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskAttemptEventNodeFailed.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskAttemptEventNodeFailed.java
index ee143bb..6d97466 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskAttemptEventNodeFailed.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskAttemptEventNodeFailed.java
@@ -19,7 +19,8 @@ package org.apache.tez.dag.app.dag.event;
 
 import org.apache.tez.dag.records.TezTaskAttemptID;
 
-public class TaskAttemptEventNodeFailed extends TaskAttemptEvent {
+public class TaskAttemptEventNodeFailed extends TaskAttemptEvent 
+  implements DiagnosableEvent{
 
   private final String message;
 
@@ -29,6 +30,7 @@ public class TaskAttemptEventNodeFailed extends TaskAttemptEvent {
     this.message = diagMessage;
   }
 
+  @Override
   public String getDiagnosticInfo() {
     return this.message;
   }

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/3749a18f/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskAttemptEventOutputFailed.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskAttemptEventOutputFailed.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskAttemptEventOutputFailed.java
new file mode 100644
index 0000000..678e1e7
--- /dev/null
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskAttemptEventOutputFailed.java
@@ -0,0 +1,44 @@
+/**
+* Licensed to the Apache Software Foundation (ASF) under one
+* or more contributor license agreements.  See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership.  The ASF licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License.  You may obtain a copy of the License at
+*
+*     http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package org.apache.tez.dag.app.dag.event;
+
+import org.apache.tez.dag.records.TezTaskAttemptID;
+import org.apache.tez.runtime.api.impl.TezEvent;
+
+public class TaskAttemptEventOutputFailed extends TaskAttemptEvent {
+  
+  private TezEvent inputFailedEvent;
+  private int consumerTaskNumber;
+  
+  public TaskAttemptEventOutputFailed(TezTaskAttemptID attemptId,
+      TezEvent tezEvent, int numConsumers) {
+    super(attemptId, TaskAttemptEventType.TA_OUTPUT_FAILED);
+    this.inputFailedEvent = tezEvent;
+    this.consumerTaskNumber = numConsumers;
+  }
+  
+  public TezEvent getInputFailedEvent() {
+    return inputFailedEvent;
+  }
+  
+  public int getConsumerTaskNumber() {
+    return consumerTaskNumber;
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/3749a18f/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskAttemptEventStatusUpdate.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskAttemptEventStatusUpdate.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskAttemptEventStatusUpdate.java
index 30aefde..13577c5 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskAttemptEventStatusUpdate.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskAttemptEventStatusUpdate.java
@@ -31,7 +31,8 @@ public class TaskAttemptEventStatusUpdate extends TaskAttemptEvent {
   
   private TaskStatusUpdateEvent taskAttemptStatus;
   
-  public TaskAttemptEventStatusUpdate(TezTaskAttemptID id, TaskStatusUpdateEvent statusEvent) {
+  public TaskAttemptEventStatusUpdate(TezTaskAttemptID id,
+      TaskStatusUpdateEvent statusEvent) {
     super(id, TaskAttemptEventType.TA_STATUS_UPDATE);
     this.taskAttemptStatus = statusEvent;
   }

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/3749a18f/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskAttemptEventType.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskAttemptEventType.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskAttemptEventType.java
index 16e4e3f..5210e33 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskAttemptEventType.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskAttemptEventType.java
@@ -55,6 +55,7 @@ public enum TaskAttemptEventType {
   // The node running the task attempt failed.
   TA_NODE_FAILED,
   
-//Producer: Job
-  TA_TOO_MANY_FETCH_FAILURES,
+  // Producer: consumer destination vertex
+  TA_OUTPUT_FAILED,
+  
 }

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/3749a18f/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexEventSourceTaskAttemptCompleted.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexEventSourceTaskAttemptCompleted.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexEventSourceTaskAttemptCompleted.java
index 388beba..6cd38a9 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexEventSourceTaskAttemptCompleted.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexEventSourceTaskAttemptCompleted.java
@@ -19,21 +19,20 @@
 package org.apache.tez.dag.app.dag.event;
 
 import org.apache.tez.dag.records.TezVertexID;
-import org.apache.tez.runtime.records.TezDependentTaskCompletionEvent;
 
 public class VertexEventSourceTaskAttemptCompleted extends VertexEvent {
 
-  private TezDependentTaskCompletionEvent completionEvent;
+  private VertexEventTaskAttemptCompleted completionEvent;
 
   public VertexEventSourceTaskAttemptCompleted(
       TezVertexID targetVertexId,
-      TezDependentTaskCompletionEvent completionEvent) {
+      VertexEventTaskAttemptCompleted completionEvent) {
     super(targetVertexId, 
         VertexEventType.V_SOURCE_TASK_ATTEMPT_COMPLETED);
     this.completionEvent = completionEvent;
   }
 
-  public TezDependentTaskCompletionEvent getCompletionEvent() {
+  public VertexEventTaskAttemptCompleted getCompletionEvent() {
     return completionEvent;
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/3749a18f/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexEventTaskAttemptCompleted.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexEventTaskAttemptCompleted.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexEventTaskAttemptCompleted.java
index c9d3f7d..5b07674 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexEventTaskAttemptCompleted.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexEventTaskAttemptCompleted.java
@@ -18,20 +18,27 @@
 
 package org.apache.tez.dag.app.dag.event;
 
-import org.apache.tez.runtime.records.TezDependentTaskCompletionEvent;
+import org.apache.tez.dag.app.dag.TaskAttemptStateInternal;
+import org.apache.tez.dag.records.TezTaskAttemptID;
 
 public class VertexEventTaskAttemptCompleted extends VertexEvent {
 
-  private TezDependentTaskCompletionEvent completionEvent;
-
-  public VertexEventTaskAttemptCompleted(
-      TezDependentTaskCompletionEvent completionEvent) {
-    super(completionEvent.getTaskAttemptID().getTaskID().getVertexID(), 
+  private TezTaskAttemptID attemptId;
+  private TaskAttemptStateInternal attempState;
+  
+  public VertexEventTaskAttemptCompleted(TezTaskAttemptID taskAttemptId,
+      TaskAttemptStateInternal state) {
+    super(taskAttemptId.getTaskID().getVertexID(), 
         VertexEventType.V_TASK_ATTEMPT_COMPLETED);
-    this.completionEvent = completionEvent;
+    this.attemptId = taskAttemptId;
+    this.attempState = state;
   }
 
-  public TezDependentTaskCompletionEvent getCompletionEvent() {
-    return completionEvent;
+  public TezTaskAttemptID getTaskAttemptId() {
+    return attemptId;
+  }
+  
+  public TaskAttemptStateInternal getTaskAttemptState() {
+    return attempState;
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/3749a18f/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexEventTaskAttemptFetchFailure.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexEventTaskAttemptFetchFailure.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexEventTaskAttemptFetchFailure.java
deleted file mode 100644
index 5b2b955..0000000
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexEventTaskAttemptFetchFailure.java
+++ /dev/null
@@ -1,46 +0,0 @@
-/**
-* Licensed to the Apache Software Foundation (ASF) under one
-* or more contributor license agreements.  See the NOTICE file
-* distributed with this work for additional information
-* regarding copyright ownership.  The ASF licenses this file
-* to you under the Apache License, Version 2.0 (the
-* "License"); you may not use this file except in compliance
-* with the License.  You may obtain a copy of the License at
-*
-*     http://www.apache.org/licenses/LICENSE-2.0
-*
-* Unless required by applicable law or agreed to in writing, software
-* distributed under the License is distributed on an "AS IS" BASIS,
-* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-* See the License for the specific language governing permissions and
-* limitations under the License.
-*/
-package org.apache.tez.dag.app.dag.event;
-
-import java.util.List;
-
-import org.apache.tez.dag.records.TezTaskAttemptID;
-
-public class VertexEventTaskAttemptFetchFailure extends VertexEvent {
-
-  private final TezTaskAttemptID target;
-  private final List<TezTaskAttemptID> sources;
-
-  public VertexEventTaskAttemptFetchFailure(TezTaskAttemptID reduce, 
-      List<TezTaskAttemptID> maps) {
-    super(
-        reduce.getTaskID().getVertexID(), 
-        VertexEventType.V_TASK_ATTEMPT_FETCH_FAILURE);
-    this.target = reduce;
-    this.sources = maps;
-  }
-
-  public List<TezTaskAttemptID> getSources() {
-    return sources;
-  }
-
-  public TezTaskAttemptID getTarget() {
-    return target;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/3749a18f/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexEventType.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexEventType.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexEventType.java
index dc7e2dd..7d640af 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexEventType.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexEventType.java
@@ -49,12 +49,9 @@ public enum VertexEventType {
   V_TASK_SUCCEEDED,
   V_ATTEMPT_KILLED,
   
-  //Producer:TaskAttemptListener
-  V_TASK_ATTEMPT_FETCH_FAILURE,
-
   //Producer:Any component
   V_DIAGNOSTIC_UPDATE,
-  INTERNAL_ERROR,
+  V_INTERNAL_ERROR,
   V_COUNTER_UPDATE,
   
   V_ROUTE_EVENT,

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/3749a18f/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/BroadcastEdgeManager.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/BroadcastEdgeManager.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/BroadcastEdgeManager.java
index 55a2c86..21562e7 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/BroadcastEdgeManager.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/BroadcastEdgeManager.java
@@ -21,21 +21,20 @@ package org.apache.tez.dag.app.dag.impl;
 import java.util.List;
 
 import org.apache.tez.dag.app.dag.EdgeManager;
-import org.apache.tez.dag.app.dag.Vertex;
 import org.apache.tez.runtime.api.events.DataMovementEvent;
-import org.apache.tez.runtime.api.events.InputFailedEvent;
 import org.apache.tez.runtime.api.events.InputReadErrorEvent;
+import org.apache.tez.runtime.api.events.InputFailedEvent;
 
 public class BroadcastEdgeManager extends EdgeManager {
 
   @Override
-  public int getNumDestinationTaskInputs(Vertex sourceVertex,
+  public int getNumDestinationTaskInputs(int numSourceTasks, 
       int destinationTaskIndex) {
-    return sourceVertex.getTotalTasks();
+    return numSourceTasks;
   }
   
   @Override
-  public int getNumSourceTaskOutputs(Vertex destinationVertex,
+  public int getNumSourceTaskOutputs(int numDestinationTasks,
       int sourceTaskIndex) {
     return 1;
   }
@@ -66,4 +65,10 @@ public class BroadcastEdgeManager extends EdgeManager {
     }    
   }
 
+  @Override
+  public int getDestinationConsumerTaskNumber(int sourceTaskIndex,
+      int numDestTasks) {
+    return numDestTasks;
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/3749a18f/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 ee12221..fbf5e9d 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
@@ -79,6 +79,7 @@ import org.apache.tez.dag.app.dag.event.DAGEventSchedulerUpdateTAAssigned;
 import org.apache.tez.dag.app.dag.event.DAGEventType;
 import org.apache.tez.dag.app.dag.event.DAGEventVertexCompleted;
 import org.apache.tez.dag.app.dag.event.DAGAppMasterEventDAGFinished;
+import org.apache.tez.dag.app.dag.event.DAGEventVertexReRunning;
 import org.apache.tez.dag.app.dag.event.VertexEvent;
 import org.apache.tez.dag.app.dag.event.VertexEventType;
 import org.apache.tez.dag.app.dag.event.VertexEventTermination;
@@ -196,6 +197,9 @@ public class DAGImpl implements org.apache.tez.dag.app.dag.DAG,
               EnumSet.of(DAGState.RUNNING, DAGState.SUCCEEDED, DAGState.TERMINATING,DAGState.FAILED),
               DAGEventType.DAG_VERTEX_COMPLETED,
               new VertexCompletedTransition())
+          .addTransition(DAGState.RUNNING, DAGState.RUNNING,
+              DAGEventType.DAG_VERTEX_RERUNNING,
+              new VertexReRunningTransition())
           .addTransition(DAGState.RUNNING, DAGState.TERMINATING,
               DAGEventType.DAG_KILL, new DAGKilledTransition())
           .addTransition(DAGState.RUNNING, DAGState.RUNNING,
@@ -230,6 +234,7 @@ public class DAGImpl implements org.apache.tez.dag.app.dag.DAG,
               // Ignore-able events
           .addTransition(DAGState.TERMINATING, DAGState.TERMINATING,
               EnumSet.of(DAGEventType.DAG_KILL,
+                         DAGEventType.DAG_VERTEX_RERUNNING,
                          DAGEventType.DAG_SCHEDULER_UPDATE))
 
           // Transitions from SUCCEEDED state
@@ -260,6 +265,7 @@ public class DAGImpl implements org.apache.tez.dag.app.dag.DAG,
           // Ignore-able events
           .addTransition(DAGState.FAILED, DAGState.FAILED,
               EnumSet.of(DAGEventType.DAG_KILL,
+                  DAGEventType.DAG_VERTEX_RERUNNING,
                   DAGEventType.DAG_VERTEX_COMPLETED))
 
           // Transitions from KILLED state
@@ -276,6 +282,7 @@ public class DAGImpl implements org.apache.tez.dag.app.dag.DAG,
           .addTransition(DAGState.KILLED, DAGState.KILLED,
               EnumSet.of(DAGEventType.DAG_KILL,
                   DAGEventType.DAG_START,
+                  DAGEventType.DAG_VERTEX_RERUNNING,
                   DAGEventType.DAG_SCHEDULER_UPDATE,
                   DAGEventType.DAG_VERTEX_COMPLETED))
 
@@ -1117,27 +1124,25 @@ public class DAGImpl implements org.apache.tez.dag.app.dag.DAG,
       Vertex vertex = job.vertices.get(vertexEvent.getVertexId());
       job.numCompletedVertices++;
       if (vertexEvent.getVertexState() == VertexState.SUCCEEDED) {
-        vertexSucceeded(job, vertex);
+        job.vertexSucceeded(vertex);
         job.dagScheduler.vertexCompleted(vertex);
       }
       else if (vertexEvent.getVertexState() == VertexState.FAILED) {
         job.enactKill(DAGTerminationCause.VERTEX_FAILURE, VertexTerminationCause.OTHER_VERTEX_FAILURE);
-        vertexFailed(job, vertex);
+        job.vertexFailed(vertex);
         forceTransitionToKillWait = true;
       }
       else if (vertexEvent.getVertexState() == VertexState.KILLED) {
-        vertexKilled(job, vertex);
+        job.vertexKilled(vertex);
         forceTransitionToKillWait = true;
       }
 
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Vertex completed."
-            + ", numCompletedVertices=" + job.numCompletedVertices
-            + ", numSuccessfulVertices=" + job.numSuccessfulVertices
-            + ", numFailedVertices=" + job.numFailedVertices
-            + ", numKilledVertices=" + job.numKilledVertices
-            + ", numVertices=" + job.numVertices);
-      }
+      LOG.info("Vertex " + vertex.getVertexId() + " completed."
+          + ", numCompletedVertices=" + job.numCompletedVertices
+          + ", numSuccessfulVertices=" + job.numSuccessfulVertices
+          + ", numFailedVertices=" + job.numFailedVertices
+          + ", numKilledVertices=" + job.numKilledVertices
+          + ", numVertices=" + job.numVertices);
 
       // if the job has not finished but a failure/kill occurred, then force the transition to KILL_WAIT.
       DAGState state = checkJobForCompletion(job);
@@ -1149,34 +1154,58 @@ public class DAGImpl implements org.apache.tez.dag.app.dag.DAG,
       }
     }
 
-    private void vertexSucceeded(DAGImpl job, Vertex vertex) {
-      job.numSuccessfulVertices++;
-      // TODO: Metrics
-      //job.metrics.completedTask(task);
+  }
+  
+  private static class VertexReRunningTransition implements
+      SingleArcTransition<DAGImpl, DAGEvent> {
+    @Override
+    public void transition(DAGImpl job, DAGEvent event) {
+      DAGEventVertexReRunning vertexEvent = (DAGEventVertexReRunning) event;
+      Vertex vertex = job.vertices.get(vertexEvent.getVertexId());
+      job.numCompletedVertices--;
+      job.vertexReRunning(vertex);
+      
+
+      LOG.info("Vertex " + vertex.getVertexId() + " re-running."
+          + ", numCompletedVertices=" + job.numCompletedVertices
+          + ", numSuccessfulVertices=" + job.numSuccessfulVertices
+          + ", numFailedVertices=" + job.numFailedVertices
+          + ", numKilledVertices=" + job.numKilledVertices
+          + ", numVertices=" + job.numVertices);
     }
+  }
+  
+  private void vertexSucceeded(Vertex vertex) {
+    numSuccessfulVertices++;
+    // TODO: Metrics
+    //job.metrics.completedTask(task);
+  }
+  
+  private void vertexReRunning(Vertex vertex) {
+    numSuccessfulVertices--;
+    addDiagnostic("Vertex re-running " + vertex.getVertexId());
+    // TODO: Metrics
+    //job.metrics.completedTask(task);
+  }
 
-    private void vertexFailed(DAGImpl job, Vertex vertex) {
-      job.numFailedVertices++;
-      job.addDiagnostic("Vertex failed " + vertex.getVertexId());
-      // TODO: Metrics
-      //job.metrics.failedTask(task);
-    }
+  private void vertexFailed(Vertex vertex) {
+    numFailedVertices++;
+    addDiagnostic("Vertex failed " + vertex.getVertexId());
+    // TODO: Metrics
+    //job.metrics.failedTask(task);
+  }
 
-    private void vertexKilled(DAGImpl job, Vertex vertex) {
-      job.numKilledVertices++;
-      job.addDiagnostic("Vertex killed " + vertex.getVertexId());
-      // TODO: Metrics
-      //job.metrics.killedTask(task);
-    }
+  private void vertexKilled(Vertex vertex) {
+    numKilledVertices++;
+    addDiagnostic("Vertex killed " + vertex.getVertexId());
+    // TODO: Metrics
+    //job.metrics.killedTask(task);
   }
 
   private void addDiagnostic(String diag) {
     diagnostics.add(diag);
   }
 
-
-
-
   private static class DiagnosticsUpdateTransition implements
       SingleArcTransition<DAGImpl, DAGEvent> {
     @Override
@@ -1228,6 +1257,10 @@ public class DAGImpl implements org.apache.tez.dag.app.dag.DAG,
     @Override
     public void transition(DAGImpl job, DAGEvent event) {
       //TODO Is this JH event required.
+      LOG.info(job.getID() + " terminating due to internal error");
+      // terminate all vertices
+      job.enactKill(DAGTerminationCause.INTERNAL_ERROR,
+          VertexTerminationCause.INTERNAL_ERROR);
       job.setFinishTime();
       job.logJobHistoryUnsuccesfulEvent(DAGStatus.State.FAILED);
       job.finished(DAGState.ERROR);

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/3749a18f/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/Edge.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/Edge.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/Edge.java
index aaca662..bcdb4af 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/Edge.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/Edge.java
@@ -28,6 +28,7 @@ import org.apache.tez.dag.api.TezUncheckedException;
 import org.apache.tez.dag.app.dag.EdgeManager;
 import org.apache.tez.dag.app.dag.Task;
 import org.apache.tez.dag.app.dag.Vertex;
+import org.apache.tez.dag.app.dag.event.TaskAttemptEventOutputFailed;
 import org.apache.tez.dag.app.dag.event.TaskEventAddTezEvent;
 import org.apache.tez.dag.records.TezTaskAttemptID;
 import org.apache.tez.dag.records.TezTaskID;
@@ -97,23 +98,25 @@ public class Edge {
   }
 
   public void setDestinationVertex(Vertex destinationVertex) {
-    if (this.destinationVertex != null && this.destinationVertex != destinationVertex) {
+    if (this.destinationVertex != null
+        && this.destinationVertex != destinationVertex) {
       throw new TezUncheckedException("Destination vertex exists: "
           + destinationVertex.getName());
     }
     this.destinationVertex = destinationVertex;
   }
-  
+
   public InputSpec getDestinationSpec(int destinationTaskIndex) {
     return new InputSpec(sourceVertex.getName(),
         edgeProperty.getEdgeDestination(),
-        edgeManager.getNumDestinationTaskInputs(sourceVertex, destinationTaskIndex));
- }
-  
+        edgeManager.getNumDestinationTaskInputs(sourceVertex.getTotalTasks(),
+            destinationTaskIndex));
+  }
+
   public OutputSpec getSourceSpec(int sourceTaskIndex) {
     return new OutputSpec(destinationVertex.getName(),
-        edgeProperty.getEdgeSource(), 
-        edgeManager.getNumSourceTaskOutputs(destinationVertex, sourceTaskIndex));
+        edgeProperty.getEdgeSource(), edgeManager.getNumSourceTaskOutputs(
+            destinationVertex.getTotalTasks(), sourceTaskIndex));
   }
   
   public void startEventBuffering() {
@@ -133,17 +136,25 @@ public class Edge {
     sourceEventBuffer.clear();
   }
   
+  @SuppressWarnings("unchecked")
   public void sendTezEventToSourceTasks(TezEvent tezEvent) {
     if (!bufferEvents.get()) {
       switch (tezEvent.getEventType()) {
       case INPUT_READ_ERROR_EVENT:
         InputReadErrorEvent event = (InputReadErrorEvent) tezEvent.getEvent();
-        TezTaskAttemptID destAttemptId = tezEvent.getSourceInfo().getTaskAttemptID();
+        TezTaskAttemptID destAttemptId = tezEvent.getSourceInfo()
+            .getTaskAttemptID();
         int destTaskIndex = destAttemptId.getTaskID().getId();
-        int srcTaskIndex = edgeManager.routeEventToSourceTasks(destTaskIndex, event);
-        // TODO this is BROKEN. TEZ-431
-//        TezTaskID srcTaskId = sourceVertex.getTask(srcTaskIndex).getTaskId();
-//        sendEventToTask(srcTaskId, tezEvent);
+        int srcTaskIndex = edgeManager.routeEventToSourceTasks(destTaskIndex,
+            event);
+        int numConsumers = edgeManager.getDestinationConsumerTaskNumber(
+            srcTaskIndex, destinationVertex.getTotalTasks());
+        TezTaskID srcTaskId = sourceVertex.getTask(srcTaskIndex).getTaskId();
+        int taskAttemptIndex = event.getVersion();
+        TezTaskAttemptID srcTaskAttemptId = new TezTaskAttemptID(srcTaskId,
+            taskAttemptIndex);
+        eventHandler.handle(new TaskAttemptEventOutputFailed(srcTaskAttemptId,
+            tezEvent, numConsumers));
         break;
       default:
         throw new TezUncheckedException("Unhandled tez event type: "
@@ -201,13 +212,6 @@ public class Edge {
     }
   }
   
-  private void sendEventToDestination(List<Integer> destTaskIndeces, TezEvent tezEvent) {
-    for(Integer destTaskIndex : destTaskIndeces) {
-      TezTaskID destTaskId = destinationVertex.getTask(destTaskIndex).getTaskId();
-      sendEventToTask(destTaskId, tezEvent);
-    }
-  }
-  
   @SuppressWarnings("unchecked")
   private void sendEventToTask(TezTaskID taskId, TezEvent tezEvent) {
     eventHandler.handle(new TaskEventAddTezEvent(taskId, tezEvent));

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/3749a18f/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/ImmediateStartVertexScheduler.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/ImmediateStartVertexScheduler.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/ImmediateStartVertexScheduler.java
index a4e5f3b..b79a426 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/ImmediateStartVertexScheduler.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/ImmediateStartVertexScheduler.java
@@ -22,7 +22,6 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.tez.dag.app.dag.Vertex;
 import org.apache.tez.dag.app.dag.VertexScheduler;
 import org.apache.tez.dag.records.TezTaskAttemptID;
-import org.apache.tez.runtime.records.TezDependentTaskCompletionEvent;
 
 /**
  * Starts all tasks immediately on vertex start
@@ -40,8 +39,7 @@ public class ImmediateStartVertexScheduler implements VertexScheduler {
   }
 
   @Override
-  public void onSourceTaskCompleted(TezTaskAttemptID attemptId, 
-      TezDependentTaskCompletionEvent event) {
+  public void onSourceTaskCompleted(TezTaskAttemptID attemptId) {
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/3749a18f/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/OneToOneEdgeManager.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/OneToOneEdgeManager.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/OneToOneEdgeManager.java
index 29abfac..1ec9451 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/OneToOneEdgeManager.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/OneToOneEdgeManager.java
@@ -21,21 +21,20 @@ package org.apache.tez.dag.app.dag.impl;
 import java.util.List;
 
 import org.apache.tez.dag.app.dag.EdgeManager;
-import org.apache.tez.dag.app.dag.Vertex;
 import org.apache.tez.runtime.api.events.DataMovementEvent;
-import org.apache.tez.runtime.api.events.InputFailedEvent;
 import org.apache.tez.runtime.api.events.InputReadErrorEvent;
+import org.apache.tez.runtime.api.events.InputFailedEvent;
 
 public class OneToOneEdgeManager extends EdgeManager {
 
   @Override
-  public int getNumDestinationTaskInputs(Vertex sourceVertex,
+  public int getNumDestinationTaskInputs(int numDestinationTasks, 
       int destinationTaskIndex) {
     return 1;
   }
   
   @Override
-  public int getNumSourceTaskOutputs(Vertex destinationVertex,
+  public int getNumSourceTaskOutputs(int numDestinationTasks, 
       int sourceTaskIndex) {
     return 1;
   }
@@ -63,4 +62,9 @@ public class OneToOneEdgeManager extends EdgeManager {
   void addDestinationTaskIndex(int sourceTaskIndex, List<Integer> taskIndeces) {
     taskIndeces.add(new Integer(sourceTaskIndex));
   }
+
+  @Override
+  public int getDestinationConsumerTaskNumber(int sourceTaskIndex, int numDestTasks) {
+    return 1;
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/3749a18f/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/ScatterGatherEdgeManager.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/ScatterGatherEdgeManager.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/ScatterGatherEdgeManager.java
index 3d1d289..b1dd475 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/ScatterGatherEdgeManager.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/ScatterGatherEdgeManager.java
@@ -21,35 +21,21 @@ package org.apache.tez.dag.app.dag.impl;
 import java.util.List;
 
 import org.apache.tez.dag.app.dag.EdgeManager;
-import org.apache.tez.dag.app.dag.Vertex;
 import org.apache.tez.runtime.api.events.DataMovementEvent;
-import org.apache.tez.runtime.api.events.InputFailedEvent;
 import org.apache.tez.runtime.api.events.InputReadErrorEvent;
+import org.apache.tez.runtime.api.events.InputFailedEvent;
 
 public class ScatterGatherEdgeManager extends EdgeManager {
 
-  private int initialDestinationTaskNumber = -1;
-
   @Override
-  public int getNumDestinationTaskInputs(Vertex sourceVertex,
+  public int getNumDestinationTaskInputs(int numSourceTasks,
       int destinationTaskIndex) {
-    return sourceVertex.getTotalTasks();
+    return numSourceTasks;
   }
   
   @Override
-  public int getNumSourceTaskOutputs(Vertex destinationVertex,
-      int sourceTaskIndex) {
-    if(initialDestinationTaskNumber == -1) {
-      // the downstream vertex may not have started and so its number of tasks
-      // may change. So save this initial count and provide a consistent view 
-      // to all source tasks, including late starters and retries.
-      // When the number of destination tasks change then the routing will have 
-      // to be updated too.
-      // This value may be obtained from config too if destination task initial 
-      // parallelism is not specified.
-      initialDestinationTaskNumber = destinationVertex.getTotalTasks();
-    }
-    return initialDestinationTaskNumber;
+  public int getNumSourceTaskOutputs(int numDestinationTasks, int sourceTaskIndex) {
+    return numDestinationTasks;
   }
 
   @Override
@@ -73,5 +59,10 @@ public class ScatterGatherEdgeManager extends EdgeManager {
       InputReadErrorEvent event) {
     return event.getIndex();
   }
+
+  @Override
+  public int getDestinationConsumerTaskNumber(int sourceTaskIndex, int numDestTasks) {
+    return numDestTasks;
+  }
   
 }

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/3749a18f/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/ShuffleVertexManager.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/ShuffleVertexManager.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/ShuffleVertexManager.java
index a4dd555..e039c72 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/ShuffleVertexManager.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/ShuffleVertexManager.java
@@ -43,7 +43,6 @@ import org.apache.tez.mapreduce.hadoop.MRHelpers;
 import org.apache.tez.runtime.api.events.DataMovementEvent;
 import org.apache.tez.runtime.api.events.InputFailedEvent;
 import org.apache.tez.runtime.api.events.InputReadErrorEvent;
-import org.apache.tez.runtime.records.TezDependentTaskCompletionEvent;
 
 /**
  * Starts scheduling tasks when number of completed source tasks crosses 
@@ -91,13 +90,13 @@ public class ShuffleVertexManager implements VertexScheduler {
   }
   
   
-  public class CustomEdgeManager extends EdgeManager {
+  public class CustomShuffleEdgeManager extends EdgeManager {
     int numSourceTaskOutputs;
     int numDestinationTasks;
     int basePartitionRange;
     int remainderRangeForLastShuffler;
     
-    CustomEdgeManager(int numSourceTaskOutputs, int numDestinationTasks,
+    CustomShuffleEdgeManager(int numSourceTaskOutputs, int numDestinationTasks,
         int basePartitionRange, int remainderPartitionForLastShuffler) {
       this.numSourceTaskOutputs = numSourceTaskOutputs;
       this.numDestinationTasks = numDestinationTasks;
@@ -106,7 +105,7 @@ public class ShuffleVertexManager implements VertexScheduler {
     }
 
     @Override
-    public int getNumDestinationTaskInputs(Vertex sourceVertex,
+    public int getNumDestinationTaskInputs(int numSourceTasks, 
         int destinationTaskIndex) {
       int partitionRange = 1;
       if(destinationTaskIndex < numDestinationTasks-1) {
@@ -114,11 +113,11 @@ public class ShuffleVertexManager implements VertexScheduler {
       } else {
         partitionRange = remainderRangeForLastShuffler;
       }
-      return sourceVertex.getTotalTasks() * partitionRange;
+      return numSourceTasks * partitionRange;
     }
 
     @Override
-    public int getNumSourceTaskOutputs(Vertex destinationVertex,
+    public int getNumSourceTaskOutputs(int numDestinationTasks, 
         int sourceTaskIndex) {
       return numSourceTaskOutputs;
     }
@@ -163,6 +162,12 @@ public class ShuffleVertexManager implements VertexScheduler {
       }
       return event.getIndex()/partitionRange;
     }
+
+    @Override
+    public int getDestinationConsumerTaskNumber(int sourceTaskIndex,
+        int numDestTasks) {
+      return numDestTasks;
+    }
   }
 
   
@@ -182,8 +187,7 @@ public class ShuffleVertexManager implements VertexScheduler {
   }
 
   @Override
-  public void onSourceTaskCompleted(TezTaskAttemptID srcAttemptId, 
-      TezDependentTaskCompletionEvent event) {
+  public void onSourceTaskCompleted(TezTaskAttemptID srcAttemptId) {
     updateSourceTaskCount();
     TezTaskID srcTaskId = srcAttemptId.getTaskID();
     TezVertexID srcVertexId = srcTaskId.getVertexID();
@@ -194,9 +198,10 @@ public class ShuffleVertexManager implements VertexScheduler {
         ++numSourceTasksCompleted;
         if (enableAutoParallelism) {
           // save output size
-          long sourceTaskOutputSize = event.getDataSize();
+          // TODO TEZ-481
+          long sourceTaskOutputSize = 100000000l;//sourceTaskAttempt.getDataSize();
           if (LOG.isDebugEnabled()) {
-            LOG.debug("Source task: " + event.getTaskAttemptID()
+            LOG.debug("Source task: " + srcAttemptId
                 + " finished with output size: " + sourceTaskOutputSize);
           }
           completedSourceTasksOutputSize += sourceTaskOutputSize;
@@ -282,7 +287,9 @@ public class ShuffleVertexManager implements VertexScheduler {
       Map<Vertex, EdgeManager> edgeManagers = new HashMap<Vertex, EdgeManager>(
           bipartiteSources.size());
       for(Vertex vertex : bipartiteSources.values()) {
-        edgeManagers.put(vertex, new CustomEdgeManager(currentParallelism,
+        // use currentParallelism for numSourceTasks to maintain original state
+        // for the source tasks
+        edgeManagers.put(vertex, new CustomShuffleEdgeManager(currentParallelism,
             finalTaskParallelism, basePartitionRange,
             remainderRangeForLastShuffler));
       }

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/3749a18f/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 b79f856..bfd14e2 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
@@ -41,6 +41,7 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.event.Event;
 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;
@@ -70,9 +71,8 @@ import org.apache.tez.dag.app.dag.event.DAGEventType;
 import org.apache.tez.dag.app.dag.event.DiagnosableEvent;
 import org.apache.tez.dag.app.dag.event.TaskAttemptEvent;
 import org.apache.tez.dag.app.dag.event.TaskAttemptEventContainerTerminated;
-import org.apache.tez.dag.app.dag.event.TaskAttemptEventContainerTerminating;
 import org.apache.tez.dag.app.dag.event.TaskAttemptEventDiagnosticsUpdate;
-import org.apache.tez.dag.app.dag.event.TaskAttemptEventNodeFailed;
+import org.apache.tez.dag.app.dag.event.TaskAttemptEventOutputFailed;
 import org.apache.tez.dag.app.dag.event.TaskAttemptEventSchedule;
 import org.apache.tez.dag.app.dag.event.TaskAttemptEventStartedRemotely;
 import org.apache.tez.dag.app.dag.event.TaskAttemptEventStatusUpdate;
@@ -89,8 +89,10 @@ import org.apache.tez.dag.records.TezTaskAttemptID;
 import org.apache.tez.dag.records.TezTaskID;
 import org.apache.tez.dag.records.TezVertexID;
 import org.apache.tez.dag.utils.TezBuilderUtils;
+import org.apache.tez.runtime.api.events.InputReadErrorEvent;
 import org.apache.tez.runtime.api.events.TaskStatusUpdateEvent;
 import org.apache.tez.runtime.api.impl.TaskSpec;
+import org.apache.tez.runtime.api.impl.TezEvent;
 
 import com.google.common.annotations.VisibleForTesting;
 
@@ -132,6 +134,9 @@ public class TaskAttemptImpl implements TaskAttempt,
   // Used to store locality information when
   Set<String> taskHosts = new HashSet<String>();
   Set<String> taskRacks = new HashSet<String>();
+  
+  private Set<String> uniquefailedOutputReports = new HashSet<String>();
+  private static final double MAX_ALLOWED_OUTPUT_FAILURES_FRACTION = 0.5;
 
   protected final TaskLocationHint locationHint;
   protected final boolean isRescheduled;
@@ -184,7 +189,7 @@ public class TaskAttemptImpl implements TaskAttempt,
         .addTransition(TaskAttemptStateInternal.RUNNING, TaskAttemptStateInternal.KILL_IN_PROGRESS, TaskAttemptEventType.TA_NODE_FAILED, new TerminatedWhileRunningTransition(KILLED_HELPER))
         .addTransition(TaskAttemptStateInternal.RUNNING, TaskAttemptStateInternal.FAIL_IN_PROGRESS, TaskAttemptEventType.TA_CONTAINER_TERMINATING, new TerminatedWhileRunningTransition(FAILED_HELPER))
         .addTransition(TaskAttemptStateInternal.RUNNING, TaskAttemptStateInternal.FAILED, TaskAttemptEventType.TA_CONTAINER_TERMINATED, new ContainerCompletedWhileRunningTransition())
-        .addTransition(TaskAttemptStateInternal.RUNNING, TaskAttemptStateInternal.FAIL_IN_PROGRESS, TaskAttemptEventType.TA_TOO_MANY_FETCH_FAILURES, new TerminatedWhileRunningTransition(FAILED_HELPER))
+        .addTransition(TaskAttemptStateInternal.RUNNING, EnumSet.of(TaskAttemptStateInternal.FAIL_IN_PROGRESS, TaskAttemptStateInternal.RUNNING), TaskAttemptEventType.TA_OUTPUT_FAILED, new OutputReportedFailedTransition())
 
         .addTransition(TaskAttemptStateInternal.OUTPUT_CONSUMABLE, TaskAttemptStateInternal.OUTPUT_CONSUMABLE, TaskAttemptEventType.TA_STATUS_UPDATE, STATUS_UPDATER)
         .addTransition(TaskAttemptStateInternal.OUTPUT_CONSUMABLE, TaskAttemptStateInternal.OUTPUT_CONSUMABLE, TaskAttemptEventType.TA_DIAGNOSTICS_UPDATE, DIAGNOSTIC_INFORMATION_UPDATE_TRANSITION)
@@ -198,28 +203,28 @@ public class TaskAttemptImpl implements TaskAttempt,
         .addTransition(TaskAttemptStateInternal.OUTPUT_CONSUMABLE, TaskAttemptStateInternal.KILL_IN_PROGRESS, TaskAttemptEventType.TA_NODE_FAILED, new TerminatedWhileRunningTransition(KILLED_HELPER))
         .addTransition(TaskAttemptStateInternal.OUTPUT_CONSUMABLE, TaskAttemptStateInternal.FAIL_IN_PROGRESS, TaskAttemptEventType.TA_CONTAINER_TERMINATING, new TerminatedWhileRunningTransition(FAILED_HELPER))
         .addTransition(TaskAttemptStateInternal.OUTPUT_CONSUMABLE, TaskAttemptStateInternal.FAILED, TaskAttemptEventType.TA_CONTAINER_TERMINATED, new ContainerCompletedBeforeRunningTransition())
-        .addTransition(TaskAttemptStateInternal.OUTPUT_CONSUMABLE, TaskAttemptStateInternal.FAIL_IN_PROGRESS, TaskAttemptEventType.TA_TOO_MANY_FETCH_FAILURES, new TerminatedWhileRunningTransition(FAILED_HELPER))
+        .addTransition(TaskAttemptStateInternal.OUTPUT_CONSUMABLE, EnumSet.of(TaskAttemptStateInternal.FAIL_IN_PROGRESS, TaskAttemptStateInternal.OUTPUT_CONSUMABLE), TaskAttemptEventType.TA_OUTPUT_FAILED, new OutputReportedFailedTransition())
 
         .addTransition(TaskAttemptStateInternal.KILL_IN_PROGRESS, TaskAttemptStateInternal.KILLED, TaskAttemptEventType.TA_CONTAINER_TERMINATED, new ContainerCompletedWhileTerminating())
         .addTransition(TaskAttemptStateInternal.KILL_IN_PROGRESS, TaskAttemptStateInternal.KILL_IN_PROGRESS, TaskAttemptEventType.TA_DIAGNOSTICS_UPDATE, DIAGNOSTIC_INFORMATION_UPDATE_TRANSITION)
-        .addTransition(TaskAttemptStateInternal.KILL_IN_PROGRESS, TaskAttemptStateInternal.KILL_IN_PROGRESS, EnumSet.of(TaskAttemptEventType.TA_STARTED_REMOTELY, TaskAttemptEventType.TA_STATUS_UPDATE, TaskAttemptEventType.TA_OUTPUT_CONSUMABLE, TaskAttemptEventType.TA_COMMIT_PENDING, TaskAttemptEventType.TA_DONE, TaskAttemptEventType.TA_FAILED, TaskAttemptEventType.TA_TIMED_OUT, TaskAttemptEventType.TA_FAIL_REQUEST, TaskAttemptEventType.TA_KILL_REQUEST, TaskAttemptEventType.TA_NODE_FAILED, TaskAttemptEventType.TA_CONTAINER_TERMINATING, TaskAttemptEventType.TA_TOO_MANY_FETCH_FAILURES))
+        .addTransition(TaskAttemptStateInternal.KILL_IN_PROGRESS, TaskAttemptStateInternal.KILL_IN_PROGRESS, EnumSet.of(TaskAttemptEventType.TA_STARTED_REMOTELY, TaskAttemptEventType.TA_STATUS_UPDATE, TaskAttemptEventType.TA_OUTPUT_CONSUMABLE, TaskAttemptEventType.TA_COMMIT_PENDING, TaskAttemptEventType.TA_DONE, TaskAttemptEventType.TA_FAILED, TaskAttemptEventType.TA_TIMED_OUT, TaskAttemptEventType.TA_FAIL_REQUEST, TaskAttemptEventType.TA_KILL_REQUEST, TaskAttemptEventType.TA_NODE_FAILED, TaskAttemptEventType.TA_CONTAINER_TERMINATING, TaskAttemptEventType.TA_OUTPUT_FAILED))
 
         .addTransition(TaskAttemptStateInternal.FAIL_IN_PROGRESS, TaskAttemptStateInternal.FAILED, TaskAttemptEventType.TA_CONTAINER_TERMINATED, new ContainerCompletedWhileTerminating())
         .addTransition(TaskAttemptStateInternal.FAIL_IN_PROGRESS, TaskAttemptStateInternal.FAIL_IN_PROGRESS, TaskAttemptEventType.TA_DIAGNOSTICS_UPDATE, DIAGNOSTIC_INFORMATION_UPDATE_TRANSITION)
-        .addTransition(TaskAttemptStateInternal.FAIL_IN_PROGRESS, TaskAttemptStateInternal.FAIL_IN_PROGRESS, EnumSet.of(TaskAttemptEventType.TA_STARTED_REMOTELY, TaskAttemptEventType.TA_STATUS_UPDATE, TaskAttemptEventType.TA_OUTPUT_CONSUMABLE, TaskAttemptEventType.TA_COMMIT_PENDING, TaskAttemptEventType.TA_DONE, TaskAttemptEventType.TA_FAILED, TaskAttemptEventType.TA_TIMED_OUT, TaskAttemptEventType.TA_FAIL_REQUEST, TaskAttemptEventType.TA_KILL_REQUEST, TaskAttemptEventType.TA_NODE_FAILED, TaskAttemptEventType.TA_CONTAINER_TERMINATING, TaskAttemptEventType.TA_TOO_MANY_FETCH_FAILURES))
+        .addTransition(TaskAttemptStateInternal.FAIL_IN_PROGRESS, TaskAttemptStateInternal.FAIL_IN_PROGRESS, EnumSet.of(TaskAttemptEventType.TA_STARTED_REMOTELY, TaskAttemptEventType.TA_STATUS_UPDATE, TaskAttemptEventType.TA_OUTPUT_CONSUMABLE, TaskAttemptEventType.TA_COMMIT_PENDING, TaskAttemptEventType.TA_DONE, TaskAttemptEventType.TA_FAILED, TaskAttemptEventType.TA_TIMED_OUT, TaskAttemptEventType.TA_FAIL_REQUEST, TaskAttemptEventType.TA_KILL_REQUEST, TaskAttemptEventType.TA_NODE_FAILED, TaskAttemptEventType.TA_CONTAINER_TERMINATING, TaskAttemptEventType.TA_OUTPUT_FAILED))
 
         .addTransition(TaskAttemptStateInternal.KILLED, TaskAttemptStateInternal.KILLED, TaskAttemptEventType.TA_DIAGNOSTICS_UPDATE, DIAGNOSTIC_INFORMATION_UPDATE_TRANSITION)
-        .addTransition(TaskAttemptStateInternal.KILLED, TaskAttemptStateInternal.KILLED, EnumSet.of(TaskAttemptEventType.TA_STARTED_REMOTELY, TaskAttemptEventType.TA_STATUS_UPDATE, TaskAttemptEventType.TA_OUTPUT_CONSUMABLE, TaskAttemptEventType.TA_COMMIT_PENDING, TaskAttemptEventType.TA_DONE, TaskAttemptEventType.TA_FAILED, TaskAttemptEventType.TA_TIMED_OUT, TaskAttemptEventType.TA_FAIL_REQUEST, TaskAttemptEventType.TA_KILL_REQUEST, TaskAttemptEventType.TA_NODE_FAILED, TaskAttemptEventType.TA_CONTAINER_TERMINATING, TaskAttemptEventType.TA_CONTAINER_TERMINATED, TaskAttemptEventType.TA_TOO_MANY_FETCH_FAILURES))
+        .addTransition(TaskAttemptStateInternal.KILLED, TaskAttemptStateInternal.KILLED, EnumSet.of(TaskAttemptEventType.TA_STARTED_REMOTELY, TaskAttemptEventType.TA_STATUS_UPDATE, TaskAttemptEventType.TA_OUTPUT_CONSUMABLE, TaskAttemptEventType.TA_COMMIT_PENDING, TaskAttemptEventType.TA_DONE, TaskAttemptEventType.TA_FAILED, TaskAttemptEventType.TA_TIMED_OUT, TaskAttemptEventType.TA_FAIL_REQUEST, TaskAttemptEventType.TA_KILL_REQUEST, TaskAttemptEventType.TA_NODE_FAILED, TaskAttemptEventType.TA_CONTAINER_TERMINATING, TaskAttemptEventType.TA_CONTAINER_TERMINATED, TaskAttemptEventType.TA_OUTPUT_FAILED))
 
         .addTransition(TaskAttemptStateInternal.FAILED, TaskAttemptStateInternal.FAILED, TaskAttemptEventType.TA_DIAGNOSTICS_UPDATE, DIAGNOSTIC_INFORMATION_UPDATE_TRANSITION)
-        .addTransition(TaskAttemptStateInternal.FAILED, TaskAttemptStateInternal.FAILED, EnumSet.of(TaskAttemptEventType.TA_STARTED_REMOTELY, TaskAttemptEventType.TA_STATUS_UPDATE, TaskAttemptEventType.TA_OUTPUT_CONSUMABLE, TaskAttemptEventType.TA_COMMIT_PENDING, TaskAttemptEventType.TA_DONE, TaskAttemptEventType.TA_FAILED, TaskAttemptEventType.TA_TIMED_OUT, TaskAttemptEventType.TA_FAIL_REQUEST, TaskAttemptEventType.TA_KILL_REQUEST, TaskAttemptEventType.TA_NODE_FAILED, TaskAttemptEventType.TA_CONTAINER_TERMINATING, TaskAttemptEventType.TA_CONTAINER_TERMINATED, TaskAttemptEventType.TA_TOO_MANY_FETCH_FAILURES))
+        .addTransition(TaskAttemptStateInternal.FAILED, TaskAttemptStateInternal.FAILED, EnumSet.of(TaskAttemptEventType.TA_STARTED_REMOTELY, TaskAttemptEventType.TA_STATUS_UPDATE, TaskAttemptEventType.TA_OUTPUT_CONSUMABLE, TaskAttemptEventType.TA_COMMIT_PENDING, TaskAttemptEventType.TA_DONE, TaskAttemptEventType.TA_FAILED, TaskAttemptEventType.TA_TIMED_OUT, TaskAttemptEventType.TA_FAIL_REQUEST, TaskAttemptEventType.TA_KILL_REQUEST, TaskAttemptEventType.TA_NODE_FAILED, TaskAttemptEventType.TA_CONTAINER_TERMINATING, TaskAttemptEventType.TA_CONTAINER_TERMINATED, TaskAttemptEventType.TA_OUTPUT_FAILED))
 
         // How will duplicate history events be handled ?
         // TODO Maybe consider not failing REDUCE tasks in this case. Also, MAP_TASKS in case there's only one phase in the job.
         .addTransition(TaskAttemptStateInternal.SUCCEEDED, TaskAttemptStateInternal.SUCCEEDED, TaskAttemptEventType.TA_DIAGNOSTICS_UPDATE, DIAGNOSTIC_INFORMATION_UPDATE_TRANSITION)
         .addTransition(TaskAttemptStateInternal.SUCCEEDED, TaskAttemptStateInternal.KILLED, TaskAttemptEventType.TA_KILL_REQUEST, new TerminatedAfterSuccessTransition(KILLED_HELPER))
         .addTransition(TaskAttemptStateInternal.SUCCEEDED, TaskAttemptStateInternal.KILLED, TaskAttemptEventType.TA_NODE_FAILED, new TerminatedAfterSuccessTransition(KILLED_HELPER))
-        .addTransition(TaskAttemptStateInternal.SUCCEEDED, TaskAttemptStateInternal.FAILED, TaskAttemptEventType.TA_TOO_MANY_FETCH_FAILURES, new TerminatedAfterSuccessTransition(FAILED_HELPER))
+        .addTransition(TaskAttemptStateInternal.SUCCEEDED, EnumSet.of(TaskAttemptStateInternal.FAILED, TaskAttemptStateInternal.SUCCEEDED), TaskAttemptEventType.TA_OUTPUT_FAILED, new OutputReportedFailedTransition())
         .addTransition(TaskAttemptStateInternal.SUCCEEDED, TaskAttemptStateInternal.SUCCEEDED, EnumSet.of(TaskAttemptEventType.TA_TIMED_OUT, TaskAttemptEventType.TA_FAIL_REQUEST, TaskAttemptEventType.TA_CONTAINER_TERMINATING, TaskAttemptEventType.TA_CONTAINER_TERMINATED))
 
 
@@ -551,7 +556,7 @@ public class TaskAttemptImpl implements TaskAttempt,
   // always called in write lock
   private void setFinishTime() {
     // set the finish time only if launch time is set
-    if (launchTime != 0) {
+    if (launchTime != 0 && finishTime != 0) {
       finishTime = clock.getTime();
     }
   }
@@ -967,13 +972,19 @@ public class TaskAttemptImpl implements TaskAttempt,
         TerminatedTransitionHelper helper) {
       super(helper);
     }
+    
+    protected boolean sendSchedulerEvent() {
+      return true;
+    }
 
     @Override
     public void transition(TaskAttemptImpl ta, TaskAttemptEvent event) {
       super.transition(ta, event);
       // Inform the scheduler
-      ta.sendEvent(new AMSchedulerEventTAEnded(ta, ta.containerId, helper
-          .getTaskAttemptState()));
+      if (sendSchedulerEvent()) {
+        ta.sendEvent(new AMSchedulerEventTAEnded(ta, ta.containerId, helper
+            .getTaskAttemptState()));
+      }
       // Decrement speculator container request.
       //ta.maybeSendSpeculatorContainerNoLongerRequired();
     }
@@ -989,8 +1000,6 @@ public class TaskAttemptImpl implements TaskAttempt,
     @Override
     public void transition(TaskAttemptImpl ta, TaskAttemptEvent event) {
       super.transition(ta, event);
-      TaskAttemptEventNodeFailed nfEvent = (TaskAttemptEventNodeFailed) event;
-      ta.addDiagnosticInfo(nfEvent.getDiagnosticInfo());
     }
   }
 
@@ -1004,8 +1013,6 @@ public class TaskAttemptImpl implements TaskAttempt,
     @Override
     public void transition(TaskAttemptImpl ta, TaskAttemptEvent event) {
       super.transition(ta, event);
-      TaskAttemptEventContainerTerminating tEvent = (TaskAttemptEventContainerTerminating) event;
-      ta.addDiagnosticInfo(tEvent.getDiagnosticInfo());
     }
   }
 
@@ -1019,9 +1026,6 @@ public class TaskAttemptImpl implements TaskAttempt,
     public void transition(TaskAttemptImpl ta, TaskAttemptEvent event) {
       super.transition(ta, event);
       ta.sendTaskAttemptCleanupEvent();
-
-      TaskAttemptEventContainerTerminated tEvent = (TaskAttemptEventContainerTerminated) event;
-      ta.addDiagnosticInfo(tEvent.getDiagnosticInfo());
     }
 
   }
@@ -1041,15 +1045,6 @@ public class TaskAttemptImpl implements TaskAttempt,
 
       ta.updateProgressSplits();
 
-      // TODO TEZ-431
-//      // Inform the job about fetch failures if they exist.
-//      if (ta.reportedStatus.fetchFailedMaps != null
-//          && ta.reportedStatus.fetchFailedMaps.size() > 0) {
-//        ta.sendEvent(new VertexEventTaskAttemptFetchFailure(ta.attemptId,
-//            ta.reportedStatus.fetchFailedMaps));
-//      }
-      // TODO at some point. Nodes may be interested in FetchFailure info.
-      // Can be used to blacklist nodes.
     }
   }
 
@@ -1119,8 +1114,6 @@ public class TaskAttemptImpl implements TaskAttempt,
     public void transition(TaskAttemptImpl ta, TaskAttemptEvent event) {
       super.transition(ta, event);
       ta.sendTaskAttemptCleanupEvent();
-      TaskAttemptEventContainerTerminated tEvent = (TaskAttemptEventContainerTerminated) event;
-      ta.addDiagnosticInfo(tEvent.getDiagnosticInfo());
     }
   }
 
@@ -1139,6 +1132,13 @@ public class TaskAttemptImpl implements TaskAttempt,
   protected static class TerminatedAfterSuccessTransition extends
       TerminatedBeforeRunningTransition {
 
+    @Override
+    protected boolean sendSchedulerEvent() {
+      // since the success transition would have sent the event
+      // there is no need to send it again
+      return false;
+    }
+    
     public TerminatedAfterSuccessTransition(TerminatedTransitionHelper helper) {
       super(helper);
     }
@@ -1150,6 +1150,55 @@ public class TaskAttemptImpl implements TaskAttempt,
     }
 
   }
+  
+  protected static class OutputReportedFailedTransition implements
+  MultipleArcTransition<TaskAttemptImpl, TaskAttemptEvent, TaskAttemptStateInternal> {
+
+    @Override
+    public TaskAttemptStateInternal transition(TaskAttemptImpl attempt,
+        TaskAttemptEvent event) {
+      TaskAttemptEventOutputFailed outputFailedEvent = 
+          (TaskAttemptEventOutputFailed) event;
+      TezEvent tezEvent = outputFailedEvent.getInputFailedEvent();
+      TezTaskAttemptID failedDestTaId = tezEvent.getSourceInfo().getTaskAttemptID();
+      InputReadErrorEvent readErrorEvent = (InputReadErrorEvent)tezEvent.getEvent();
+      int failedInputIndexOnDestTa = readErrorEvent.getIndex();
+      if (readErrorEvent.getVersion() != attempt.getID().getId()) {
+        throw new TezUncheckedException(attempt.getID()
+            + " incorrectly blamed for read error from " + failedDestTaId
+            + " at inputIndex " + failedInputIndexOnDestTa + " version"
+            + readErrorEvent.getVersion());
+      }
+      LOG.info(attempt.getID()
+            + " blamed for read error from " + failedDestTaId
+            + " at inputIndex " + failedInputIndexOnDestTa);
+      String failedReportId = failedDestTaId + "_" + failedInputIndexOnDestTa;
+      attempt.uniquefailedOutputReports.add(failedReportId);
+      float failureFraction = ((float) attempt.uniquefailedOutputReports.size())
+          / outputFailedEvent.getConsumerTaskNumber();
+      
+      // If needed we can also use the absolute number of reported output errors
+      // If needed we can launch a background task without failing this task
+      // to generate a copy of the output just in case.
+      if (failureFraction <= MAX_ALLOWED_OUTPUT_FAILURES_FRACTION) {
+        return attempt.getInternalState();
+      }
+      String message = attempt.getID() + " being failed for too many output errors";
+      LOG.info(message);
+      attempt.addDiagnosticInfo(message);
+      if (attempt.getInternalState() == TaskAttemptStateInternal.SUCCEEDED) {
+        (new TerminatedAfterSuccessTransition(FAILED_HELPER)).transition(
+            attempt, event);
+        return TaskAttemptStateInternal.FAILED;
+      } else {
+        (new TerminatedWhileRunningTransition(FAILED_HELPER)).transition(
+            attempt, event);
+        return TaskAttemptStateInternal.FAIL_IN_PROGRESS;
+      }
+      // TODO at some point. Nodes may be interested in FetchFailure info.
+      // Can be used to blacklist nodes.
+    }
+  }
 
   private void initTaskAttemptStatus(TaskAttemptStatus result) {
     result.progress = 0.0f;

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/3749a18f/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 c1a9415..36d0abb 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
@@ -40,7 +40,6 @@ import org.apache.hadoop.yarn.state.StateMachine;
 import org.apache.hadoop.yarn.state.StateMachineFactory;
 import org.apache.hadoop.yarn.util.Clock;
 import org.apache.hadoop.yarn.util.Records;
-import org.apache.tez.common.counters.TaskCounter;
 import org.apache.tez.common.counters.TezCounters;
 import org.apache.tez.dag.api.ProcessorDescriptor;
 import org.apache.tez.dag.api.TezConfiguration;
@@ -55,6 +54,7 @@ import org.apache.tez.dag.app.TaskAttemptListener;
 import org.apache.tez.dag.app.TaskHeartbeatHandler;
 import org.apache.tez.dag.app.dag.Task;
 import org.apache.tez.dag.app.dag.TaskAttempt;
+import org.apache.tez.dag.app.dag.TaskAttemptStateInternal;
 import org.apache.tez.dag.app.dag.TaskStateInternal;
 import org.apache.tez.dag.app.dag.Vertex;
 import org.apache.tez.dag.app.dag.event.DAGEvent;
@@ -76,7 +76,6 @@ 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.impl.TezEvent;
-import org.apache.tez.runtime.records.TezDependentTaskCompletionEvent;
 
 import com.google.common.annotations.VisibleForTesting;
 
@@ -209,16 +208,17 @@ public class TaskImpl implements Task, EventHandler<TaskEvent> {
       // the stages. i.e. Task would only SUCCEED after all output consumed.
     .addTransition(TaskStateInternal.SUCCEEDED, //only possible for map tasks
         EnumSet.of(TaskStateInternal.SCHEDULED, TaskStateInternal.SUCCEEDED, TaskStateInternal.FAILED),
-        TaskEventType.T_ATTEMPT_FAILED, new MapRetroactiveFailureTransition())
+        TaskEventType.T_ATTEMPT_FAILED, new TaskRetroactiveFailureTransition())
     .addTransition(TaskStateInternal.SUCCEEDED, //only possible for map tasks
         EnumSet.of(TaskStateInternal.SCHEDULED, TaskStateInternal.SUCCEEDED),
-        TaskEventType.T_ATTEMPT_KILLED, new MapRetroactiveKilledTransition())
+        TaskEventType.T_ATTEMPT_KILLED, new TaskRetroactiveKilledTransition())
     .addTransition(TaskStateInternal.SUCCEEDED, TaskStateInternal.SUCCEEDED,
         TaskEventType.T_ADD_TEZ_EVENT, ADD_TEZ_EVENT_TRANSITION)
     // Ignore-able transitions.
     .addTransition(
         TaskStateInternal.SUCCEEDED, TaskStateInternal.SUCCEEDED,
         EnumSet.of(TaskEventType.T_ADD_SPEC_ATTEMPT,
+            TaskEventType.T_TERMINATE,
             TaskEventType.T_ATTEMPT_LAUNCHED))
 
     // Transitions from FAILED state
@@ -749,41 +749,26 @@ public class TaskImpl implements Task, EventHandler<TaskEvent> {
   }
 
   private void sendTaskAttemptCompletionEvent(TezTaskAttemptID attemptId,
-      TezDependentTaskCompletionEvent.Status status) {
-    TaskAttempt attempt = attempts.get(attemptId);
+      TaskAttemptStateInternal attemptState) {
     // raise the completion event only if the container is assigned
     // to nextAttemptNumber
     if (needsWaitAfterOutputConsumable()) {
       // An event may have been sent out during the OUTPUT_READY state itself.
       // Make sure the same event is not being sent out again.
       if (attemptId == outputConsumableAttempt
-          && status == TezDependentTaskCompletionEvent.Status.SUCCEEDED) {
+          && attemptState == TaskAttemptStateInternal.SUCCEEDED) {
         if (outputConsumableAttemptSuccessSent) {
           return;
         }
       }
     }
-    if (attempt.getNodeHttpAddress() != null) {
-
-      int runTime = 0;
-      if (attempt.getFinishTime() != 0 && attempt.getLaunchTime() != 0)
-        runTime = (int) (attempt.getFinishTime() - attempt.getLaunchTime());
-
-      // TODO TEZ-347. Get this event from Task instead of generating here
-      long dataSize = getCounters().findCounter(TaskCounter.MAP_OUTPUT_BYTES).getValue();
-      TezDependentTaskCompletionEvent tce = new TezDependentTaskCompletionEvent(
-          -1, attemptId, status, runTime, dataSize);
-
-      // raise the event to job so that it adds the completion event to its
-      // data structures
-      eventHandler.handle(new VertexEventTaskAttemptCompleted(tce));
-    }
+    eventHandler.handle(new VertexEventTaskAttemptCompleted(attemptId, attemptState));
   }
 
   // always called inside a transition, in turn inside the Write Lock
   private void handleTaskAttemptCompletion(TezTaskAttemptID attemptId,
-      TezDependentTaskCompletionEvent.Status status) {
-    this.sendTaskAttemptCompletionEvent(attemptId, status);
+      TaskAttemptStateInternal attemptState) {
+    this.sendTaskAttemptCompletionEvent(attemptId, attemptState);
   }
 
   // TODO: Recovery
@@ -893,7 +878,7 @@ public class TaskImpl implements Task, EventHandler<TaskEvent> {
 
       if (task.outputConsumableAttempt == null) {
         task.sendTaskAttemptCompletionEvent(attemptId,
-            TezDependentTaskCompletionEvent.Status.SUCCEEDED);
+            TaskAttemptStateInternal.SUCCEEDED);
         task.outputConsumableAttempt = attemptId;
         task.outputConsumableAttemptSuccessSent = true;
         if (LOG.isDebugEnabled()) {
@@ -932,7 +917,7 @@ public class TaskImpl implements Task, EventHandler<TaskEvent> {
       }
       
       task.handleTaskAttemptCompletion(successTaId, 
-          TezDependentTaskCompletionEvent.Status.SUCCEEDED);
+          TaskAttemptStateInternal.SUCCEEDED);
       task.finishedAttempts++;
       --task.numberUncompletedAttempts;
       task.successfulAttempt = successTaId;
@@ -974,7 +959,7 @@ public class TaskImpl implements Task, EventHandler<TaskEvent> {
       }
       task.handleTaskAttemptCompletion(
           castEvent.getTaskAttemptID(),
-          TezDependentTaskCompletionEvent.Status.KILLED);
+          TaskAttemptStateInternal.KILLED);
       task.finishedAttempts++;
       // we don't need a new event if we already have a spare
       if (--task.numberUncompletedAttempts == 0
@@ -994,7 +979,7 @@ public class TaskImpl implements Task, EventHandler<TaskEvent> {
     public TaskStateInternal transition(TaskImpl task, TaskEvent event) {
       task.handleTaskAttemptCompletion(
           ((TaskEventTAUpdate) event).getTaskAttemptID(),
-          TezDependentTaskCompletionEvent.Status.KILLED);
+          TaskAttemptStateInternal.KILLED);
       task.finishedAttempts++;
       // check whether all attempts are finished
       if (task.finishedAttempts == task.attempts.size()) {
@@ -1027,7 +1012,7 @@ public class TaskImpl implements Task, EventHandler<TaskEvent> {
       if (castEvent.getTaskAttemptID().equals(task.outputConsumableAttempt)) {
         task.outputConsumableAttempt = null;
         task.handleTaskAttemptCompletion(castEvent.getTaskAttemptID(),
-            TezDependentTaskCompletionEvent.Status.FAILED);
+            TaskAttemptStateInternal.FAILED);
       }
 
       // The attempt would have informed the scheduler about it's failure
@@ -1036,7 +1021,7 @@ public class TaskImpl implements Task, EventHandler<TaskEvent> {
       if (task.failedAttempts < task.maxAttempts) {
         task.handleTaskAttemptCompletion(
             ((TaskEventTAUpdate) event).getTaskAttemptID(),
-            TezDependentTaskCompletionEvent.Status.FAILED);
+            TaskAttemptStateInternal.FAILED);
         // we don't need a new event if we already have a spare
         if (--task.numberUncompletedAttempts == 0
             && task.successfulAttempt == null) {
@@ -1045,7 +1030,7 @@ public class TaskImpl implements Task, EventHandler<TaskEvent> {
       } else {
         task.handleTaskAttemptCompletion(
             ((TaskEventTAUpdate) event).getTaskAttemptID(),
-            TezDependentTaskCompletionEvent.Status.TIPFAILED);
+            TaskAttemptStateInternal.FAILED);
 
         if (task.historyTaskStartGenerated) {
           task.logJobHistoryTaskFailedEvent(TaskState.FAILED);
@@ -1065,36 +1050,35 @@ public class TaskImpl implements Task, EventHandler<TaskEvent> {
     }
   }
 
-  private static class MapRetroactiveFailureTransition
+  private static class TaskRetroactiveFailureTransition
       extends AttemptFailedTransition {
 
     @Override
     public TaskStateInternal transition(TaskImpl task, TaskEvent event) {
-      if (event instanceof TaskEventTAUpdate) {
-        TaskEventTAUpdate castEvent = (TaskEventTAUpdate) event;
-        if (task.getInternalState() == TaskStateInternal.SUCCEEDED &&
-            !castEvent.getTaskAttemptID().equals(task.successfulAttempt)) {
-          // don't allow a different task attempt to override a previous
-          // succeeded state
-          return TaskStateInternal.SUCCEEDED;
-        }
-      }
-
       if (task.leafVertex) {
         LOG.error("Unexpected event for task of leaf vertex " + event.getType());
         task.internalError(event.getType());
       }
 
+      TaskEventTAUpdate castEvent = (TaskEventTAUpdate) event;
+      if (task.getInternalState() == TaskStateInternal.SUCCEEDED &&
+          !castEvent.getTaskAttemptID().equals(task.successfulAttempt)) {
+        // don't allow a different task attempt to override a previous
+        // succeeded state
+        return TaskStateInternal.SUCCEEDED;
+      }
+
       // tell the job about the rescheduling
-      task.eventHandler.handle(
-          new VertexEventTaskReschedule(task.taskId));
+      task.eventHandler.handle(new VertexEventTaskReschedule(task.taskId));
       // super.transition is mostly coded for the case where an
       //  UNcompleted task failed.  When a COMPLETED task retroactively
       //  fails, we have to let AttemptFailedTransition.transition
       //  believe that there's no redundancy.
       unSucceed(task);
-      // fake increase in Uncomplete attempts for super.transition
+      
+      // fake values for code for super.transition
       ++task.numberUncompletedAttempts;
+      task.finishedAttempts--;
       return super.transition(task, event);
     }
 
@@ -1104,7 +1088,7 @@ public class TaskImpl implements Task, EventHandler<TaskEvent> {
     }
   }
 
-  private static class MapRetroactiveKilledTransition implements
+  private static class TaskRetroactiveKilledTransition implements
     MultipleArcTransition<TaskImpl, TaskEvent, TaskStateInternal> {
 
     @Override
@@ -1124,7 +1108,7 @@ public class TaskImpl implements Task, EventHandler<TaskEvent> {
         unSucceed(task);
         task.handleTaskAttemptCompletion(
             attemptId,
-            TezDependentTaskCompletionEvent.Status.KILLED);
+            TaskAttemptStateInternal.KILLED);
         task.eventHandler.handle(new VertexEventTaskReschedule(task.taskId));
         // typically we are here because this map task was run on a bad node and
         // we want to reschedule it on a different node.


[20/50] [abbrv] TEZ-443. Merge tez-dag-api and tez-engine-api into a single module - tez-api (part of TEZ-398). (sseth)

Posted by ss...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-common/src/main/java/org/apache/tez/common/counters/GenericCounter.java
----------------------------------------------------------------------
diff --git a/tez-common/src/main/java/org/apache/tez/common/counters/GenericCounter.java b/tez-common/src/main/java/org/apache/tez/common/counters/GenericCounter.java
deleted file mode 100644
index 5477606..0000000
--- a/tez-common/src/main/java/org/apache/tez/common/counters/GenericCounter.java
+++ /dev/null
@@ -1,109 +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.common.counters;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.WritableUtils;
-
-/**
- * A generic counter implementation
- */
-@InterfaceAudience.Private
-public class GenericCounter extends AbstractCounter {
-
-  private String name;
-  private String displayName;
-  private long value = 0;
-
-  public GenericCounter() {
-    // mostly for readFields
-  }
-
-  public GenericCounter(String name, String displayName) {
-    this.name = name;
-    this.displayName = displayName;
-  }
-
-  public GenericCounter(String name, String displayName, long value) {
-    this.name = name;
-    this.displayName = displayName;
-    this.value = value;
-  }
-
-  @Override @Deprecated
-  public synchronized void setDisplayName(String displayName) {
-    this.displayName = displayName;
-  }
-
-  @Override
-  public synchronized void readFields(DataInput in) throws IOException {
-    name = Text.readString(in);
-    displayName = in.readBoolean() ? Text.readString(in) : name;
-    value = WritableUtils.readVLong(in);
-  }
-
-  /**
-   * GenericCounter ::= keyName isDistinctDisplayName [displayName] value
-   */
-  @Override
-  public synchronized void write(DataOutput out) throws IOException {
-    Text.writeString(out, name);
-    boolean distinctDisplayName = ! name.equals(displayName);
-    out.writeBoolean(distinctDisplayName);
-    if (distinctDisplayName) {
-      Text.writeString(out, displayName);
-    }
-    WritableUtils.writeVLong(out, value);
-  }
-
-  @Override
-  public synchronized String getName() {
-    return name;
-  }
-
-  @Override
-  public synchronized String getDisplayName() {
-    return displayName;
-  }
-
-  @Override
-  public synchronized long getValue() {
-    return value;
-  }
-
-  @Override
-  public synchronized void setValue(long value) {
-    this.value = value;
-  }
-
-  @Override
-  public synchronized void increment(long incr) {
-    value += incr;
-  }
-
-  @Override
-  public TezCounter getUnderlyingCounter() {
-    return this;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-common/src/main/java/org/apache/tez/common/counters/JobCounter.java
----------------------------------------------------------------------
diff --git a/tez-common/src/main/java/org/apache/tez/common/counters/JobCounter.java b/tez-common/src/main/java/org/apache/tez/common/counters/JobCounter.java
deleted file mode 100644
index 1eb2be8..0000000
--- a/tez-common/src/main/java/org/apache/tez/common/counters/JobCounter.java
+++ /dev/null
@@ -1,45 +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.common.counters;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-
-// Per-job counters
-@InterfaceAudience.Public
-@InterfaceStability.Evolving
-public enum JobCounter {
-  NUM_FAILED_MAPS, 
-  NUM_FAILED_REDUCES,
-  NUM_KILLED_MAPS,
-  NUM_KILLED_REDUCES,
-  TOTAL_LAUNCHED_MAPS,
-  TOTAL_LAUNCHED_REDUCES,
-  OTHER_LOCAL_MAPS,
-  DATA_LOCAL_MAPS,
-  RACK_LOCAL_MAPS,
-  SLOTS_MILLIS_MAPS,
-  SLOTS_MILLIS_REDUCES,
-  FALLOW_SLOTS_MILLIS_MAPS,
-  FALLOW_SLOTS_MILLIS_REDUCES,
-  TOTAL_LAUNCHED_UBERTASKS,
-  NUM_UBER_SUBMAPS,
-  NUM_UBER_SUBREDUCES,
-  NUM_FAILED_UBERTASKS
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-common/src/main/java/org/apache/tez/common/counters/LimitExceededException.java
----------------------------------------------------------------------
diff --git a/tez-common/src/main/java/org/apache/tez/common/counters/LimitExceededException.java b/tez-common/src/main/java/org/apache/tez/common/counters/LimitExceededException.java
deleted file mode 100644
index e50bd81..0000000
--- a/tez-common/src/main/java/org/apache/tez/common/counters/LimitExceededException.java
+++ /dev/null
@@ -1,36 +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.common.counters;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-
-@InterfaceAudience.Private
-public class LimitExceededException extends RuntimeException {
-
-  private static final long serialVersionUID = 1L;
-
-  public LimitExceededException(String msg) {
-    super(msg);
-  }
-
-  // Only allows chaining of related exceptions
-  public LimitExceededException(LimitExceededException cause) {
-    super(cause);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-common/src/main/java/org/apache/tez/common/counters/Limits.java
----------------------------------------------------------------------
diff --git a/tez-common/src/main/java/org/apache/tez/common/counters/Limits.java b/tez-common/src/main/java/org/apache/tez/common/counters/Limits.java
deleted file mode 100644
index aacce87..0000000
--- a/tez-common/src/main/java/org/apache/tez/common/counters/Limits.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.common.counters;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.tez.common.TezJobConfig;
-
-@InterfaceAudience.Private
-public class Limits {
-
-  private static Configuration conf = null;
-  private static int GROUP_NAME_MAX;
-  private static int COUNTER_NAME_MAX;
-  private static int GROUPS_MAX;
-  private static int COUNTERS_MAX;
-  private static boolean initialized = false;
-
-  private static synchronized void ensureInitialized() {
-    if (initialized) {
-      return;
-    }
-    if (conf == null) {
-      conf = new Configuration();
-    }
-    GROUP_NAME_MAX =
-        conf.getInt(TezJobConfig.COUNTER_GROUP_NAME_MAX_KEY,
-            TezJobConfig.COUNTER_GROUP_NAME_MAX_DEFAULT);
-    COUNTER_NAME_MAX =
-        conf.getInt(TezJobConfig.COUNTER_NAME_MAX_KEY,
-            TezJobConfig.COUNTER_NAME_MAX_DEFAULT);
-    GROUPS_MAX =
-        conf.getInt(TezJobConfig.COUNTER_GROUPS_MAX_KEY,
-            TezJobConfig.COUNTER_GROUPS_MAX_DEFAULT);
-    COUNTERS_MAX =
-        conf.getInt(TezJobConfig.COUNTERS_MAX_KEY, TezJobConfig.
-            COUNTERS_MAX_DEFAULT);
-    initialized = true;
-  }
-
-  private int totalCounters;
-  private LimitExceededException firstViolation;
-
-  public static String filterName(String name, int maxLen) {
-    return name.length() > maxLen ? name.substring(0, maxLen - 1) : name;
-  }
-
-  public static String filterCounterName(String name) {
-    ensureInitialized();
-    return filterName(name, COUNTER_NAME_MAX);
-  }
-
-  public static String filterGroupName(String name) {
-    ensureInitialized();
-    return filterName(name, GROUP_NAME_MAX);
-  }
-
-  public synchronized void checkCounters(int size) {
-    ensureInitialized();
-    if (firstViolation != null) {
-      throw new LimitExceededException(firstViolation);
-    }
-    if (size > COUNTERS_MAX) {
-      firstViolation = new LimitExceededException("Too many counters: "+ size +
-                                                  " max="+ COUNTERS_MAX);
-      throw firstViolation;
-    }
-  }
-
-  public synchronized void incrCounters() {
-    checkCounters(totalCounters + 1);
-    ++totalCounters;
-  }
-
-  public synchronized void checkGroups(int size) {
-    ensureInitialized();
-    if (firstViolation != null) {
-      throw new LimitExceededException(firstViolation);
-    }
-    if (size > GROUPS_MAX) {
-      firstViolation = new LimitExceededException("Too many counter groups: "+
-                                                  size +" max="+ GROUPS_MAX);
-    }
-  }
-
-  public synchronized LimitExceededException violation() {
-    return firstViolation;
-  }
-
-  public synchronized static void setConfiguration(Configuration conf) {
-    if (Limits.conf == null && conf != null) {
-      Limits.conf = conf;
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-common/src/main/java/org/apache/tez/common/counters/ResourceBundles.java
----------------------------------------------------------------------
diff --git a/tez-common/src/main/java/org/apache/tez/common/counters/ResourceBundles.java b/tez-common/src/main/java/org/apache/tez/common/counters/ResourceBundles.java
deleted file mode 100644
index 8113cab..0000000
--- a/tez-common/src/main/java/org/apache/tez/common/counters/ResourceBundles.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.common.counters;
-
-import java.util.Locale;
-import java.util.ResourceBundle;
-import java.util.MissingResourceException;
-
-/**
- * Helper class to handle resource bundles in a saner way
- */
-public class ResourceBundles {
-
-  /**
-   * Get a resource bundle
-   * @param bundleName of the resource
-   * @return the resource bundle
-   * @throws MissingResourceException
-   */
-  public static ResourceBundle getBundle(String bundleName) {
-    return ResourceBundle.getBundle(bundleName.replace('$', '_'),
-        Locale.getDefault(), Thread.currentThread().getContextClassLoader());
-  }
-
-  /**
-   * Get a resource given bundle name and key
-   * @param <T> type of the resource
-   * @param bundleName name of the resource bundle
-   * @param key to lookup the resource
-   * @param suffix for the key to lookup
-   * @param defaultValue of the resource
-   * @return the resource or the defaultValue
-   * @throws ClassCastException if the resource found doesn't match T
-   */
-  @SuppressWarnings("unchecked")
-  public static synchronized <T> T getValue(String bundleName, String key,
-                                            String suffix, T defaultValue) {
-    T value;
-    try {
-      ResourceBundle bundle = getBundle(bundleName);
-      value = (T) bundle.getObject(getLookupKey(key, suffix));
-      if (value != null) {
-        return value;
-      }
-    }
-    catch (Exception e) {
-      // Ignore
-    }
-    return defaultValue;
-  }
-
-  private static String getLookupKey(String key, String suffix) {
-    if (suffix == null || suffix.isEmpty()) return key;
-    return key + suffix;
-  }
-
-  /**
-   * Get the counter group display name
-   * @param group the group name to lookup
-   * @param defaultValue of the group
-   * @return the group display name
-   */
-  public static String getCounterGroupName(String group, String defaultValue) {
-    return getValue(group, "CounterGroupName", "", defaultValue);
-  }
-
-  /**
-   * Get the counter display name
-   * @param group the counter group name for the counter
-   * @param counter the counter name to lookup
-   * @param defaultValue of the counter
-   * @return the counter display name
-   */
-  public static String getCounterName(String group, String counter,
-                                      String defaultValue) {
-    return getValue(group, counter, ".name", defaultValue);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-common/src/main/java/org/apache/tez/common/counters/TaskCounter.java
----------------------------------------------------------------------
diff --git a/tez-common/src/main/java/org/apache/tez/common/counters/TaskCounter.java b/tez-common/src/main/java/org/apache/tez/common/counters/TaskCounter.java
deleted file mode 100644
index b6fca27..0000000
--- a/tez-common/src/main/java/org/apache/tez/common/counters/TaskCounter.java
+++ /dev/null
@@ -1,66 +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.common.counters;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-
-// TODO TEZAM5 For MR compatibility, a conversion from tez.TaskCounters to
-// mapreduce.TaskCounters will likely be required somewhere.
-// Similarly for FileSystemCounters and others.
-
-// Counters used by Task classes
-@InterfaceAudience.Public
-@InterfaceStability.Evolving
-public enum TaskCounter {
-  // TODO Eventually, rename counters to be non-MR specific and map them to MR equivalent.
-  MAP_INPUT_RECORDS, 
-  MAP_OUTPUT_RECORDS,
-  MAP_SKIPPED_RECORDS,
-  MAP_OUTPUT_BYTES,
-  MAP_OUTPUT_MATERIALIZED_BYTES,
-  SPLIT_RAW_BYTES,
-  COMBINE_INPUT_RECORDS,
-  COMBINE_OUTPUT_RECORDS,
-  REDUCE_INPUT_GROUPS,
-  REDUCE_SHUFFLE_BYTES,
-  REDUCE_INPUT_RECORDS,
-  REDUCE_OUTPUT_RECORDS,
-  REDUCE_SKIPPED_GROUPS,
-  REDUCE_SKIPPED_RECORDS,
-  SPILLED_RECORDS,
-  SHUFFLED_MAPS, 
-  FAILED_SHUFFLE,
-  MERGED_MAP_OUTPUTS,
-  GC_TIME_MILLIS,
-  CPU_MILLISECONDS,
-  PHYSICAL_MEMORY_BYTES,
-  VIRTUAL_MEMORY_BYTES,
-  COMMITTED_HEAP_BYTES,
-  
-  INPUT_RECORDS, 
-  OUTPUT_RECORDS,
-  SKIPPED_RECORDS,
-  OUTPUT_BYTES,
-  OUTPUT_MATERIALIZED_BYTES,
-  INPUT_GROUPS,
-  SHUFFLE_BYTES,
-  SHUFFLED_TASKS, 
-  MERGED_TASK_OUTPUTS,
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-common/src/main/java/org/apache/tez/common/counters/TezCounter.java
----------------------------------------------------------------------
diff --git a/tez-common/src/main/java/org/apache/tez/common/counters/TezCounter.java b/tez-common/src/main/java/org/apache/tez/common/counters/TezCounter.java
deleted file mode 100644
index 394c820..0000000
--- a/tez-common/src/main/java/org/apache/tez/common/counters/TezCounter.java
+++ /dev/null
@@ -1,83 +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.common.counters;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.classification.InterfaceAudience.Private;
-import org.apache.hadoop.io.Writable;
-
-/**
- * A named counter that tracks the progress of a map/reduce job.
- *
- * <p><code>Counters</code> represent global counters, defined either by the
- * Map-Reduce framework or applications. Each <code>Counter</code> is named by
- * an {@link Enum} and has a long for the value.</p>
- *
- * <p><code>Counters</code> are bunched into Groups, each comprising of
- * counters from a particular <code>Enum</code> class.
- */
-@InterfaceAudience.Public
-@InterfaceStability.Stable
-public interface TezCounter extends Writable {
-
-  /**
-   * Set the display name of the counter
-   * @param displayName of the counter
-   * @deprecated (and no-op by default)
-   */
-  @Deprecated
-  void setDisplayName(String displayName);
-
-  /**
-   * @return the name of the counter
-   */
-  String getName();
-
-  /**
-   * Get the display name of the counter.
-   * @return the user facing name of the counter
-   */
-  String getDisplayName();
-
-  /**
-   * What is the current value of this counter?
-   * @return the current value
-   */
-  long getValue();
-
-  /**
-   * Set this counter by the given value
-   * @param value the value to set
-   */
-  void setValue(long value);
-
-  /**
-   * Increment this counter by the given value
-   * @param incr the value to increase this counter by
-   */
-  void increment(long incr);
- 
-  /**
-   * Return the underlying object if this is a facade.
-   * @return the undelying object.
-   */
-  @Private
-  TezCounter getUnderlyingCounter();
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-common/src/main/java/org/apache/tez/common/counters/TezCounters.java
----------------------------------------------------------------------
diff --git a/tez-common/src/main/java/org/apache/tez/common/counters/TezCounters.java b/tez-common/src/main/java/org/apache/tez/common/counters/TezCounters.java
deleted file mode 100644
index 1c9521a..0000000
--- a/tez-common/src/main/java/org/apache/tez/common/counters/TezCounters.java
+++ /dev/null
@@ -1,144 +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.common.counters;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-
-/**
- * <p><code>Counters</code> holds per job/task counters, defined either by the
- * Map-Reduce framework or applications. Each <code>Counter</code> can be of
- * any {@link Enum} type.</p>
- *
- * <p><code>Counters</code> are bunched into {@link CounterGroup}s, each
- * comprising of counters from a particular <code>Enum</code> class.
- */
-@InterfaceAudience.Public
-@InterfaceStability.Stable
-public class TezCounters extends AbstractCounters<TezCounter, CounterGroup> {
-
-  // Mix framework group implementation into CounterGroup interface
-  private static class FrameworkGroupImpl<T extends Enum<T>>
-      extends FrameworkCounterGroup<T, TezCounter> implements CounterGroup {
-
-    FrameworkGroupImpl(Class<T> cls) {
-      super(cls);
-    }
-
-    @Override
-    protected FrameworkCounter<T> newCounter(T key) {
-      return new FrameworkCounter<T>(key, getName());
-    }
-
-    @Override
-    public CounterGroupBase<TezCounter> getUnderlyingGroup() {
-      return this;
-    }
-  }
-
-  // Mix generic group implementation into CounterGroup interface
-  // and provide some mandatory group factory methods.
-  private static class GenericGroup extends AbstractCounterGroup<TezCounter>
-      implements CounterGroup {
-
-    GenericGroup(String name, String displayName, Limits limits) {
-      super(name, displayName, limits);
-    }
-
-    @Override
-    protected TezCounter newCounter(String name, String displayName, long value) {
-      return new GenericCounter(name, displayName, value);
-    }
-
-    @Override
-    protected TezCounter newCounter() {
-      return new GenericCounter();
-    }
-
-    @Override
-    public CounterGroupBase<TezCounter> getUnderlyingGroup() {
-      return this;
-    }
-  }
-
-  // Mix file system group implementation into the CounterGroup interface
-  private static class FileSystemGroup extends FileSystemCounterGroup<TezCounter>
-      implements CounterGroup {
-
-    @Override
-    protected TezCounter newCounter(String scheme, FileSystemCounter key) {
-      return new FSCounter(scheme, key);
-    }
-
-    @Override
-    public CounterGroupBase<TezCounter> getUnderlyingGroup() {
-      return this;
-    }
-  }
-
-  /**
-   * Provide factory methods for counter group factory implementation.
-   * See also the GroupFactory in
-   *  {@link org.apache.hadoop.TezCounters.Counters mapred.Counters}
-   */
-  private static class GroupFactory
-      extends CounterGroupFactory<TezCounter, CounterGroup> {
-
-    @Override
-    protected <T extends Enum<T>>
-    FrameworkGroupFactory<CounterGroup>
-        newFrameworkGroupFactory(final Class<T> cls) {
-      return new FrameworkGroupFactory<CounterGroup>() {
-        @Override public CounterGroup newGroup(String name) {
-          return new FrameworkGroupImpl<T>(cls); // impl in this package
-        }
-      };
-    }
-
-    @Override
-    protected CounterGroup newGenericGroup(String name, String displayName,
-                                           Limits limits) {
-      return new GenericGroup(name, displayName, limits);
-    }
-
-    @Override
-    protected CounterGroup newFileSystemGroup() {
-      return new FileSystemGroup();
-    }
-  }
-
-  private static final GroupFactory groupFactory = new GroupFactory();
-
-  /**
-   * Default constructor
-   */
-  public TezCounters() {
-    super(groupFactory);
-  }
-
-  /**
-   * Construct the Counters object from the another counters object
-   * @param <C> the type of counter
-   * @param <G> the type of counter group
-   * @param counters the old counters object
-   */
-  public <C extends TezCounter, G extends CounterGroupBase<C>>
-  TezCounters(AbstractCounters<C, G> counters) {
-    super(counters, groupFactory);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-common/src/main/java/org/apache/tez/records/TezContainerId.java
----------------------------------------------------------------------
diff --git a/tez-common/src/main/java/org/apache/tez/records/TezContainerId.java b/tez-common/src/main/java/org/apache/tez/records/TezContainerId.java
deleted file mode 100644
index 193b979..0000000
--- a/tez-common/src/main/java/org/apache/tez/records/TezContainerId.java
+++ /dev/null
@@ -1,78 +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.records;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
-import org.apache.hadoop.yarn.api.records.ApplicationId;
-import org.apache.hadoop.yarn.api.records.ContainerId;
-
-//TODO EVENTUALLY Once everything is on PB, get rid of this.
-//Alternately have the PB interfaces implement Writable.
-@InterfaceAudience.Private
-@InterfaceStability.Unstable
-public class TezContainerId implements Writable {
-
-  private ContainerId containerId;
-
-  public TezContainerId() {
-  }
-  
-  public TezContainerId(ContainerId containerId) {
-    this.containerId = containerId;
-  }
-
-  @Override
-  public void write(DataOutput out) throws IOException {
-    out.writeLong(containerId.getApplicationAttemptId().getApplicationId()
-        .getClusterTimestamp());
-    out.writeInt(containerId.getApplicationAttemptId().getApplicationId()
-        .getId());
-    out.writeInt(containerId.getApplicationAttemptId().getAttemptId());
-    out.writeInt(containerId.getId());
-  }
-
-  @Override
-  public void readFields(DataInput in) throws IOException {
-    long timestamp = in.readLong();
-    int appId = in.readInt();
-    int appAttemptId = in.readInt();
-    int id = in.readInt();
-
-    ApplicationId applicationId = ApplicationId.newInstance(timestamp, appId);
-    ApplicationAttemptId applicationAttemptId = ApplicationAttemptId
-        .newInstance(applicationId, appAttemptId);
-
-    this.containerId = ContainerId.newInstance(applicationAttemptId, id);
-  }
-
-  @Override
-  public String toString() {
-    return containerId.toString();
-  }
-  
-  public ContainerId getContainerId() {
-    return this.containerId;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-dag-api/findbugs-exclude.xml
----------------------------------------------------------------------
diff --git a/tez-dag-api/findbugs-exclude.xml b/tez-dag-api/findbugs-exclude.xml
deleted file mode 100644
index 5b11308..0000000
--- a/tez-dag-api/findbugs-exclude.xml
+++ /dev/null
@@ -1,16 +0,0 @@
-<!--
-  Licensed 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. See accompanying LICENSE file.
--->
-<FindBugsFilter>
-
-</FindBugsFilter>

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-dag-api/pom.xml
----------------------------------------------------------------------
diff --git a/tez-dag-api/pom.xml b/tez-dag-api/pom.xml
deleted file mode 100644
index 51d6c73..0000000
--- a/tez-dag-api/pom.xml
+++ /dev/null
@@ -1,88 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-  Licensed 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. See accompanying LICENSE file.
--->
-
-<project xmlns="http://maven.apache.org/POM/4.0.0"
-         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
-  <modelVersion>4.0.0</modelVersion>
-  <parent>
-    <groupId>org.apache.tez</groupId>
-    <artifactId>tez</artifactId>
-    <version>0.2.0-SNAPSHOT</version>
-  </parent>
-  <artifactId>tez-dag-api</artifactId>
-  <dependencies>
-    <dependency>
-      <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-common</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-yarn-api</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-yarn-client</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-yarn-common</artifactId>
-    </dependency>
-    <dependency>
-     <groupId>com.google.protobuf</groupId>
-     <artifactId>protobuf-java</artifactId>
-    </dependency>
-  </dependencies>
-
-  <build>
-    <plugins>
-      <plugin>
-        <groupId>org.apache.rat</groupId>
-        <artifactId>apache-rat-plugin</artifactId>
-        <configuration>
-        </configuration>
-      </plugin>
-      <plugin>
-        <groupId>org.apache.hadoop</groupId>
-        <artifactId>hadoop-maven-plugins</artifactId>
-        <executions>
-          <execution>
-            <id>compile-protoc</id>
-            <phase>generate-sources</phase>
-            <goals>
-              <goal>protoc</goal>
-            </goals>
-            <configuration>
-              <protocVersion>${protobuf.version}</protocVersion>
-              <protocCommand>${protoc.path}</protocCommand>
-              <imports>
-                <param>${basedir}/src/main/proto</param>
-              </imports>
-              <source>
-                <directory>${basedir}/src/main/proto</directory>
-                <includes>
-                  <include>DAGApiRecords.proto</include>
-                  <include>DAGClientAMProtocol.proto</include>
-                </includes>
-              </source>
-              <output>${project.build.directory}/generated-sources/java</output>
-            </configuration>
-          </execution>
-        </executions>
-      </plugin>
-    </plugins>
-  </build>
-
-</project>

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-dag-api/src/main/java/org/apache/tez/client/AMConfiguration.java
----------------------------------------------------------------------
diff --git a/tez-dag-api/src/main/java/org/apache/tez/client/AMConfiguration.java b/tez-dag-api/src/main/java/org/apache/tez/client/AMConfiguration.java
deleted file mode 100644
index f452c74..0000000
--- a/tez-dag-api/src/main/java/org/apache/tez/client/AMConfiguration.java
+++ /dev/null
@@ -1,100 +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.client;
-
-import java.io.IOException;
-import java.util.HashMap;
-import java.util.Map;
-
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.security.Credentials;
-import org.apache.hadoop.yarn.api.records.LocalResource;
-import org.apache.tez.dag.api.TezConfiguration;
-import org.apache.tez.dag.api.TezUncheckedException;
-
-public class AMConfiguration {
-
-  private final Path stagingDir;
-  private final String queueName;
-  private final Map<String, String> env;
-  private final Map<String, LocalResource> localResources;
-  private final TezConfiguration amConf;
-  private final Credentials credentials;
-
-  public AMConfiguration(String queueName, Map<String, String> env,
-      Map<String, LocalResource> localResources,
-      TezConfiguration conf, Credentials credentials) {
-    this.queueName = queueName;
-    if (conf != null) {
-      this.amConf = conf;
-    } else {
-      this.amConf = new TezConfiguration();
-    }
-
-    if (env != null) {
-      this.env = env;
-    } else {
-      this.env = new HashMap<String, String>(0);
-    }
-    this.localResources = localResources;
-    String stagingDirStr = amConf.get(TezConfiguration.TEZ_AM_STAGING_DIR);
-    if (stagingDirStr == null || stagingDirStr.isEmpty()) {
-      throw new TezUncheckedException("Staging directory for AM resources"
-          + " not specified in config"
-          + ", property=" + TezConfiguration.TEZ_AM_STAGING_DIR);
-    }
-    try {
-      FileSystem fs = FileSystem.get(amConf);
-      this.stagingDir = fs.resolvePath(new Path(stagingDirStr));
-    } catch (IOException e) {
-      throw new TezUncheckedException(e);
-    }
-    this.credentials = credentials;
-  }
-
-  public Path getStagingDir() {
-    return stagingDir;
-  }
-
-  public String getQueueName() {
-    return queueName;
-  }
-
-  public Map<String, String> getEnv() {
-    return env;
-  }
-
-  public Map<String, LocalResource> getLocalResources() {
-    return localResources;
-  }
-
-  public TezConfiguration getAMConf() {
-    return amConf;
-  }
-
-  public Credentials getCredentials() {
-    return credentials;
-  }
-
-  public void isCompatible(AMConfiguration other) {
-    // TODO implement
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-dag-api/src/main/java/org/apache/tez/client/TezClient.java
----------------------------------------------------------------------
diff --git a/tez-dag-api/src/main/java/org/apache/tez/client/TezClient.java b/tez-dag-api/src/main/java/org/apache/tez/client/TezClient.java
deleted file mode 100644
index df260ec..0000000
--- a/tez-dag-api/src/main/java/org/apache/tez/client/TezClient.java
+++ /dev/null
@@ -1,144 +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.client;
-
-import java.io.IOException;
-import java.text.NumberFormat;
-import java.util.Map;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.classification.InterfaceAudience.Private;
-import org.apache.hadoop.yarn.api.records.ApplicationId;
-import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
-import org.apache.hadoop.yarn.api.records.LocalResource;
-import org.apache.hadoop.yarn.client.api.YarnClient;
-import org.apache.hadoop.yarn.client.api.impl.YarnClientImpl;
-import org.apache.hadoop.yarn.conf.YarnConfiguration;
-import org.apache.hadoop.yarn.exceptions.YarnException;
-import org.apache.tez.dag.api.DAG;
-import org.apache.tez.dag.api.TezConfiguration;
-import org.apache.tez.dag.api.TezException;
-import org.apache.tez.dag.api.client.DAGClient;
-import org.apache.tez.dag.api.client.rpc.DAGClientRPCImpl;
-
-public class TezClient {
-  private static final Log LOG = LogFactory.getLog(TezClient.class);
-
-  private final TezConfiguration conf;
-  private final YarnConfiguration yarnConf;
-  private YarnClient yarnClient;
-  Map<String, LocalResource> tezJarResources = null;
-
-  /**
-   * <p>
-   * Create an instance of the TezClient which will be used to communicate with
-   * a specific instance of YARN, or TezService when that exists.
-   * </p>
-   * <p>
-   * Separate instances of TezClient should be created to communicate with
-   * different instances of YARN
-   * </p>
-   *
-   * @param conf
-   *          the configuration which will be used to establish which YARN or
-   *          Tez service instance this client is associated with.
-   */
-  public TezClient(TezConfiguration conf) {
-    this.conf = conf;
-    this.yarnConf = new YarnConfiguration(conf);
-    yarnClient = new YarnClientImpl();
-    yarnClient.init(yarnConf);
-    yarnClient.start();
-  }
-
-
-  public DAGClient submitDAGApplication(DAG dag, AMConfiguration amConfig)
-      throws TezException, IOException {
-    ApplicationId appId = createApplication();
-    return submitDAGApplication(appId, dag, amConfig);
-  }
-
-  @Private
-  // To be used only by YarnRunner
-  public DAGClient submitDAGApplication(ApplicationId appId,
-      DAG dag, AMConfiguration amConfig)
-          throws TezException, IOException {
-    try {
-      ApplicationSubmissionContext appContext =
-          TezClientUtils.createApplicationSubmissionContext(conf, appId, dag,
-              dag.getName(), amConfig, getTezJarResources());
-      LOG.info("Submitting DAG to YARN"
-          + ", applicationId=" + appId);
-      yarnClient.submitApplication(appContext);
-    } catch (YarnException e) {
-      throw new TezException(e);
-    }
-    return getDAGClient(appId);
-  }
-
-  /**
-   * Create a new YARN application
-   * @return <code>ApplicationId</code> for the new YARN application
-   * @throws YarnException
-   * @throws IOException
-   */
-  public ApplicationId createApplication() throws TezException, IOException {
-    try {
-      return yarnClient.createApplication().
-          getNewApplicationResponse().getApplicationId();
-    } catch (YarnException e) {
-      throw new TezException(e);
-    }
-  }
-
-  private synchronized Map<String, LocalResource> getTezJarResources()
-      throws IOException {
-    if (tezJarResources == null) {
-      tezJarResources = TezClientUtils.setupTezJarsLocalResources(conf);
-    }
-    return tezJarResources;
-  }
-
-  @Private
-  public DAGClient getDAGClient(ApplicationId appId)
-      throws IOException, TezException {
-      return new DAGClientRPCImpl(appId, getDefaultTezDAGID(appId),
-                                   conf);
-  }
-
-  // DO NOT CHANGE THIS. This code is replicated from TezDAGID.java
-  private static final char SEPARATOR = '_';
-  private static final String DAG = "dag";
-  private static final NumberFormat idFormat = NumberFormat.getInstance();
-  static {
-    idFormat.setGroupingUsed(false);
-    idFormat.setMinimumIntegerDigits(6);
-  }
-
-  String getDefaultTezDAGID(ApplicationId appId) {
-     return (new StringBuilder(DAG)).append(SEPARATOR).
-                   append(appId.getClusterTimestamp()).
-                   append(SEPARATOR).
-                   append(appId.getId()).
-                   append(SEPARATOR).
-                   append(idFormat.format(1)).toString();
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-dag-api/src/main/java/org/apache/tez/client/TezClientUtils.java
----------------------------------------------------------------------
diff --git a/tez-dag-api/src/main/java/org/apache/tez/client/TezClientUtils.java b/tez-dag-api/src/main/java/org/apache/tez/client/TezClientUtils.java
deleted file mode 100644
index 7c6a5ed..0000000
--- a/tez-dag-api/src/main/java/org/apache/tez/client/TezClientUtils.java
+++ /dev/null
@@ -1,560 +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.client;
-
-import java.io.File;
-import java.io.IOException;
-import java.net.InetSocketAddress;
-import java.net.URI;
-import java.net.URISyntaxException;
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.TreeMap;
-import java.util.Vector;
-import java.util.Map.Entry;
-
-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.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.LocatedFileStatus;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.RemoteIterator;
-import org.apache.hadoop.fs.permission.FsPermission;
-import org.apache.hadoop.io.DataOutputBuffer;
-import org.apache.hadoop.ipc.ProtobufRpcEngine;
-import org.apache.hadoop.ipc.RPC;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.yarn.api.ApplicationConstants;
-import org.apache.hadoop.yarn.api.ApplicationConstants.Environment;
-import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
-import org.apache.hadoop.yarn.api.records.ApplicationId;
-import org.apache.hadoop.yarn.api.records.ApplicationReport;
-import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
-import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
-import org.apache.hadoop.yarn.api.records.LocalResource;
-import org.apache.hadoop.yarn.api.records.LocalResourceType;
-import org.apache.hadoop.yarn.api.records.LocalResourceVisibility;
-import org.apache.hadoop.yarn.api.records.Resource;
-import org.apache.hadoop.yarn.api.records.YarnApplicationState;
-import org.apache.hadoop.yarn.client.api.YarnClient;
-import org.apache.hadoop.yarn.conf.YarnConfiguration;
-import org.apache.hadoop.yarn.exceptions.YarnException;
-import org.apache.hadoop.yarn.util.Apps;
-import org.apache.hadoop.yarn.util.ConverterUtils;
-import org.apache.hadoop.yarn.util.Records;
-import org.apache.log4j.Level;
-import org.apache.tez.dag.api.DAG;
-import org.apache.tez.dag.api.TezConfiguration;
-import org.apache.tez.dag.api.TezConstants;
-import org.apache.tez.dag.api.TezException;
-import org.apache.tez.dag.api.TezUncheckedException;
-import org.apache.tez.dag.api.Vertex;
-import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolBlockingPB;
-import org.apache.tez.dag.api.records.DAGProtos.ConfigurationProto;
-import org.apache.tez.dag.api.records.DAGProtos.DAGPlan;
-import org.apache.tez.dag.api.records.DAGProtos.PlanKeyValuePair;
-
-import com.google.common.annotations.VisibleForTesting;
-
-public class TezClientUtils {
-
-  private static Log LOG = LogFactory.getLog(TezClientUtils.class);
-
-  public static final FsPermission TEZ_AM_DIR_PERMISSION =
-      FsPermission.createImmutable((short) 0700); // rwx--------
-  public static final FsPermission TEZ_AM_FILE_PERMISSION =
-      FsPermission.createImmutable((short) 0644); // rw-r--r--
-
-  private static final int UTF8_CHUNK_SIZE = 16 * 1024;
-
-  /**
-   * Setup LocalResource map for Tez jars based on provided Configuration
-   * @param conf Configuration to use to access Tez jars' locations
-   * @return Map of LocalResources to use when launching Tez AM
-   * @throws IOException
-   */
-  static Map<String, LocalResource> setupTezJarsLocalResources(
-      TezConfiguration conf)
-      throws IOException {
-    Map<String, LocalResource> tezJarResources =
-        new TreeMap<String, LocalResource>();
-    if (conf.getBoolean(YarnConfiguration.IS_MINI_YARN_CLUSTER, false)) {
-      return tezJarResources;
-    }
-
-    // Add tez jars to local resource
-    String[] tezJarUris = conf.getStrings(
-        TezConfiguration.TEZ_LIB_URIS);
-    if (tezJarUris == null
-        || tezJarUris.length == 0) {
-      throw new TezUncheckedException("Invalid configuration of tez jars"
-          + ", " + TezConfiguration.TEZ_LIB_URIS
-          + " is not defined in the configurartion");
-    }
-
-    for (String tezJarUri : tezJarUris) {
-      URI uri;
-      try {
-        uri = new URI(tezJarUri.trim());
-      } catch (URISyntaxException e) {
-        String message = "Invalid URI defined in configuration for"
-            + " location of TEZ jars. providedURI=" + tezJarUri;
-        LOG.error(message);
-        throw new TezUncheckedException(message, e);
-      }
-      if (!uri.isAbsolute()) {
-        String message = "Non-absolute URI defined in configuration for"
-            + " location of TEZ jars. providedURI=" + tezJarUri;
-        LOG.error(message);
-        throw new TezUncheckedException(message);
-      }
-      Path p = new Path(uri);
-      FileSystem pathfs = p.getFileSystem(conf);
-      RemoteIterator<LocatedFileStatus> iter = pathfs.listFiles(p, false);
-      while (iter.hasNext()) {
-        LocatedFileStatus fStatus = iter.next();
-        String rsrcName = fStatus.getPath().getName();
-        // FIXME currently not checking for duplicates due to quirks
-        // in assembly generation
-        if (tezJarResources.containsKey(rsrcName)) {
-          String message = "Duplicate resource found"
-              + ", resourceName=" + rsrcName
-              + ", existingPath=" +
-              tezJarResources.get(rsrcName).getResource().toString()
-              + ", newPath=" + fStatus.getPath();
-          LOG.warn(message);
-          // throw new TezUncheckedException(message);
-        }
-        tezJarResources.put(rsrcName,
-            LocalResource.newInstance(
-                ConverterUtils.getYarnUrlFromPath(fStatus.getPath()),
-                LocalResourceType.FILE,
-                LocalResourceVisibility.PUBLIC,
-                fStatus.getLen(),
-                fStatus.getModificationTime()));
-      }
-    }
-    if (tezJarResources.isEmpty()) {
-      LOG.warn("No tez jars found in configured locations"
-          + ". Ignoring for now. Errors may occur");
-    }
-    return tezJarResources;
-  }
-
-  /**
-   * Verify or create the Staging area directory on the configured Filesystem
-   * @param stagingArea Staging area directory path
-   * @return
-   * @throws IOException
-   */
-  public static FileSystem ensureStagingDirExists(Configuration conf,
-      Path stagingArea)
-      throws IOException {
-    FileSystem fs = stagingArea.getFileSystem(conf);
-    String realUser;
-    String currentUser;
-    UserGroupInformation ugi = UserGroupInformation.getLoginUser();
-    realUser = ugi.getShortUserName();
-    currentUser = UserGroupInformation.getCurrentUser().getShortUserName();
-    if (fs.exists(stagingArea)) {
-      FileStatus fsStatus = fs.getFileStatus(stagingArea);
-      String owner = fsStatus.getOwner();
-      if (!(owner.equals(currentUser) || owner.equals(realUser))) {
-        throw new IOException("The ownership on the staging directory "
-            + stagingArea + " is not as expected. " + "It is owned by " + owner
-            + ". The directory must " + "be owned by the submitter "
-            + currentUser + " or " + "by " + realUser);
-      }
-      if (!fsStatus.getPermission().equals(TEZ_AM_DIR_PERMISSION)) {
-        LOG.info("Permissions on staging directory " + stagingArea + " are "
-            + "incorrect: " + fsStatus.getPermission()
-            + ". Fixing permissions " + "to correct value "
-            + TEZ_AM_DIR_PERMISSION);
-        fs.setPermission(stagingArea, TEZ_AM_DIR_PERMISSION);
-      }
-    } else {
-      fs.mkdirs(stagingArea, new FsPermission(TEZ_AM_DIR_PERMISSION));
-    }
-    return fs;
-  }
-
-  /**
-   * Create an ApplicationSubmissionContext to launch a Tez AM
-   * @param conf
-   * @param appId
-   * @param dag
-   * @param appStagingDir
-   * @param ts
-   * @param amQueueName
-   * @param amName
-   * @param amArgs
-   * @param amEnv
-   * @param amLocalResources
-   * @param appConf
-   * @return
-   * @throws IOException
-   * @throws YarnException
-   */
-  static ApplicationSubmissionContext createApplicationSubmissionContext(
-      Configuration conf, ApplicationId appId, DAG dag, String amName,
-      AMConfiguration amConfig,
-      Map<String, LocalResource> tezJarResources)
-          throws IOException, YarnException{
-
-    FileSystem fs = TezClientUtils.ensureStagingDirExists(conf,
-        amConfig.getStagingDir());
-
-    // Setup resource requirements
-    Resource capability = Records.newRecord(Resource.class);
-    capability.setMemory(
-        amConfig.getAMConf().getInt(TezConfiguration.TEZ_AM_RESOURCE_MEMORY_MB,
-            TezConfiguration.TEZ_AM_RESOURCE_MEMORY_MB_DEFAULT));
-    capability.setVirtualCores(
-        amConfig.getAMConf().getInt(TezConfiguration.TEZ_AM_RESOURCE_CPU_VCORES,
-            TezConfiguration.TEZ_AM_RESOURCE_CPU_VCORES_DEFAULT));
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("AppMaster capability = " + capability);
-    }
-
-    ByteBuffer securityTokens = null;
-    // Setup security tokens
-    if (amConfig.getCredentials() != null) {
-      DataOutputBuffer dob = new DataOutputBuffer();
-      amConfig.getCredentials().writeTokenStorageToStream(dob);
-      securityTokens = ByteBuffer.wrap(dob.getData(), 0,
-          dob.getLength());
-    }
-
-    // Setup the command to run the AM
-    List<String> vargs = new ArrayList<String>(8);
-    vargs.add(Environment.JAVA_HOME.$() + "/bin/java");
-
-    String amLogLevel = amConfig.getAMConf().get(
-        TezConfiguration.TEZ_AM_LOG_LEVEL,
-        TezConfiguration.TEZ_AM_LOG_LEVEL_DEFAULT);
-    addLog4jSystemProperties(amLogLevel, vargs);
-
-    vargs.add(amConfig.getAMConf().get(TezConfiguration.TEZ_AM_JAVA_OPTS,
-        TezConfiguration.DEFAULT_TEZ_AM_JAVA_OPTS));
-
-    vargs.add(TezConfiguration.TEZ_APPLICATION_MASTER_CLASS);
-    vargs.add("1>" + ApplicationConstants.LOG_DIR_EXPANSION_VAR +
-        File.separator + ApplicationConstants.STDOUT);
-    vargs.add("2>" + ApplicationConstants.LOG_DIR_EXPANSION_VAR +
-        File.separator + ApplicationConstants.STDERR);
-
-
-    Vector<String> vargsFinal = new Vector<String>(8);
-    // Final command
-    StringBuilder mergedCommand = new StringBuilder();
-    for (CharSequence str : vargs) {
-      mergedCommand.append(str).append(" ");
-    }
-    vargsFinal.add(mergedCommand.toString());
-
-    LOG.debug("Command to launch container for ApplicationMaster is : "
-        + mergedCommand);
-
-    // Setup the CLASSPATH in environment
-    // i.e. add { Hadoop jars, job jar, CWD } to classpath.
-    Map<String, String> environment = new HashMap<String, String>();
-
-    boolean isMiniCluster =
-        conf.getBoolean(YarnConfiguration.IS_MINI_YARN_CLUSTER, false);
-    if (isMiniCluster) {
-      Apps.addToEnvironment(environment, Environment.CLASSPATH.name(),
-          System.getProperty("java.class.path"));
-    }
-
-    Apps.addToEnvironment(environment,
-        Environment.CLASSPATH.name(),
-        Environment.PWD.$());
-
-    Apps.addToEnvironment(environment,
-        Environment.CLASSPATH.name(),
-        Environment.PWD.$() + File.separator + "*");
-
-    // Add YARN/COMMON/HDFS jars to path
-    if (!isMiniCluster) {
-      for (String c : conf.getStrings(
-          YarnConfiguration.YARN_APPLICATION_CLASSPATH,
-          YarnConfiguration.DEFAULT_YARN_APPLICATION_CLASSPATH)) {
-        Apps.addToEnvironment(environment, Environment.CLASSPATH.name(),
-            c.trim());
-      }
-    }
-
-    if (amConfig.getEnv() != null) {
-      for (Map.Entry<String, String> entry : amConfig.getEnv().entrySet()) {
-        Apps.addToEnvironment(environment, entry.getKey(), entry.getValue());
-      }
-    }
-
-    Map<String, LocalResource> localResources =
-        new TreeMap<String, LocalResource>();
-
-    if (amConfig.getLocalResources() != null) {
-      localResources.putAll(amConfig.getLocalResources());
-    }
-    localResources.putAll(tezJarResources);
-
-    // emit conf as PB file
-    Configuration finalTezConf = createFinalTezConfForApp(amConfig.getAMConf());
-    Path binaryConfPath =  new Path(amConfig.getStagingDir(),
-        TezConfiguration.TEZ_PB_BINARY_CONF_NAME + "." + appId.toString());
-    FSDataOutputStream amConfPBOutBinaryStream = null;
-    try {
-      ConfigurationProto.Builder confProtoBuilder =
-          ConfigurationProto.newBuilder();
-      Iterator<Entry<String, String>> iter = finalTezConf.iterator();
-      while (iter.hasNext()) {
-        Entry<String, String> entry = iter.next();
-        PlanKeyValuePair.Builder kvp = PlanKeyValuePair.newBuilder();
-        kvp.setKey(entry.getKey());
-        kvp.setValue(entry.getValue());
-        confProtoBuilder.addConfKeyValues(kvp);
-      }
-      //binary output
-      amConfPBOutBinaryStream = FileSystem.create(fs, binaryConfPath,
-          new FsPermission(TEZ_AM_FILE_PERMISSION));
-      confProtoBuilder.build().writeTo(amConfPBOutBinaryStream);
-    } finally {
-      if(amConfPBOutBinaryStream != null){
-        amConfPBOutBinaryStream.close();
-      }
-    }
-
-    LocalResource binaryConfLRsrc =
-        TezClientUtils.createLocalResource(fs,
-            binaryConfPath, LocalResourceType.FILE,
-            LocalResourceVisibility.APPLICATION);
-    localResources.put(TezConfiguration.TEZ_PB_BINARY_CONF_NAME,
-        binaryConfLRsrc);
-
-    if(dag != null) {
-      // Add tez jars to vertices too
-      for (Vertex v : dag.getVertices()) {
-        v.getTaskLocalResources().putAll(tezJarResources);
-        v.getTaskLocalResources().put(TezConfiguration.TEZ_PB_BINARY_CONF_NAME,
-            binaryConfLRsrc);
-      }
-
-      // emit protobuf DAG file style
-      Path binaryPath =  new Path(amConfig.getStagingDir(),
-          TezConfiguration.TEZ_PB_PLAN_BINARY_NAME + "." + appId.toString());
-      amConfig.getAMConf().set(TezConfiguration.TEZ_AM_PLAN_REMOTE_PATH,
-          binaryPath.toUri().toString());
-
-      DAGPlan dagPB = dag.createDag(null);
-
-      FSDataOutputStream dagPBOutBinaryStream = null;
-
-      try {
-        //binary output
-        dagPBOutBinaryStream = FileSystem.create(fs, binaryPath,
-            new FsPermission(TEZ_AM_FILE_PERMISSION));
-        dagPB.writeTo(dagPBOutBinaryStream);
-      } finally {
-        if(dagPBOutBinaryStream != null){
-          dagPBOutBinaryStream.close();
-        }
-      }
-
-      localResources.put(TezConfiguration.TEZ_PB_PLAN_BINARY_NAME,
-          TezClientUtils.createLocalResource(fs,
-              binaryPath, LocalResourceType.FILE,
-              LocalResourceVisibility.APPLICATION));
-
-      if (Level.DEBUG.isGreaterOrEqual(Level.toLevel(amLogLevel))) {
-        Path textPath = localizeDagPlanAsText(dagPB, fs,
-            amConfig.getStagingDir(), appId);
-        localResources.put(TezConfiguration.TEZ_PB_PLAN_TEXT_NAME,
-            TezClientUtils.createLocalResource(fs,
-                textPath, LocalResourceType.FILE,
-                LocalResourceVisibility.APPLICATION));
-      }
-    } else {
-      Apps.addToEnvironment(environment,
-          TezConstants.TEZ_AM_IS_SESSION_ENV, "set");
-    }
-
-    Map<ApplicationAccessType, String> acls
-        = new HashMap<ApplicationAccessType, String>();
-
-    // Setup ContainerLaunchContext for AM container
-    ContainerLaunchContext amContainer =
-        ContainerLaunchContext.newInstance(localResources, environment,
-            vargsFinal, null, securityTokens, acls);
-
-    // Set up the ApplicationSubmissionContext
-    ApplicationSubmissionContext appContext = Records
-        .newRecord(ApplicationSubmissionContext.class);
-
-    appContext.setApplicationType(TezConfiguration.TEZ_APPLICATION_TYPE);
-    appContext.setApplicationId(appId);
-    appContext.setResource(capability);
-    appContext.setQueue(amConfig.getQueueName());
-    appContext.setApplicationName(amName);
-    appContext.setCancelTokensWhenComplete(amConfig.getAMConf().getBoolean(
-        TezConfiguration.TEZ_AM_CANCEL_DELEGATION_TOKEN,
-        TezConfiguration.TEZ_AM_CANCEL_DELEGATION_TOKEN_DEFAULT));
-    appContext.setAMContainerSpec(amContainer);
-
-    return appContext;
-
-  }
-
-  @VisibleForTesting
-  static void addLog4jSystemProperties(String logLevel,
-      List<String> vargs) {
-    vargs.add("-Dlog4j.configuration="
-        + TezConfiguration.TEZ_CONTAINER_LOG4J_PROPERTIES_FILE);
-    vargs.add("-D" + YarnConfiguration.YARN_APP_CONTAINER_LOG_DIR + "="
-        + ApplicationConstants.LOG_DIR_EXPANSION_VAR);
-    vargs.add("-D" + TezConfiguration.TEZ_ROOT_LOGGER_NAME + "=" + logLevel
-        + "," + TezConfiguration.TEZ_CONTAINER_LOGGER_NAME);
-  }
-
-  static Configuration createFinalTezConfForApp(TezConfiguration amConf) {
-    Configuration conf = new Configuration(false);
-    conf.setQuietMode(true);
-
-    assert amConf != null;
-    Iterator<Entry<String, String>> iter = amConf.iterator();
-    while (iter.hasNext()) {
-      Entry<String, String> entry = iter.next();
-      // Copy all tez config parameters.
-      if (entry.getKey().startsWith(TezConfiguration.TEZ_PREFIX)) {
-        conf.set(entry.getKey(), entry.getValue());
-        if (LOG.isDebugEnabled()) {
-          LOG.debug("Adding tez dag am parameter: " + entry.getKey()
-              + ", with value: " + entry.getValue());
-        }
-      }
-    }
-    return conf;
-  }
-
-  /**
-   * Helper function to create a YARN LocalResource
-   * @param fs FileSystem object
-   * @param p Path of resource to localize
-   * @param type LocalResource Type
-   * @return
-   * @throws IOException
-   */
-  static LocalResource createLocalResource(FileSystem fs, Path p,
-      LocalResourceType type,
-      LocalResourceVisibility visibility) throws IOException {
-    LocalResource rsrc = Records.newRecord(LocalResource.class);
-    FileStatus rsrcStat = fs.getFileStatus(p);
-    rsrc.setResource(ConverterUtils.getYarnUrlFromPath(fs.resolvePath(rsrcStat
-        .getPath())));
-    rsrc.setSize(rsrcStat.getLen());
-    rsrc.setTimestamp(rsrcStat.getModificationTime());
-    rsrc.setType(type);
-    rsrc.setVisibility(visibility);
-    return rsrc;
-  }
-
-  private static Path localizeDagPlanAsText(DAGPlan dagPB, FileSystem fs,
-      Path appStagingDir, ApplicationId appId) throws IOException {
-    Path textPath = new Path(appStagingDir,
-        TezConfiguration.TEZ_PB_PLAN_TEXT_NAME + "." + appId.toString());
-    FSDataOutputStream dagPBOutTextStream = null;
-    try {
-      dagPBOutTextStream = FileSystem.create(fs, textPath, new FsPermission(
-          TEZ_AM_FILE_PERMISSION));
-      String dagPBStr = dagPB.toString();
-      int dagPBStrLen = dagPBStr.length();
-      if (dagPBStrLen <= UTF8_CHUNK_SIZE) {
-        dagPBOutTextStream.writeUTF(dagPBStr);
-      } else {
-        int startIndex = 0;
-        while (startIndex < dagPBStrLen) {
-          int endIndex = startIndex + UTF8_CHUNK_SIZE;
-          if (endIndex > dagPBStrLen) {
-            endIndex = dagPBStrLen;
-          }
-          dagPBOutTextStream.writeUTF(dagPBStr.substring(startIndex, endIndex));
-          startIndex += UTF8_CHUNK_SIZE;
-        }
-      }
-    } finally {
-      if (dagPBOutTextStream != null) {
-        dagPBOutTextStream.close();
-      }
-    }
-    return textPath;
-  }
-
-  static DAGClientAMProtocolBlockingPB getAMProxy(YarnClient yarnClient,
-      Configuration conf,
-      ApplicationId applicationId) throws TezException, IOException {
-    ApplicationReport appReport;
-    try {
-      appReport = yarnClient.getApplicationReport(
-          applicationId);
-
-      if(appReport == null) {
-        throw new TezUncheckedException("Could not retrieve application report"
-            + " from YARN, applicationId=" + applicationId);
-      }
-      YarnApplicationState appState = appReport.getYarnApplicationState();
-      if(appState != YarnApplicationState.RUNNING) {
-        if (appState == YarnApplicationState.FINISHED
-            || appState == YarnApplicationState.KILLED
-            || appState == YarnApplicationState.FAILED) {
-          throw new TezUncheckedException("Application not running"
-              + ", applicationId=" + applicationId
-              + ", yarnApplicationState=" + appReport.getYarnApplicationState()
-              + ", finalApplicationStatus="
-              + appReport.getFinalApplicationStatus()
-              + ", trackingUrl=" + appReport.getTrackingUrl());
-        }
-        return null;
-      }
-    } catch (YarnException e) {
-      throw new TezException(e);
-    }
-    return getAMProxy(conf, appReport.getHost(), appReport.getRpcPort());
-  }
-
-  static DAGClientAMProtocolBlockingPB getAMProxy(Configuration conf,
-      String amHost, int amRpcPort) throws IOException {
-    InetSocketAddress addr = new InetSocketAddress(amHost,
-        amRpcPort);
-
-    RPC.setProtocolEngine(conf, DAGClientAMProtocolBlockingPB.class,
-        ProtobufRpcEngine.class);
-    DAGClientAMProtocolBlockingPB proxy =
-        (DAGClientAMProtocolBlockingPB) RPC.getProxy(
-            DAGClientAMProtocolBlockingPB.class, 0, addr, conf);
-    return proxy;
-  }
-
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-dag-api/src/main/java/org/apache/tez/client/TezSession.java
----------------------------------------------------------------------
diff --git a/tez-dag-api/src/main/java/org/apache/tez/client/TezSession.java b/tez-dag-api/src/main/java/org/apache/tez/client/TezSession.java
deleted file mode 100644
index acf523d..0000000
--- a/tez-dag-api/src/main/java/org/apache/tez/client/TezSession.java
+++ /dev/null
@@ -1,184 +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.client;
-
-import java.io.IOException;
-import java.util.Map;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.classification.InterfaceAudience.Private;
-import org.apache.hadoop.yarn.api.records.ApplicationId;
-import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
-import org.apache.hadoop.yarn.api.records.LocalResource;
-import org.apache.hadoop.yarn.client.api.YarnClient;
-import org.apache.hadoop.yarn.exceptions.YarnException;
-import org.apache.tez.dag.api.DAG;
-import org.apache.tez.dag.api.TezConfiguration;
-import org.apache.tez.dag.api.TezException;
-import org.apache.tez.dag.api.TezUncheckedException;
-import org.apache.tez.dag.api.Vertex;
-import org.apache.tez.dag.api.client.DAGClient;
-import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolBlockingPB;
-import org.apache.tez.dag.api.client.rpc.DAGClientRPCImpl;
-import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolRPC.ShutdownSessionRequestProto;
-import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolRPC.SubmitDAGRequestProto;
-import org.apache.tez.dag.api.records.DAGProtos.DAGPlan;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.protobuf.ServiceException;
-
-public class TezSession {
-
-  private static final Log LOG = LogFactory.getLog(TezSession.class);
-
-  private final String sessionName;
-  private ApplicationId applicationId;
-  private LocalResource tezConfPBLRsrc = null;
-  private final TezSessionConfiguration sessionConfig;
-  private YarnClient yarnClient;
-  private Map<String, LocalResource> tezJarResources;
-  private boolean sessionStarted = false;
-
-  public TezSession(String sessionName,
-      ApplicationId applicationId,
-      TezSessionConfiguration sessionConfig) {
-    this.sessionName = sessionName;
-    this.sessionConfig = sessionConfig;
-    this.applicationId = applicationId;
-  }
-
-  public TezSession(String sessionName,
-      TezSessionConfiguration sessionConfig) {
-    this(sessionName, null, sessionConfig);
-  }
-
-  public synchronized void start() throws TezException, IOException {
-    yarnClient = YarnClient.createYarnClient();
-    yarnClient.init(sessionConfig.getYarnConfiguration());
-    yarnClient.start();
-
-    tezJarResources = TezClientUtils.setupTezJarsLocalResources(
-        sessionConfig.getTezConfiguration());
-
-    try {
-      if (applicationId == null) {
-        applicationId = yarnClient.createApplication().
-            getNewApplicationResponse().getApplicationId();
-      }
-
-      ApplicationSubmissionContext appContext =
-          TezClientUtils.createApplicationSubmissionContext(
-              sessionConfig.getTezConfiguration(), applicationId,
-              null, sessionName, sessionConfig.getAMConfiguration(),
-              tezJarResources);
-      tezConfPBLRsrc = appContext.getAMContainerSpec().getLocalResources().get(
-          TezConfiguration.TEZ_PB_BINARY_CONF_NAME);
-      yarnClient.submitApplication(appContext);
-    } catch (YarnException e) {
-      throw new TezException(e);
-    }
-    sessionStarted = true;
-  }
-
-  public synchronized DAGClient submitDAG(DAG dag)
-      throws TezException, IOException {
-    if (!sessionStarted) {
-      throw new TezUncheckedException("Session not started");
-    }
-
-    String dagId = null;
-    LOG.info("Submitting dag to TezSession"
-        + ", sessionName=" + sessionName
-        + ", applicationId=" + applicationId);
-    // Add tez jars to vertices too
-    for (Vertex v : dag.getVertices()) {
-      v.getTaskLocalResources().putAll(tezJarResources);
-      if (null != tezConfPBLRsrc) {
-        v.getTaskLocalResources().put(TezConfiguration.TEZ_PB_BINARY_CONF_NAME,
-            tezConfPBLRsrc);
-      }
-    }
-    DAGPlan dagPlan = dag.createDag(sessionConfig.getTezConfiguration());
-    SubmitDAGRequestProto requestProto =
-        SubmitDAGRequestProto.newBuilder().setDAGPlan(dagPlan).build();
-
-    DAGClientAMProtocolBlockingPB proxy;
-    while (true) {
-      proxy = TezClientUtils.getAMProxy(yarnClient,
-          sessionConfig.getYarnConfiguration(), applicationId);
-      if (proxy != null) {
-        break;
-      }
-      try {
-        Thread.sleep(100l);
-      } catch (InterruptedException e) {
-        // Ignore
-      }
-    }
-
-    try {
-      dagId = proxy.submitDAG(null, requestProto).getDagId();
-    } catch (ServiceException e) {
-      throw new TezException(e);
-    }
-    LOG.info("Submitted dag to TezSession"
-        + ", sessionName=" + sessionName
-        + ", applicationId=" + applicationId
-        + ", dagId=" + dagId);
-    return new DAGClientRPCImpl(applicationId, dagId,
-        sessionConfig.getTezConfiguration());
-  }
-
-  public synchronized void stop() throws TezException, IOException {
-    LOG.info("Shutting down Tez Session"
-        + ", sessionName=" + sessionName
-        + ", applicationId=" + applicationId);
-    DAGClientAMProtocolBlockingPB proxy = TezClientUtils.getAMProxy(yarnClient,
-        sessionConfig.getYarnConfiguration(), applicationId);
-    if (proxy != null) {
-      try {
-        ShutdownSessionRequestProto request =
-            ShutdownSessionRequestProto.newBuilder().build();
-        proxy.shutdownSession(null, request);
-        return;
-      } catch (ServiceException e) {
-        LOG.info("Failed to shutdown Tez Session via proxy", e);
-      }
-    }
-    LOG.info("Could not connect to AM, killing session via YARN"
-        + ", sessionName=" + sessionName
-        + ", applicationId=" + applicationId);
-    try {
-      yarnClient.killApplication(applicationId);
-    } catch (YarnException e) {
-      throw new TezException(e);
-    }
-  }
-
-  public String getSessionName() {
-    return sessionName;
-  }
-
-  @Private
-  @VisibleForTesting
-  public synchronized ApplicationId getApplicationId() {
-    return applicationId;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-dag-api/src/main/java/org/apache/tez/client/TezSessionConfiguration.java
----------------------------------------------------------------------
diff --git a/tez-dag-api/src/main/java/org/apache/tez/client/TezSessionConfiguration.java b/tez-dag-api/src/main/java/org/apache/tez/client/TezSessionConfiguration.java
deleted file mode 100644
index 61ca60b..0000000
--- a/tez-dag-api/src/main/java/org/apache/tez/client/TezSessionConfiguration.java
+++ /dev/null
@@ -1,57 +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.client;
-
-import org.apache.hadoop.yarn.conf.YarnConfiguration;
-import org.apache.tez.dag.api.TezConfiguration;
-
-public class TezSessionConfiguration {
-
-  private final AMConfiguration amConfiguration;
-  private final YarnConfiguration yarnConfig;
-  private final TezConfiguration tezConfig;
-
-  public TezSessionConfiguration(AMConfiguration amConfiguration,
-      TezConfiguration tezConfig) {
-    this.amConfiguration = amConfiguration;
-    this.tezConfig = tezConfig;
-    this.yarnConfig = new YarnConfiguration(tezConfig);
-  }
-
-  TezSessionConfiguration(AMConfiguration amConfiguration,
-      TezConfiguration tezConfig,
-      YarnConfiguration yarnConf) {
-    this.amConfiguration = amConfiguration;
-    this.tezConfig = tezConfig;
-    this.yarnConfig = yarnConf;
-  }
-
-  public AMConfiguration getAMConfiguration() {
-    return amConfiguration;
-  }
-
-  public YarnConfiguration getYarnConfiguration() {
-    return yarnConfig;
-  }
-
-  public TezConfiguration getTezConfiguration() {
-    return tezConfig;
-  }
-
-}


[36/50] [abbrv] Rename tez-engine-api to tez-runtime-api and tez-engine is split into 2: - tez-engine-library for user-visible Input/Output/Processor implementations - tez-engine-internals for framework internals

Posted by ss...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/hadoop/TestDeprecatedKeys.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/hadoop/TestDeprecatedKeys.java b/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/hadoop/TestDeprecatedKeys.java
index e5bd108..12a3740 100644
--- a/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/hadoop/TestDeprecatedKeys.java
+++ b/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/hadoop/TestDeprecatedKeys.java
@@ -21,8 +21,8 @@ package org.apache.tez.mapreduce.hadoop;
 import static org.junit.Assert.assertEquals;
 
 import org.apache.hadoop.mapred.JobConf;
-import org.apache.tez.common.Constants;
 import org.apache.tez.common.TezJobConfig;
+import org.apache.tez.runtime.library.common.Constants;
 import org.junit.Test;
 
 public class TestDeprecatedKeys {
@@ -43,22 +43,22 @@ public class TestDeprecatedKeys {
     MultiStageMRConfToTezTranslator.translateVertexConfToTez(jobConf, null);
 
     assertEquals(0.4f, jobConf.getFloat(
-        TezJobConfig.TEZ_ENGINE_SHUFFLE_INPUT_BUFFER_PERCENT, 0f), 0.01f);
-    assertEquals(20000l, jobConf.getLong(Constants.TEZ_ENGINE_TASK_MEMORY, 0));
+        TezJobConfig.TEZ_RUNTIME_SHUFFLE_INPUT_BUFFER_PERCENT, 0f), 0.01f);
+    assertEquals(20000l, jobConf.getLong(Constants.TEZ_RUNTIME_TASK_MEMORY, 0));
     assertEquals(2000,
-        jobConf.getInt(TezJobConfig.TEZ_ENGINE_IO_SORT_FACTOR, 0));
+        jobConf.getInt(TezJobConfig.TEZ_RUNTIME_IO_SORT_FACTOR, 0));
     assertEquals(0.55f, jobConf.getFloat(
-        TezJobConfig.TEZ_ENGINE_SHUFFLE_MEMORY_LIMIT_PERCENT, 0), 0.01f);
+        TezJobConfig.TEZ_RUNTIME_SHUFFLE_MEMORY_LIMIT_PERCENT, 0), 0.01f);
     assertEquals(0.60f,
-        jobConf.getFloat(TezJobConfig.TEZ_ENGINE_SHUFFLE_MEMTOMEM_SEGMENTS, 0),
+        jobConf.getFloat(TezJobConfig.TEZ_RUNTIME_SHUFFLE_MEMTOMEM_SEGMENTS, 0),
         0.01f);
     assertEquals(0.22f,
-        jobConf.getFloat(TezJobConfig.TEZ_ENGINE_SHUFFLE_MERGE_PERCENT, 0),
+        jobConf.getFloat(TezJobConfig.TEZ_RUNTIME_SHUFFLE_MERGE_PERCENT, 0),
         0.01f);
     assertEquals(true, jobConf.getBoolean(
-        TezJobConfig.TEZ_ENGINE_SHUFFLE_ENABLE_MEMTOMEM, false));
+        TezJobConfig.TEZ_RUNTIME_SHUFFLE_ENABLE_MEMTOMEM, false));
     assertEquals(0.33f,
-        jobConf.getFloat(TezJobConfig.TEZ_ENGINE_INPUT_BUFFER_PERCENT, 0),
+        jobConf.getFloat(TezJobConfig.TEZ_RUNTIME_INPUT_BUFFER_PERCENT, 0),
         0.01f);
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/processor/MapUtils.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/processor/MapUtils.java b/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/processor/MapUtils.java
index 4b2c0e8..9590e72 100644
--- a/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/processor/MapUtils.java
+++ b/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/processor/MapUtils.java
@@ -48,15 +48,15 @@ import org.apache.hadoop.util.DiskChecker.DiskErrorException;
 import org.apache.tez.common.TezJobConfig;
 import org.apache.tez.common.TezUtils;
 import org.apache.tez.dag.api.ProcessorDescriptor;
-import org.apache.tez.engine.api.impl.InputSpec;
-import org.apache.tez.engine.api.impl.OutputSpec;
-import org.apache.tez.engine.api.impl.TaskSpec;
-import org.apache.tez.engine.api.impl.TezUmbilical;
-import org.apache.tez.engine.common.security.JobTokenIdentifier;
-import org.apache.tez.engine.newruntime.LogicalIOProcessorRuntimeTask;
 import org.apache.tez.mapreduce.TezTestUtils;
 import org.apache.tez.mapreduce.hadoop.MRJobConfig;
 import org.apache.tez.mapreduce.processor.map.MapProcessor;
+import org.apache.tez.runtime.LogicalIOProcessorRuntimeTask;
+import org.apache.tez.runtime.api.impl.InputSpec;
+import org.apache.tez.runtime.api.impl.OutputSpec;
+import org.apache.tez.runtime.api.impl.TaskSpec;
+import org.apache.tez.runtime.api.impl.TezUmbilical;
+import org.apache.tez.runtime.library.common.security.JobTokenIdentifier;
 
 public class MapUtils {
 

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/processor/map/TestMapProcessor.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/processor/map/TestMapProcessor.java b/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/processor/map/TestMapProcessor.java
index 89292ab..5b8eedf 100644
--- a/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/processor/map/TestMapProcessor.java
+++ b/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/processor/map/TestMapProcessor.java
@@ -31,19 +31,9 @@ import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapreduce.MRConfig;
-import org.apache.tez.common.Constants;
 import org.apache.tez.common.TezJobConfig;
 import org.apache.tez.dag.api.InputDescriptor;
 import org.apache.tez.dag.api.OutputDescriptor;
-import org.apache.tez.engine.api.TezInputContext;
-import org.apache.tez.engine.api.impl.InputSpec;
-import org.apache.tez.engine.api.impl.OutputSpec;
-import org.apache.tez.engine.common.InputAttemptIdentifier;
-import org.apache.tez.engine.common.sort.impl.IFile;
-import org.apache.tez.engine.common.task.local.output.TezLocalTaskOutputFiles;
-import org.apache.tez.engine.common.task.local.output.TezTaskOutput;
-import org.apache.tez.engine.lib.output.LocalOnFileSorterOutput;
-import org.apache.tez.engine.newruntime.LogicalIOProcessorRuntimeTask;
 import org.apache.tez.mapreduce.TestUmbilical;
 import org.apache.tez.mapreduce.hadoop.MRJobConfig;
 import org.apache.tez.mapreduce.hadoop.MultiStageMRConfToTezTranslator;
@@ -51,6 +41,16 @@ import org.apache.tez.mapreduce.hadoop.MultiStageMRConfigUtil;
 import org.apache.tez.mapreduce.input.MRInputLegacy;
 import org.apache.tez.mapreduce.partition.MRPartitioner;
 import org.apache.tez.mapreduce.processor.MapUtils;
+import org.apache.tez.runtime.LogicalIOProcessorRuntimeTask;
+import org.apache.tez.runtime.api.TezInputContext;
+import org.apache.tez.runtime.api.impl.InputSpec;
+import org.apache.tez.runtime.api.impl.OutputSpec;
+import org.apache.tez.runtime.library.common.Constants;
+import org.apache.tez.runtime.library.common.InputAttemptIdentifier;
+import org.apache.tez.runtime.library.common.sort.impl.IFile;
+import org.apache.tez.runtime.library.common.task.local.output.TezLocalTaskOutputFiles;
+import org.apache.tez.runtime.library.common.task.local.output.TezTaskOutput;
+import org.apache.tez.runtime.library.output.LocalOnFileSorterOutput;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -85,10 +85,10 @@ public class TestMapProcessor {
     job.set(TezJobConfig.LOCAL_DIRS, workDir.toString());
     job.set(MRConfig.LOCAL_DIR, workDir.toString());
     job.setClass(
-        Constants.TEZ_ENGINE_TASK_OUTPUT_MANAGER,
+        Constants.TEZ_RUNTIME_TASK_OUTPUT_MANAGER,
         TezLocalTaskOutputFiles.class, 
         TezTaskOutput.class);
-    job.set(TezJobConfig.TEZ_ENGINE_PARTITIONER_CLASS, MRPartitioner.class.getName());
+    job.set(TezJobConfig.TEZ_RUNTIME_PARTITIONER_CLASS, MRPartitioner.class.getName());
     job.setNumReduceTasks(1);
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/processor/reduce/TestReduceProcessor.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/processor/reduce/TestReduceProcessor.java b/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/processor/reduce/TestReduceProcessor.java
index 274c353..d2c7952 100644
--- a/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/processor/reduce/TestReduceProcessor.java
+++ b/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/processor/reduce/TestReduceProcessor.java
@@ -33,21 +33,11 @@ import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.SequenceFileOutputFormat;
 import org.apache.hadoop.mapreduce.MRConfig;
 import org.apache.hadoop.security.token.Token;
-import org.apache.tez.common.Constants;
 import org.apache.tez.common.TezJobConfig;
 import org.apache.tez.common.TezUtils;
 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.engine.api.impl.InputSpec;
-import org.apache.tez.engine.api.impl.OutputSpec;
-import org.apache.tez.engine.api.impl.TaskSpec;
-import org.apache.tez.engine.common.security.JobTokenIdentifier;
-import org.apache.tez.engine.common.task.local.output.TezLocalTaskOutputFiles;
-import org.apache.tez.engine.common.task.local.output.TezTaskOutput;
-import org.apache.tez.engine.lib.input.LocalMergedInput;
-import org.apache.tez.engine.lib.output.LocalOnFileSorterOutput;
-import org.apache.tez.engine.newruntime.LogicalIOProcessorRuntimeTask;
 import org.apache.tez.mapreduce.TestUmbilical;
 import org.apache.tez.mapreduce.TezTestUtils;
 import org.apache.tez.mapreduce.hadoop.IDConverter;
@@ -58,6 +48,16 @@ import org.apache.tez.mapreduce.input.MRInputLegacy;
 import org.apache.tez.mapreduce.output.MROutput;
 import org.apache.tez.mapreduce.partition.MRPartitioner;
 import org.apache.tez.mapreduce.processor.MapUtils;
+import org.apache.tez.runtime.LogicalIOProcessorRuntimeTask;
+import org.apache.tez.runtime.api.impl.InputSpec;
+import org.apache.tez.runtime.api.impl.OutputSpec;
+import org.apache.tez.runtime.api.impl.TaskSpec;
+import org.apache.tez.runtime.library.common.Constants;
+import org.apache.tez.runtime.library.common.security.JobTokenIdentifier;
+import org.apache.tez.runtime.library.common.task.local.output.TezLocalTaskOutputFiles;
+import org.apache.tez.runtime.library.common.task.local.output.TezTaskOutput;
+import org.apache.tez.runtime.library.input.LocalMergedInput;
+import org.apache.tez.runtime.library.output.LocalOnFileSorterOutput;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
@@ -90,10 +90,10 @@ public class TestReduceProcessor {
     job.set(TezJobConfig.LOCAL_DIRS, workDir.toString());
     job.set(MRConfig.LOCAL_DIR, workDir.toString());
     job.setClass(
-        Constants.TEZ_ENGINE_TASK_OUTPUT_MANAGER,
+        Constants.TEZ_RUNTIME_TASK_OUTPUT_MANAGER,
         TezLocalTaskOutputFiles.class, 
         TezTaskOutput.class);
-    job.set(TezJobConfig.TEZ_ENGINE_PARTITIONER_CLASS, MRPartitioner.class.getName());
+    job.set(TezJobConfig.TEZ_RUNTIME_PARTITIONER_CLASS, MRPartitioner.class.getName());
     job.setNumReduceTasks(1);
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-internals/findbugs-exclude.xml
----------------------------------------------------------------------
diff --git a/tez-runtime-internals/findbugs-exclude.xml b/tez-runtime-internals/findbugs-exclude.xml
new file mode 100644
index 0000000..5b11308
--- /dev/null
+++ b/tez-runtime-internals/findbugs-exclude.xml
@@ -0,0 +1,16 @@
+<!--
+  Licensed 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. See accompanying LICENSE file.
+-->
+<FindBugsFilter>
+
+</FindBugsFilter>

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-internals/pom.xml
----------------------------------------------------------------------
diff --git a/tez-runtime-internals/pom.xml b/tez-runtime-internals/pom.xml
new file mode 100644
index 0000000..4f64701
--- /dev/null
+++ b/tez-runtime-internals/pom.xml
@@ -0,0 +1,95 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  Licensed 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. See accompanying LICENSE file.
+-->
+
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <groupId>org.apache.tez</groupId>
+    <artifactId>tez</artifactId>
+    <version>0.2.0-SNAPSHOT</version>
+  </parent>
+  <artifactId>tez-runtime-internals</artifactId>
+
+  <dependencies>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-yarn-api</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.tez</groupId>
+      <artifactId>tez-common</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.tez</groupId>
+      <artifactId>tez-api</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.tez</groupId>
+      <artifactId>tez-runtime-library</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-yarn-common</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>com.google.inject</groupId>
+      <artifactId>guice</artifactId>
+    </dependency>
+    <dependency>
+     <groupId>com.google.protobuf</groupId>
+     <artifactId>protobuf-java</artifactId>
+    </dependency>
+  </dependencies>
+
+  <build>
+    <plugins>
+      <plugin>
+        <groupId>org.apache.rat</groupId>
+        <artifactId>apache-rat-plugin</artifactId>
+        <configuration>
+        </configuration>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.hadoop</groupId>
+        <artifactId>hadoop-maven-plugins</artifactId>
+        <executions>
+          <execution>
+            <id>compile-protoc</id>
+            <phase>generate-sources</phase>
+            <goals>
+              <goal>protoc</goal>
+            </goals>
+            <configuration>
+              <protocVersion>${protobuf.version}</protocVersion>
+              <protocCommand>${protoc.path}</protocCommand>
+              <imports>
+                <param>${basedir}/src/main/proto</param>
+              </imports>
+              <source>
+                <directory>${basedir}/src/main/proto</directory>
+                <includes>
+                  <include>Events.proto</include>
+                </includes>
+              </source>
+              <output>${project.build.directory}/generated-sources/java</output>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
+    </plugins>
+  </build>
+</project>

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-internals/src/main/java/org/apache/tez/common/ContainerContext.java
----------------------------------------------------------------------
diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/common/ContainerContext.java b/tez-runtime-internals/src/main/java/org/apache/tez/common/ContainerContext.java
new file mode 100644
index 0000000..df92bdc
--- /dev/null
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/common/ContainerContext.java
@@ -0,0 +1,64 @@
+/**
+* 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.common;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.Writable;
+
+// TODO EVENTUALLY move this over to PB. Fix package/module.
+// TODO EVENTUALLY unit tests for functionality.
+public class ContainerContext implements Writable {
+
+  String containerIdentifier;
+  String pid;
+
+  public ContainerContext() {
+    containerIdentifier = "";
+    pid = "";
+  }
+
+  public ContainerContext(String containerIdStr, String pid) {
+    this.containerIdentifier = containerIdStr;
+    this.pid = pid;
+  }
+
+  public String getContainerIdentifier() {
+    return containerIdentifier;
+  }
+
+  public String getPid() {
+    return pid;
+  }
+
+  @Override
+  public void readFields(DataInput in) throws IOException {
+    this.containerIdentifier = Text.readString(in);
+    this.pid = Text.readString(in);
+  }
+
+  @Override
+  public void write(DataOutput out) throws IOException {
+    Text.writeString(out, containerIdentifier);
+    Text.writeString(out, pid);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-internals/src/main/java/org/apache/tez/common/ContainerTask.java
----------------------------------------------------------------------
diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/common/ContainerTask.java b/tez-runtime-internals/src/main/java/org/apache/tez/common/ContainerTask.java
new file mode 100644
index 0000000..c865631
--- /dev/null
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/common/ContainerTask.java
@@ -0,0 +1,74 @@
+/**
+ * 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.common;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hadoop.io.Writable;
+import org.apache.tez.runtime.api.impl.TaskSpec;
+
+public class ContainerTask implements Writable {
+
+  TaskSpec taskSpec;
+  boolean shouldDie;
+
+  public ContainerTask() {
+  }
+
+  public ContainerTask(TaskSpec taskSpec, boolean shouldDie) {
+    this.taskSpec = taskSpec;
+    this.shouldDie = shouldDie;
+  }
+
+  public TaskSpec getTaskSpec() {
+    return taskSpec;
+  }
+
+  public boolean shouldDie() {
+    return shouldDie;
+  }
+
+  @Override
+  public void write(DataOutput out) throws IOException {
+    out.writeBoolean(shouldDie);
+    if (taskSpec != null) {
+      out.writeBoolean(true);
+      taskSpec.write(out);
+    } else {
+      out.writeBoolean(false);
+    }
+  }
+
+  @Override
+  public void readFields(DataInput in) throws IOException {
+    shouldDie = in.readBoolean();
+    boolean taskComing = in.readBoolean();
+    if (taskComing) {
+      taskSpec = new TaskSpec();
+      taskSpec.readFields(in);
+    }
+  }
+
+  @Override
+  public String toString() {
+    return "shouldDie: " + shouldDie + ", TaskSpec: "
+        + taskSpec;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-internals/src/main/java/org/apache/tez/common/TezTaskUmbilicalProtocol.java
----------------------------------------------------------------------
diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/common/TezTaskUmbilicalProtocol.java b/tez-runtime-internals/src/main/java/org/apache/tez/common/TezTaskUmbilicalProtocol.java
new file mode 100644
index 0000000..1966790
--- /dev/null
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/common/TezTaskUmbilicalProtocol.java
@@ -0,0 +1,57 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.common;
+
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.ipc.ProtocolInfo;
+import org.apache.hadoop.ipc.VersionedProtocol;
+import org.apache.tez.common.records.ProceedToCompletionResponse;
+import org.apache.tez.dag.api.TezException;
+import org.apache.tez.dag.records.TezTaskAttemptID;
+import org.apache.tez.runtime.api.impl.TezHeartbeatRequest;
+import org.apache.tez.runtime.api.impl.TezHeartbeatResponse;
+
+/** Protocol that task child process uses to contact its parent process.  The
+ * parent is a daemon which which polls the central master for a new map or
+ * reduce task and runs it as a child process.  All communication between child
+ * and parent is via this protocol. */
+@InterfaceAudience.Private
+@InterfaceStability.Stable
+@ProtocolInfo(protocolName = "TezTaskUmbilicalProtocol", protocolVersion = 1)
+public interface TezTaskUmbilicalProtocol extends VersionedProtocol {
+
+  public static final long versionID = 19L;
+
+  ContainerTask getTask(ContainerContext containerContext) throws IOException;
+
+  boolean canCommit(TezTaskAttemptID taskid) throws IOException;
+
+  ProceedToCompletionResponse
+      proceedToCompletion(TezTaskAttemptID taskAttemptId) throws IOException;
+
+  /// Copies from TezUmbilical until complete re-factor is done
+  // TODONEWTEZ
+
+  public TezHeartbeatResponse heartbeat(TezHeartbeatRequest request)
+      throws IOException, TezException;
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-internals/src/main/java/org/apache/tez/runtime/LogicalIOProcessorRuntimeTask.java
----------------------------------------------------------------------
diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/LogicalIOProcessorRuntimeTask.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/LogicalIOProcessorRuntimeTask.java
new file mode 100644
index 0000000..8aff6d1
--- /dev/null
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/LogicalIOProcessorRuntimeTask.java
@@ -0,0 +1,475 @@
+/**
+ * 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;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.LinkedBlockingQueue;
+
+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.token.Token;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.tez.dag.api.ProcessorDescriptor;
+import org.apache.tez.dag.api.TezUncheckedException;
+import org.apache.tez.dag.records.TezTaskAttemptID;
+import org.apache.tez.runtime.api.Event;
+import org.apache.tez.runtime.api.Input;
+import org.apache.tez.runtime.api.LogicalIOProcessor;
+import org.apache.tez.runtime.api.LogicalInput;
+import org.apache.tez.runtime.api.LogicalOutput;
+import org.apache.tez.runtime.api.Output;
+import org.apache.tez.runtime.api.Processor;
+import org.apache.tez.runtime.api.TezInputContext;
+import org.apache.tez.runtime.api.TezOutputContext;
+import org.apache.tez.runtime.api.TezProcessorContext;
+import org.apache.tez.runtime.api.impl.EventMetaData;
+import org.apache.tez.runtime.api.impl.InputSpec;
+import org.apache.tez.runtime.api.impl.OutputSpec;
+import org.apache.tez.runtime.api.impl.TaskSpec;
+import org.apache.tez.runtime.api.impl.TezEvent;
+import org.apache.tez.runtime.api.impl.TezInputContextImpl;
+import org.apache.tez.runtime.api.impl.TezOutputContextImpl;
+import org.apache.tez.runtime.api.impl.TezProcessorContextImpl;
+import org.apache.tez.runtime.api.impl.TezUmbilical;
+import org.apache.tez.runtime.api.impl.EventMetaData.EventProducerConsumerType;
+import org.apache.tez.runtime.library.common.security.JobTokenIdentifier;
+import org.apache.tez.runtime.library.shuffle.common.ShuffleUtils;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+
+@Private
+public class LogicalIOProcessorRuntimeTask extends RuntimeTask {
+
+  private static final Log LOG = LogFactory
+      .getLog(LogicalIOProcessorRuntimeTask.class);
+
+  private final List<InputSpec> inputSpecs;
+  private final List<LogicalInput> inputs;
+
+  private final List<OutputSpec> outputSpecs;
+  private final List<LogicalOutput> outputs;
+
+  private List<TezInputContext> inputContexts;
+  private List<TezOutputContext> outputContexts;
+  private TezProcessorContext processorContext;
+  
+  private final ProcessorDescriptor processorDescriptor;
+  private final LogicalIOProcessor processor;
+
+  private final Map<String, ByteBuffer> serviceConsumerMetadata;
+
+  private Map<String, LogicalInput> inputMap;
+  private Map<String, LogicalOutput> outputMap;
+
+  private LinkedBlockingQueue<TezEvent> eventsToBeProcessed;
+  private Thread eventRouterThread = null;
+
+  private final int appAttemptNumber;
+
+  public LogicalIOProcessorRuntimeTask(TaskSpec taskSpec, int appAttemptNumber,
+      Configuration tezConf, TezUmbilical tezUmbilical,
+      Token<JobTokenIdentifier> jobToken) throws IOException {
+    // TODO Remove jobToken from here post TEZ-421
+    super(taskSpec, tezConf, tezUmbilical);
+    LOG.info("Initializing LogicalIOProcessorRuntimeTask with TaskSpec: "
+        + taskSpec);
+    this.inputContexts = new ArrayList<TezInputContext>(taskSpec.getInputs().size());
+    this.outputContexts = new ArrayList<TezOutputContext>(taskSpec.getOutputs().size());
+    this.inputSpecs = taskSpec.getInputs();
+    this.inputs = createInputs(inputSpecs);
+    this.outputSpecs = taskSpec.getOutputs();
+    this.outputs = createOutputs(outputSpecs);
+    this.processorDescriptor = taskSpec.getProcessorDescriptor();
+    this.processor = createProcessor(processorDescriptor);
+    this.serviceConsumerMetadata = new HashMap<String, ByteBuffer>();
+    this.serviceConsumerMetadata.put(ShuffleUtils.SHUFFLE_HANDLER_SERVICE_ID,
+        ShuffleUtils.convertJobTokenToBytes(jobToken));
+    this.eventsToBeProcessed = new LinkedBlockingQueue<TezEvent>();
+    this.state = State.NEW;
+    this.appAttemptNumber = appAttemptNumber;
+  }
+
+  public void initialize() throws Exception {
+    LOG.info("Initializing LogicalProcessorIORuntimeTask");
+    Preconditions.checkState(this.state == State.NEW, "Already initialized");
+    this.state = State.INITED;
+    inputMap = new LinkedHashMap<String, LogicalInput>(inputs.size());
+    outputMap = new LinkedHashMap<String, LogicalOutput>(outputs.size());
+
+    // TODO Maybe close initialized inputs / outputs in case of failure to
+    // initialize.
+    // Initialize all inputs. TODO: Multi-threaded at some point.
+    for (int i = 0; i < inputs.size(); i++) {
+      String srcVertexName = inputSpecs.get(i).getSourceVertexName();
+      initializeInput(inputs.get(i),
+          inputSpecs.get(i));
+      inputMap.put(srcVertexName, inputs.get(i));
+    }
+
+    // Initialize all outputs. TODO: Multi-threaded at some point.
+    for (int i = 0; i < outputs.size(); i++) {
+      String destVertexName = outputSpecs.get(i).getDestinationVertexName();
+      initializeOutput(outputs.get(i), outputSpecs.get(i));
+      outputMap.put(destVertexName, outputs.get(i));
+    }
+
+    // Initialize processor.
+    initializeLogicalIOProcessor();
+    startRouterThread();
+  }
+
+  public void run() throws Exception {
+    synchronized (this.state) {
+      Preconditions.checkState(this.state == State.INITED,
+          "Can only run while in INITED state. Current: " + this.state);
+      this.state = State.RUNNING;
+    }
+    LogicalIOProcessor lioProcessor = (LogicalIOProcessor) processor;
+    lioProcessor.run(inputMap, outputMap);
+  }
+
+  public void close() throws Exception {
+    try {
+      Preconditions.checkState(this.state == State.RUNNING,
+          "Can only run while in RUNNING state. Current: " + this.state);
+      this.state = State.CLOSED;
+
+      // Close the Inputs.
+      for (int i = 0; i < inputs.size(); i++) {
+        String srcVertexName = inputSpecs.get(i).getSourceVertexName();
+        List<Event> closeInputEvents = inputs.get(i).close();
+        sendTaskGeneratedEvents(closeInputEvents,
+            EventProducerConsumerType.INPUT, taskSpec.getVertexName(),
+            srcVertexName, taskSpec.getTaskAttemptID());
+      }
+
+      // Close the Processor.
+      processor.close();
+
+      // Close the Outputs.
+      for (int i = 0; i < outputs.size(); i++) {
+        String destVertexName = outputSpecs.get(i).getDestinationVertexName();
+        List<Event> closeOutputEvents = outputs.get(i).close();
+        sendTaskGeneratedEvents(closeOutputEvents,
+            EventProducerConsumerType.OUTPUT, taskSpec.getVertexName(),
+            destVertexName, taskSpec.getTaskAttemptID());
+      }
+    } finally {
+      setTaskDone();
+      if (eventRouterThread != null) {
+        eventRouterThread.interrupt();
+      }
+    }
+  }
+
+  private void initializeInput(Input input, InputSpec inputSpec)
+      throws Exception {
+    TezInputContext tezInputContext = createInputContext(inputSpec);
+    inputContexts.add(tezInputContext);
+    if (input instanceof LogicalInput) {
+      ((LogicalInput) input).setNumPhysicalInputs(inputSpec
+          .getPhysicalEdgeCount());
+    }
+    LOG.info("Initializing Input using InputSpec: " + inputSpec);
+    List<Event> events = input.initialize(tezInputContext);
+    sendTaskGeneratedEvents(events, EventProducerConsumerType.INPUT,
+        tezInputContext.getTaskVertexName(),
+        tezInputContext.getSourceVertexName(), taskSpec.getTaskAttemptID());
+  }
+
+  private void initializeOutput(Output output, OutputSpec outputSpec)
+      throws Exception {
+    TezOutputContext tezOutputContext = createOutputContext(outputSpec);
+    outputContexts.add(tezOutputContext);
+    if (output instanceof LogicalOutput) {
+      ((LogicalOutput) output).setNumPhysicalOutputs(outputSpec
+          .getPhysicalEdgeCount());
+    }
+    LOG.info("Initializing Output using OutputSpec: " + outputSpec);
+    List<Event> events = output.initialize(tezOutputContext);
+    sendTaskGeneratedEvents(events, EventProducerConsumerType.OUTPUT,
+        tezOutputContext.getTaskVertexName(),
+        tezOutputContext.getDestinationVertexName(),
+        taskSpec.getTaskAttemptID());
+  }
+
+  private void initializeLogicalIOProcessor() throws Exception {
+    LOG.info("Initializing processor"
+        + ", processorClassName=" + processorDescriptor.getClassName());
+    TezProcessorContext processorContext = createProcessorContext();
+    this.processorContext = processorContext;
+    processor.initialize(processorContext);
+  }
+
+  private TezInputContext createInputContext(InputSpec inputSpec) {
+    TezInputContext inputContext = new TezInputContextImpl(tezConf,
+        appAttemptNumber, tezUmbilical, taskSpec.getVertexName(),
+        inputSpec.getSourceVertexName(), taskSpec.getTaskAttemptID(),
+        tezCounters,
+        inputSpec.getInputDescriptor().getUserPayload() == null ? taskSpec
+            .getProcessorDescriptor().getUserPayload() : inputSpec
+            .getInputDescriptor().getUserPayload(), this,
+        serviceConsumerMetadata);
+    return inputContext;
+  }
+
+  private TezOutputContext createOutputContext(OutputSpec outputSpec) {
+    TezOutputContext outputContext = new TezOutputContextImpl(tezConf,
+        appAttemptNumber, tezUmbilical, taskSpec.getVertexName(),
+        outputSpec.getDestinationVertexName(), taskSpec.getTaskAttemptID(),
+        tezCounters,
+        outputSpec.getOutputDescriptor().getUserPayload() == null ? taskSpec
+            .getProcessorDescriptor().getUserPayload() : outputSpec
+            .getOutputDescriptor().getUserPayload(), this,
+        serviceConsumerMetadata);
+    return outputContext;
+  }
+
+  private TezProcessorContext createProcessorContext() {
+    TezProcessorContext processorContext = new TezProcessorContextImpl(tezConf,
+        appAttemptNumber, tezUmbilical, taskSpec.getVertexName(), taskSpec.getTaskAttemptID(),
+        tezCounters, processorDescriptor.getUserPayload(), this,
+        serviceConsumerMetadata);
+    return processorContext;
+  }
+
+  private List<LogicalInput> createInputs(List<InputSpec> inputSpecs) {
+    List<LogicalInput> inputs = new ArrayList<LogicalInput>(inputSpecs.size());
+    for (InputSpec inputSpec : inputSpecs) {
+      LOG.info("Creating Input from InputSpec: "
+          + inputSpec);
+      Input input = RuntimeUtils.createClazzInstance(inputSpec
+          .getInputDescriptor().getClassName());
+
+      if (input instanceof LogicalInput) {
+        inputs.add((LogicalInput) input);
+      } else {
+        throw new TezUncheckedException(input.getClass().getName()
+            + " is not a sub-type of LogicalInput."
+            + " Only LogicalInput sub-types supported by LogicalIOProcessor.");
+      }
+    }
+    return inputs;
+  }
+
+  private List<LogicalOutput> createOutputs(List<OutputSpec> outputSpecs) {
+    List<LogicalOutput> outputs = new ArrayList<LogicalOutput>(
+        outputSpecs.size());
+    for (OutputSpec outputSpec : outputSpecs) {
+      LOG.info("Creating Output from OutputSpec"
+          + outputSpec);
+      Output output = RuntimeUtils.createClazzInstance(outputSpec
+          .getOutputDescriptor().getClassName());
+      if (output instanceof LogicalOutput) {
+        outputs.add((LogicalOutput) output);
+      } else {
+        throw new TezUncheckedException(output.getClass().getName()
+            + " is not a sub-type of LogicalOutput."
+            + " Only LogicalOutput sub-types supported by LogicalIOProcessor.");
+      }
+    }
+    return outputs;
+  }
+
+  private LogicalIOProcessor createProcessor(
+      ProcessorDescriptor processorDescriptor) {
+    Processor processor = RuntimeUtils.createClazzInstance(processorDescriptor
+        .getClassName());
+    if (!(processor instanceof LogicalIOProcessor)) {
+      throw new TezUncheckedException(processor.getClass().getName()
+          + " is not a sub-type of LogicalIOProcessor."
+          + " Only LogicalOutput sub-types supported by LogicalIOProcessor.");
+    }
+    return (LogicalIOProcessor) processor;
+  }
+
+  private void sendTaskGeneratedEvents(List<Event> events,
+      EventProducerConsumerType generator, String taskVertexName,
+      String edgeVertexName, TezTaskAttemptID taskAttemptID) {
+    if (events == null || events.isEmpty()) {
+      return;
+    }
+    EventMetaData eventMetaData = new EventMetaData(generator,
+        taskVertexName, edgeVertexName, taskAttemptID);
+    List<TezEvent> tezEvents = new ArrayList<TezEvent>(events.size());
+    for (Event e : events) {
+      TezEvent te = new TezEvent(e, eventMetaData);
+      tezEvents.add(te);
+    }
+    if (LOG.isDebugEnabled()) {
+      for (TezEvent e : tezEvents) {
+        LOG.debug("Generated event info"
+            + ", eventMetaData=" + eventMetaData.toString()
+            + ", eventType=" + e.getEventType());
+      }
+    }
+    tezUmbilical.addEvents(tezEvents);
+  }
+
+  private boolean handleEvent(TezEvent e) {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Handling TezEvent in task"
+          + ", taskAttemptId=" + taskSpec.getTaskAttemptID()
+          + ", eventType=" + e.getEventType()
+          + ", eventSourceInfo=" + e.getSourceInfo()
+          + ", eventDestinationInfo=" + e.getDestinationInfo());
+    }
+    try {
+      switch (e.getDestinationInfo().getEventGenerator()) {
+      case INPUT:
+        LogicalInput input = inputMap.get(
+            e.getDestinationInfo().getEdgeVertexName());
+        if (input != null) {
+          input.handleEvents(Collections.singletonList(e.getEvent()));
+        } else {
+          throw new TezUncheckedException("Unhandled event for invalid target: "
+              + e);
+        }
+        break;
+      case OUTPUT:
+        LogicalOutput output = outputMap.get(
+            e.getDestinationInfo().getEdgeVertexName());
+        if (output != null) {
+          output.handleEvents(Collections.singletonList(e.getEvent()));
+        } else {
+          throw new TezUncheckedException("Unhandled event for invalid target: "
+              + e);
+        }
+        break;
+      case PROCESSOR:
+        processor.handleEvents(Collections.singletonList(e.getEvent()));
+        break;
+      case SYSTEM:
+        LOG.warn("Trying to send a System event in a Task: " + e);
+        break;
+      }
+    } catch (Throwable t) {
+      LOG.warn("Failed to handle event", t);
+      setFatalError(t, "Failed to handle event");
+      EventMetaData sourceInfo = new EventMetaData(
+          e.getDestinationInfo().getEventGenerator(),
+          taskSpec.getVertexName(), e.getDestinationInfo().getEdgeVertexName(),
+          getTaskAttemptID());
+      tezUmbilical.signalFatalError(getTaskAttemptID(),
+          StringUtils.stringifyException(t), sourceInfo);
+      return false;
+    }
+    return true;
+  }
+
+  @Override
+  public synchronized void handleEvents(Collection<TezEvent> events) {
+    if (events == null || events.isEmpty()) {
+      return;
+    }
+    eventCounter.addAndGet(events.size());
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Received events to be processed by task"
+          + ", taskAttemptId=" + taskSpec.getTaskAttemptID()
+          + ", eventCount=" + events.size()
+          + ", newEventCounter=" + eventCounter.get());
+    }
+    eventsToBeProcessed.addAll(events);
+  }
+
+  private void startRouterThread() {
+    eventRouterThread = new Thread(new Runnable() {
+      public void run() {
+        while (!isTaskDone() && !Thread.currentThread().isInterrupted()) {
+          try {
+            TezEvent e = eventsToBeProcessed.take();
+            if (e == null) {
+              continue;
+            }
+            // TODO TODONEWTEZ
+            if (!handleEvent(e)) {
+              LOG.warn("Stopping Event Router thread as failed to handle"
+                  + " event: " + e);
+              return;
+            }
+          } catch (InterruptedException e) {
+            if (!isTaskDone()) {
+              LOG.warn("Event Router thread interrupted. Returning.");
+            }
+            return;
+          }
+        }
+      }
+    });
+
+    eventRouterThread.setName("TezTaskEventRouter["
+        + taskSpec.getTaskAttemptID().toString() + "]");
+    eventRouterThread.start();
+  }
+
+  public synchronized void cleanup() {
+    setTaskDone();
+    if (eventRouterThread != null) {
+      eventRouterThread.interrupt();
+    }
+  }
+  
+  @Private
+  @VisibleForTesting
+  public List<TezInputContext> getInputContexts() {
+    return this.inputContexts;
+  }
+  
+  @Private
+  @VisibleForTesting
+  public List<TezOutputContext> getOutputContexts() {
+    return this.outputContexts;
+  }
+
+  @Private
+  @VisibleForTesting
+  public TezProcessorContext getProcessorContext() {
+    return this.processorContext;
+  }
+  
+  @Private
+  @VisibleForTesting
+  public Map<String, LogicalInput> getInputs() {
+    return this.inputMap;
+  }
+  
+  @Private
+  @VisibleForTesting
+  public Map<String, LogicalOutput> getOutputs() {
+    return this.outputMap;
+  }
+  
+  @Private
+  @VisibleForTesting
+  public LogicalIOProcessor getProcessor() {
+    return this.processor;
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-internals/src/main/java/org/apache/tez/runtime/RuntimeTask.java
----------------------------------------------------------------------
diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/RuntimeTask.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/RuntimeTask.java
new file mode 100644
index 0000000..f018333
--- /dev/null
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/RuntimeTask.java
@@ -0,0 +1,106 @@
+/**
+ * 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;
+
+import java.util.Collection;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.tez.common.counters.TezCounters;
+import org.apache.tez.dag.records.TezTaskAttemptID;
+import org.apache.tez.runtime.api.impl.TaskSpec;
+import org.apache.tez.runtime.api.impl.TezEvent;
+import org.apache.tez.runtime.api.impl.TezUmbilical;
+
+public abstract class RuntimeTask {
+
+  protected AtomicBoolean hasFatalError = new AtomicBoolean(false);
+  protected Throwable fatalError = null;
+  protected String fatalErrorMessage = null;
+  protected float progress;
+  protected final TezCounters tezCounters;
+  protected final TaskSpec taskSpec;
+  protected final Configuration tezConf;
+  protected final TezUmbilical tezUmbilical;
+  protected final AtomicInteger eventCounter;
+  private final AtomicBoolean taskDone;
+
+  protected RuntimeTask(TaskSpec taskSpec, Configuration tezConf,
+      TezUmbilical tezUmbilical) {
+    this.taskSpec = taskSpec;
+    this.tezConf = tezConf;
+    this.tezUmbilical = tezUmbilical;
+    this.tezCounters = new TezCounters();
+    this.eventCounter = new AtomicInteger(0);
+    this.progress = 0.0f;
+    this.taskDone = new AtomicBoolean(false);
+  }
+
+  protected enum State {
+    NEW, INITED, RUNNING, CLOSED;
+  }
+
+  protected State state;
+
+  public String getVertexName() {
+    return taskSpec.getVertexName();
+  }
+
+  public void setFatalError(Throwable t, String message) {
+    hasFatalError.set(true);
+    this.fatalError = t;
+    this.fatalErrorMessage = message;
+  }
+
+  public boolean hadFatalError() {
+    return hasFatalError.get();
+  }
+
+  public synchronized void setProgress(float progress) {
+    this.progress = progress;
+  }
+
+  public synchronized float getProgress() {
+    return this.progress;
+  }
+
+  public TezCounters getCounters() {
+    return this.tezCounters;
+  }
+
+  public TezTaskAttemptID getTaskAttemptID() {
+    return taskSpec.getTaskAttemptID();
+  }
+
+  public abstract void handleEvents(Collection<TezEvent> events);
+
+  public int getEventCounter() {
+    return eventCounter.get();
+  }
+
+  public boolean isTaskDone() {
+    return taskDone.get();
+  }
+
+  protected void setTaskDone() {
+    taskDone.set(true);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-internals/src/main/java/org/apache/tez/runtime/RuntimeUtils.java
----------------------------------------------------------------------
diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/RuntimeUtils.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/RuntimeUtils.java
new file mode 100644
index 0000000..8a1b550
--- /dev/null
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/RuntimeUtils.java
@@ -0,0 +1,62 @@
+/**
+ * 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;
+
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+import org.apache.tez.dag.api.TezUncheckedException;
+
+public class RuntimeUtils {
+
+  private static final Map<String, Class<?>> CLAZZ_CACHE = new ConcurrentHashMap<String, Class<?>>();
+
+  private static Class<?> getClazz(String className) {
+    Class<?> clazz = CLAZZ_CACHE.get(className);
+    if (clazz == null) {
+      try {
+        clazz = Class.forName(className);
+      } catch (ClassNotFoundException e) {
+        throw new TezUncheckedException("Unable to load class: " + className, e);
+      }
+    }
+    return clazz;
+  }
+
+  private static <T> T getNewInstance(Class<T> clazz) {
+    T instance;
+    try {
+      instance = clazz.newInstance();
+    } catch (InstantiationException e) {
+      throw new TezUncheckedException(
+          "Unable to instantiate class with 0 arguments: " + clazz.getName(), e);
+    } catch (IllegalAccessException e) {
+      throw new TezUncheckedException(
+          "Unable to instantiate class with 0 arguments: " + clazz.getName(), e);
+    }
+    return instance;
+  }
+
+  public static <T> T createClazzInstance(String className) {
+    Class<?> clazz = getClazz(className);
+    @SuppressWarnings("unchecked")
+    T instance = (T) getNewInstance(clazz);
+    return instance;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/events/TaskAttemptCompletedEvent.java
----------------------------------------------------------------------
diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/events/TaskAttemptCompletedEvent.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/events/TaskAttemptCompletedEvent.java
new file mode 100644
index 0000000..597718f
--- /dev/null
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/events/TaskAttemptCompletedEvent.java
@@ -0,0 +1,28 @@
+/**
+ * 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.api.events;
+
+import org.apache.tez.runtime.api.Event;
+
+public class TaskAttemptCompletedEvent extends Event {
+
+  public TaskAttemptCompletedEvent() {
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/events/TaskAttemptFailedEvent.java
----------------------------------------------------------------------
diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/events/TaskAttemptFailedEvent.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/events/TaskAttemptFailedEvent.java
new file mode 100644
index 0000000..935fdbb
--- /dev/null
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/events/TaskAttemptFailedEvent.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.runtime.api.events;
+
+import org.apache.tez.runtime.api.Event;
+
+public class TaskAttemptFailedEvent extends Event {
+
+  private final String diagnostics;
+
+  public TaskAttemptFailedEvent(String diagnostics) {
+    this.diagnostics = diagnostics;
+  }
+
+  public String getDiagnostics() {
+    return diagnostics;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/events/TaskStatusUpdateEvent.java
----------------------------------------------------------------------
diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/events/TaskStatusUpdateEvent.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/events/TaskStatusUpdateEvent.java
new file mode 100644
index 0000000..47c2998
--- /dev/null
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/events/TaskStatusUpdateEvent.java
@@ -0,0 +1,70 @@
+/**
+ * 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.api.events;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hadoop.io.Writable;
+import org.apache.tez.common.counters.TezCounters;
+import org.apache.tez.runtime.api.Event;
+
+public class TaskStatusUpdateEvent extends Event implements Writable {
+
+  private TezCounters tezCounters;
+  private float progress;
+
+  public TaskStatusUpdateEvent() {
+  }
+
+  public TaskStatusUpdateEvent(TezCounters tezCounters, float progress) {
+    this.tezCounters = tezCounters;
+    this.progress = progress;
+  }
+
+  public TezCounters getCounters() {
+    return tezCounters;
+  }
+
+  public float getProgress() {
+    return progress;
+  }
+
+  @Override
+  public void write(DataOutput out) throws IOException {
+    out.writeFloat(progress);
+    if (tezCounters != null) {
+      out.writeBoolean(true);
+      tezCounters.write(out);
+    } else {
+      out.writeBoolean(false);
+    }
+  }
+
+  @Override
+  public void readFields(DataInput in) throws IOException {
+    progress = in.readFloat();
+    if (in.readBoolean()) {
+      tezCounters = new TezCounters();
+      tezCounters.readFields(in);
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/EventMetaData.java
----------------------------------------------------------------------
diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/EventMetaData.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/EventMetaData.java
new file mode 100644
index 0000000..d650fa3
--- /dev/null
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/EventMetaData.java
@@ -0,0 +1,152 @@
+/**
+ * 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.api.impl;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hadoop.io.Writable;
+import org.apache.tez.dag.records.TezTaskAttemptID;
+
+/**
+ * Class that encapsulates all the information to identify the unique
+ * object that either generated an Event or is the recipient of an Event.
+ */
+public class EventMetaData implements Writable {
+
+  public static enum EventProducerConsumerType {
+    INPUT,
+    PROCESSOR,
+    OUTPUT,
+    SYSTEM
+  }
+
+  /**
+   * Producer Type ( one of Input/Output/Processor ) that generated the Event
+   * or Consumer Type that will consume the Event.
+   */
+  private EventProducerConsumerType producerConsumerType;
+
+  /**
+   * Name of the vertex where the event was generated.
+   */
+  private String taskVertexName;
+
+  /**
+   * Name of the vertex to which the Input or Output is connected to.
+   */
+  private String edgeVertexName;
+
+  /**
+   * i'th physical input/output that this event maps to.
+   */
+  private int index;
+
+  /**
+   * Task Attempt ID
+   */
+  private TezTaskAttemptID taskAttemptID;
+
+  public EventMetaData() {
+  }
+
+  public EventMetaData(EventProducerConsumerType generator,
+      String taskVertexName, String edgeVertexName,
+      TezTaskAttemptID taskAttemptID) {
+    this.producerConsumerType = generator;
+    this.taskVertexName = taskVertexName;
+    this.edgeVertexName = edgeVertexName;
+    this.taskAttemptID = taskAttemptID;
+  }
+
+  public EventProducerConsumerType getEventGenerator() {
+    return producerConsumerType;
+  }
+
+  public TezTaskAttemptID getTaskAttemptID() {
+    return taskAttemptID;
+  }
+
+  public String getTaskVertexName() {
+    return taskVertexName;
+  }
+
+  public String getEdgeVertexName() {
+    return edgeVertexName;
+  }
+
+  @Override
+  public void write(DataOutput out) throws IOException {
+    out.writeInt(producerConsumerType.ordinal());
+    if (taskVertexName != null) {
+      out.writeBoolean(true);
+      out.writeUTF(taskVertexName);
+    } else {
+      out.writeBoolean(false);
+    }
+    if (edgeVertexName != null) {
+      out.writeBoolean(true);
+      out.writeUTF(edgeVertexName);
+    } else {
+      out.writeBoolean(false);
+    }
+    if(taskAttemptID != null) {
+      out.writeBoolean(true);
+      taskAttemptID.write(out);
+    } else {
+      out.writeBoolean(false);
+    }
+    
+    out.writeInt(index);
+  }
+
+  @Override
+  public void readFields(DataInput in) throws IOException {
+    producerConsumerType = EventProducerConsumerType.values()[in.readInt()];
+    if (in.readBoolean()) {
+      taskVertexName = in.readUTF();
+    }
+    if (in.readBoolean()) {
+      edgeVertexName = in.readUTF();
+    }
+    if (in.readBoolean()) {
+      taskAttemptID = new TezTaskAttemptID();
+      taskAttemptID.readFields(in);
+    }
+    index = in.readInt();
+  }
+
+  public int getIndex() {
+    return index;
+  }
+
+  public void setIndex(int index) {
+    this.index = index;
+  }
+
+  @Override
+  public String toString() {
+    return "{ producerConsumerType=" + producerConsumerType
+        + ", taskVertexName=" + taskVertexName
+        + ", edgeVertexName=" + edgeVertexName
+        + ", taskAttemptId=" + taskAttemptID
+        + ", index=" + index + " }";
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/EventType.java
----------------------------------------------------------------------
diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/EventType.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/EventType.java
new file mode 100644
index 0000000..81ff5fc
--- /dev/null
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/EventType.java
@@ -0,0 +1,29 @@
+/**
+ * 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.api.impl;
+
+public enum EventType {
+  TASK_ATTEMPT_COMPLETED_EVENT,
+  TASK_ATTEMPT_FAILED_EVENT,
+  DATA_MOVEMENT_EVENT,
+  INPUT_READ_ERROR_EVENT,
+  INPUT_FAILED_EVENT,
+  INTPUT_INFORMATION_EVENT,
+  TASK_STATUS_UPDATE_EVENT
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/InputSpec.java
----------------------------------------------------------------------
diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/InputSpec.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/InputSpec.java
new file mode 100644
index 0000000..78ed886
--- /dev/null
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/InputSpec.java
@@ -0,0 +1,88 @@
+/**
+ * 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.api.impl;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hadoop.io.Writable;
+import org.apache.tez.dag.api.DagTypeConverters;
+import org.apache.tez.dag.api.InputDescriptor;
+import org.apache.tez.dag.api.records.DAGProtos.TezEntityDescriptorProto;
+
+public class InputSpec implements Writable {
+
+  private String sourceVertexName;
+  private InputDescriptor inputDescriptor;
+  private int physicalEdgeCount;
+
+  public InputSpec() {
+  }
+
+  public InputSpec(String sourceVertexName, InputDescriptor inputDescriptor,
+      int physicalEdgeCount) {
+    this.sourceVertexName = sourceVertexName;
+    this.inputDescriptor = inputDescriptor;
+    this.physicalEdgeCount = physicalEdgeCount;
+  }
+
+  public String getSourceVertexName() {
+    return sourceVertexName;
+  }
+
+  public InputDescriptor getInputDescriptor() {
+    return inputDescriptor;
+  }
+
+  public int getPhysicalEdgeCount() {
+    return physicalEdgeCount;
+  }
+
+  @Override
+  public void write(DataOutput out) throws IOException {
+    // TODONEWTEZ convert to PB
+    out.writeUTF(sourceVertexName);
+    out.writeInt(physicalEdgeCount);
+    byte[] inputDescBytes =
+        DagTypeConverters.convertToDAGPlan(inputDescriptor).toByteArray();
+    out.writeInt(inputDescBytes.length);
+    out.write(inputDescBytes);
+  }
+
+  @Override
+  public void readFields(DataInput in) throws IOException {
+    sourceVertexName = in.readUTF();
+    physicalEdgeCount = in.readInt();
+    int inputDescLen = in.readInt();
+    byte[] inputDescBytes = new byte[inputDescLen];
+    in.readFully(inputDescBytes);
+    inputDescriptor =
+        DagTypeConverters.convertInputDescriptorFromDAGPlan(
+            TezEntityDescriptorProto.parseFrom(inputDescBytes));
+  }
+
+  public String toString() {
+    return "{ sourceVertexName=" + sourceVertexName
+        + ", physicalEdgeCount" + physicalEdgeCount
+        + ", inputClassName=" + inputDescriptor.getClassName()
+        + " }";
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/OutputSpec.java
----------------------------------------------------------------------
diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/OutputSpec.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/OutputSpec.java
new file mode 100644
index 0000000..4034cdd
--- /dev/null
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/OutputSpec.java
@@ -0,0 +1,87 @@
+/**
+ * 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.api.impl;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hadoop.io.Writable;
+import org.apache.tez.dag.api.DagTypeConverters;
+import org.apache.tez.dag.api.OutputDescriptor;
+import org.apache.tez.dag.api.records.DAGProtos.TezEntityDescriptorProto;
+
+public class OutputSpec implements Writable {
+
+  private String destinationVertexName;
+  private OutputDescriptor outputDescriptor;
+  private int physicalEdgeCount;
+
+  public OutputSpec() {
+  }
+
+  public OutputSpec(String destinationVertexName,
+      OutputDescriptor outputDescriptor, int physicalEdgeCount) {
+    this.destinationVertexName = destinationVertexName;
+    this.outputDescriptor = outputDescriptor;
+    this.physicalEdgeCount = physicalEdgeCount;
+  }
+
+  public String getDestinationVertexName() {
+    return destinationVertexName;
+  }
+
+  public OutputDescriptor getOutputDescriptor() {
+    return outputDescriptor;
+  }
+
+  public int getPhysicalEdgeCount() {
+    return physicalEdgeCount;
+  }
+
+  @Override
+  public void write(DataOutput out) throws IOException {
+    // TODONEWTEZ convert to PB
+    out.writeUTF(destinationVertexName);
+    out.writeInt(physicalEdgeCount);
+    byte[] inputDescBytes =
+        DagTypeConverters.convertToDAGPlan(outputDescriptor).toByteArray();
+    out.writeInt(inputDescBytes.length);
+    out.write(inputDescBytes);
+  }
+
+  @Override
+  public void readFields(DataInput in) throws IOException {
+    destinationVertexName = in.readUTF();
+    physicalEdgeCount = in.readInt();
+    int inputDescLen = in.readInt();
+    byte[] inputDescBytes = new byte[inputDescLen];
+    in.readFully(inputDescBytes);
+    outputDescriptor =
+        DagTypeConverters.convertOutputDescriptorFromDAGPlan(
+            TezEntityDescriptorProto.parseFrom(inputDescBytes));
+  }
+
+  public String toString() {
+    return "{ destinationVertexName=" + destinationVertexName
+        + ", physicalEdgeCount" + physicalEdgeCount
+        + ", outputClassName=" + outputDescriptor.getClassName()
+        + " }";
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TaskSpec.java
----------------------------------------------------------------------
diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TaskSpec.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TaskSpec.java
new file mode 100644
index 0000000..6e0995a
--- /dev/null
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TaskSpec.java
@@ -0,0 +1,146 @@
+/**
+ * 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.api.impl;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.io.Writable;
+import org.apache.tez.dag.api.DagTypeConverters;
+import org.apache.tez.dag.api.ProcessorDescriptor;
+import org.apache.tez.dag.api.records.DAGProtos.TezEntityDescriptorProto;
+import org.apache.tez.dag.records.TezTaskAttemptID;
+
+public class TaskSpec implements Writable {
+
+  private TezTaskAttemptID taskAttemptId;
+  private String vertexName;
+  private String user;
+  private ProcessorDescriptor processorDescriptor;
+  private List<InputSpec> inputSpecList;
+  private List<OutputSpec> outputSpecList;
+
+  public TaskSpec() {
+  }
+
+  // TODO NEWTEZ Remove user
+  public TaskSpec(TezTaskAttemptID taskAttemptID, String user,
+      String vertexName, ProcessorDescriptor processorDescriptor,
+      List<InputSpec> inputSpecList, List<OutputSpec> outputSpecList) {
+    this.taskAttemptId = taskAttemptID;
+    this.vertexName = vertexName;
+    this.user = user;
+    this.processorDescriptor = processorDescriptor;
+    this.inputSpecList = inputSpecList;
+    this.outputSpecList = outputSpecList;
+  }
+
+  public String getVertexName() {
+    return vertexName;
+  }
+
+  public TezTaskAttemptID getTaskAttemptID() {
+    return taskAttemptId;
+  }
+
+  public String getUser() {
+    return user;
+  }
+
+  public ProcessorDescriptor getProcessorDescriptor() {
+    return processorDescriptor;
+  }
+
+  public List<InputSpec> getInputs() {
+    return inputSpecList;
+  }
+
+  public List<OutputSpec> getOutputs() {
+    return outputSpecList;
+  }
+
+  @Override
+  public void write(DataOutput out) throws IOException {
+    taskAttemptId.write(out);
+    out.writeUTF(vertexName);
+    byte[] procDesc =
+        DagTypeConverters.convertToDAGPlan(processorDescriptor).toByteArray();
+    out.writeInt(procDesc.length);
+    out.write(procDesc);
+    out.writeInt(inputSpecList.size());
+    for (InputSpec inputSpec : inputSpecList) {
+      inputSpec.write(out);
+    }
+    out.writeInt(outputSpecList.size());
+    for (OutputSpec outputSpec : outputSpecList) {
+      outputSpec.write(out);
+    }
+  }
+
+  @Override
+  public void readFields(DataInput in) throws IOException {
+    taskAttemptId = new TezTaskAttemptID();
+    taskAttemptId.readFields(in);
+    vertexName = in.readUTF();
+    int procDescLength = in.readInt();
+    // TODO at least 3 buffer copies here. Need to convert this to full PB
+    // TEZ-305
+    byte[] procDescBytes = new byte[procDescLength];
+    in.readFully(procDescBytes);
+    processorDescriptor =
+        DagTypeConverters.convertProcessorDescriptorFromDAGPlan(
+            TezEntityDescriptorProto.parseFrom(procDescBytes));
+    int numInputSpecs = in.readInt();
+    inputSpecList = new ArrayList<InputSpec>(numInputSpecs);
+    for (int i = 0; i < numInputSpecs; i++) {
+      InputSpec inputSpec = new InputSpec();
+      inputSpec.readFields(in);
+      inputSpecList.add(inputSpec);
+    }
+    int numOutputSpecs = in.readInt();
+    outputSpecList = new ArrayList<OutputSpec>(numOutputSpecs);
+    for (int i = 0; i < numOutputSpecs; i++) {
+      OutputSpec outputSpec = new OutputSpec();
+      outputSpec.readFields(in);
+      outputSpecList.add(outputSpec);
+    }
+  }
+
+  @Override
+  public String toString() {
+    StringBuffer sb = new StringBuffer();
+    sb.append("TaskAttemptID:" + taskAttemptId);
+    sb.append("processorName=" + processorDescriptor.getClassName()
+        + ", inputSpecListSize=" + inputSpecList.size()
+        + ", outputSpecListSize=" + outputSpecList.size());
+    sb.append(", inputSpecList=[");
+    for (InputSpec i : inputSpecList) {
+      sb.append("{" + i.toString() + "}, ");
+    }
+    sb.append("], outputSpecList=[");
+    for (OutputSpec i : outputSpecList) {
+      sb.append("{" + i.toString() + "}, ");
+    }
+    sb.append("]");
+    return sb.toString();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezEvent.java
----------------------------------------------------------------------
diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezEvent.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezEvent.java
new file mode 100644
index 0000000..e195cf9
--- /dev/null
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezEvent.java
@@ -0,0 +1,248 @@
+/**
+ * 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.api.impl;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hadoop.io.Writable;
+import org.apache.tez.dag.api.TezUncheckedException;
+import org.apache.tez.runtime.api.Event;
+import org.apache.tez.runtime.api.events.DataMovementEvent;
+import org.apache.tez.runtime.api.events.EventProtos.DataMovementEventProto;
+import org.apache.tez.runtime.api.events.EventProtos.InputFailedEventProto;
+import org.apache.tez.runtime.api.events.EventProtos.InputInformationEventProto;
+import org.apache.tez.runtime.api.events.EventProtos.InputReadErrorEventProto;
+import org.apache.tez.runtime.api.events.InputFailedEvent;
+import org.apache.tez.runtime.api.events.InputInformationEvent;
+import org.apache.tez.runtime.api.events.InputReadErrorEvent;
+import org.apache.tez.runtime.api.events.TaskAttemptCompletedEvent;
+import org.apache.tez.runtime.api.events.TaskAttemptFailedEvent;
+import org.apache.tez.runtime.api.events.TaskStatusUpdateEvent;
+import org.apache.tez.runtime.internals.api.events.SystemEventProtos.TaskAttemptCompletedEventProto;
+import org.apache.tez.runtime.internals.api.events.SystemEventProtos.TaskAttemptFailedEventProto;
+
+import com.google.protobuf.ByteString;
+
+public class TezEvent implements Writable {
+
+  private EventType eventType;
+
+  private Event event;
+
+  private EventMetaData sourceInfo;
+
+  private EventMetaData destinationInfo;
+
+  public TezEvent() {
+  }
+
+  public TezEvent(Event event, EventMetaData sourceInfo) {
+    this.event = event;
+    this.setSourceInfo(sourceInfo);
+    if (event instanceof DataMovementEvent) {
+      eventType = EventType.DATA_MOVEMENT_EVENT;
+    } else if (event instanceof InputReadErrorEvent) {
+      eventType = EventType.INPUT_READ_ERROR_EVENT;
+    } else if (event instanceof TaskAttemptFailedEvent) {
+      eventType = EventType.TASK_ATTEMPT_FAILED_EVENT;
+    } else if (event instanceof TaskAttemptCompletedEvent) {
+      eventType = EventType.TASK_ATTEMPT_COMPLETED_EVENT;
+    } else if (event instanceof InputInformationEvent) {
+      eventType = EventType.INTPUT_INFORMATION_EVENT;
+    } else if (event instanceof InputFailedEvent) {
+      eventType = EventType.INPUT_FAILED_EVENT;
+    } else if (event instanceof TaskStatusUpdateEvent) {
+      eventType = EventType.TASK_STATUS_UPDATE_EVENT;
+    } else {
+      throw new TezUncheckedException("Unknown event, event="
+          + event.getClass().getName());
+    }
+  }
+
+  public Event getEvent() {
+    return event;
+  }
+
+  public EventMetaData getSourceInfo() {
+    return sourceInfo;
+  }
+
+  public void setSourceInfo(EventMetaData sourceInfo) {
+    this.sourceInfo = sourceInfo;
+  }
+
+  public EventMetaData getDestinationInfo() {
+    return destinationInfo;
+  }
+
+  public void setDestinationInfo(EventMetaData destinationInfo) {
+    this.destinationInfo = destinationInfo;
+  }
+
+  public EventType getEventType() {
+    return eventType;
+  }
+
+  private void serializeEvent(DataOutput out) throws IOException {
+    if (event == null) {
+      out.writeBoolean(false);
+      return;
+    }
+    out.writeBoolean(true);
+    out.writeInt(eventType.ordinal());
+    if (eventType.equals(EventType.TASK_STATUS_UPDATE_EVENT)) {
+      // TODO NEWTEZ convert to PB
+      TaskStatusUpdateEvent sEvt = (TaskStatusUpdateEvent) event;
+      sEvt.write(out);
+    } else {
+      byte[] eventBytes = null;
+      switch (eventType) {
+      case DATA_MOVEMENT_EVENT:
+        DataMovementEvent dmEvt = (DataMovementEvent) event;
+        eventBytes = DataMovementEventProto.newBuilder()
+          .setSourceIndex(dmEvt.getSourceIndex())
+          .setTargetIndex(dmEvt.getTargetIndex())
+          .setUserPayload(ByteString.copyFrom(dmEvt.getUserPayload()))
+          .build().toByteArray();
+        break;
+      case INPUT_READ_ERROR_EVENT:
+        InputReadErrorEvent ideEvt = (InputReadErrorEvent) event;
+        eventBytes = InputReadErrorEventProto.newBuilder()
+            .setIndex(ideEvt.getIndex())
+            .setDiagnostics(ideEvt.getDiagnostics())
+            .build().toByteArray();
+        break;
+      case TASK_ATTEMPT_FAILED_EVENT:
+        TaskAttemptFailedEvent tfEvt = (TaskAttemptFailedEvent) event;
+        eventBytes = TaskAttemptFailedEventProto.newBuilder()
+            .setDiagnostics(tfEvt.getDiagnostics())
+            .build().toByteArray();
+        break;
+      case TASK_ATTEMPT_COMPLETED_EVENT:
+        eventBytes = TaskAttemptCompletedEventProto.newBuilder()
+            .build().toByteArray();
+        break;
+      case INPUT_FAILED_EVENT:
+        InputFailedEvent ifEvt = (InputFailedEvent) event;
+        eventBytes = InputFailedEventProto.newBuilder()
+            .setSourceIndex(ifEvt.getSourceIndex())
+            .setTargetIndex(ifEvt.getTargetIndex())
+            .setVersion(ifEvt.getVersion()).build().toByteArray();
+      case INTPUT_INFORMATION_EVENT:
+        InputInformationEvent iEvt = (InputInformationEvent) event;
+        eventBytes = InputInformationEventProto.newBuilder()
+            .setUserPayload(ByteString.copyFrom(iEvt.getUserPayload()))
+            .build().toByteArray();
+      default:
+        throw new TezUncheckedException("Unknown TezEvent"
+           + ", type=" + eventType);
+      }
+      out.writeInt(eventBytes.length);
+      out.write(eventBytes);
+    }
+  }
+
+  private void deserializeEvent(DataInput in) throws IOException {
+    if (!in.readBoolean()) {
+      event = null;
+      return;
+    }
+    eventType = EventType.values()[in.readInt()];
+    if (eventType.equals(EventType.TASK_STATUS_UPDATE_EVENT)) {
+      // TODO NEWTEZ convert to PB
+      event = new TaskStatusUpdateEvent();
+      ((TaskStatusUpdateEvent)event).readFields(in);
+    } else {
+      int eventBytesLen = in.readInt();
+      byte[] eventBytes = new byte[eventBytesLen];
+      in.readFully(eventBytes);
+      switch (eventType) {
+      case DATA_MOVEMENT_EVENT:
+        DataMovementEventProto dmProto =
+            DataMovementEventProto.parseFrom(eventBytes);
+        event = new DataMovementEvent(dmProto.getSourceIndex(),
+            dmProto.getTargetIndex(),
+            dmProto.getUserPayload().toByteArray());
+        break;
+      case INPUT_READ_ERROR_EVENT:
+        InputReadErrorEventProto ideProto =
+            InputReadErrorEventProto.parseFrom(eventBytes);
+        event = new InputReadErrorEvent(ideProto.getDiagnostics(),
+            ideProto.getIndex(), ideProto.getVersion());
+        break;
+      case TASK_ATTEMPT_FAILED_EVENT:
+        TaskAttemptFailedEventProto tfProto =
+            TaskAttemptFailedEventProto.parseFrom(eventBytes);
+        event = new TaskAttemptFailedEvent(tfProto.getDiagnostics());
+        break;
+      case TASK_ATTEMPT_COMPLETED_EVENT:
+        event = new TaskAttemptCompletedEvent();
+        break;
+      case INPUT_FAILED_EVENT:
+        InputFailedEventProto ifProto =
+            InputFailedEventProto.parseFrom(eventBytes);
+        event = new InputFailedEvent(ifProto.getSourceIndex(),
+            ifProto.getTargetIndex(), ifProto.getVersion());
+        break;
+      case INTPUT_INFORMATION_EVENT:
+        InputInformationEventProto infoProto =
+            InputInformationEventProto.parseFrom(eventBytes);
+        event = new InputInformationEvent(
+            infoProto.getUserPayload().toByteArray());
+        break;
+      default:
+        throw new TezUncheckedException("Unknown TezEvent"
+           + ", type=" + eventType);
+      }
+    }
+  }
+
+  @Override
+  public void write(DataOutput out) throws IOException {
+    serializeEvent(out);
+    if (sourceInfo != null) {
+      out.writeBoolean(true);
+      sourceInfo.write(out);
+    } else {
+      out.writeBoolean(false);
+    }
+    if (destinationInfo != null) {
+      out.writeBoolean(true);
+      destinationInfo.write(out);
+    } else {
+      out.writeBoolean(false);
+    }
+  }
+
+  @Override
+  public void readFields(DataInput in) throws IOException {
+    deserializeEvent(in);
+    if (in.readBoolean()) {
+      sourceInfo = new EventMetaData();
+      sourceInfo.readFields(in);
+    }
+    if (in.readBoolean()) {
+      destinationInfo = new EventMetaData();
+      destinationInfo.readFields(in);
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezHeartbeatRequest.java
----------------------------------------------------------------------
diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezHeartbeatRequest.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezHeartbeatRequest.java
new file mode 100644
index 0000000..af7cebb
--- /dev/null
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezHeartbeatRequest.java
@@ -0,0 +1,137 @@
+/**
+ * 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.api.impl;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.Writable;
+import org.apache.tez.dag.records.TezTaskAttemptID;
+
+
+public class TezHeartbeatRequest implements Writable {
+
+  private String containerIdentifier;
+  private List<TezEvent> events;
+  private TezTaskAttemptID currentTaskAttemptID;
+  private int startIndex;
+  private int maxEvents;
+  private long requestId;
+
+  public TezHeartbeatRequest() {
+  }
+
+  public TezHeartbeatRequest(long requestId, List<TezEvent> events,
+      String containerIdentifier, TezTaskAttemptID taskAttemptID,
+      int startIndex, int maxEvents) {
+    this.containerIdentifier = containerIdentifier;
+    this.requestId = requestId;
+    this.events = Collections.unmodifiableList(events);
+    this.startIndex = startIndex;
+    this.maxEvents = maxEvents;
+    this.currentTaskAttemptID = taskAttemptID;
+  }
+
+  public String getContainerIdentifier() {
+    return containerIdentifier;
+  }
+
+  public List<TezEvent> getEvents() {
+    return events;
+  }
+
+  public int getStartIndex() {
+    return startIndex;
+  }
+
+  public int getMaxEvents() {
+    return maxEvents;
+  }
+
+  public long getRequestId() {
+    return requestId;
+  }
+
+  public TezTaskAttemptID getCurrentTaskAttemptID() {
+    return currentTaskAttemptID;
+  }
+
+  @Override
+  public void write(DataOutput out) throws IOException {
+    if (events != null) {
+      out.writeBoolean(true);
+      out.writeInt(events.size());
+      for (TezEvent e : events) {
+        e.write(out);
+      }
+    } else {
+      out.writeBoolean(false);
+    }
+    if (currentTaskAttemptID != null) {
+      out.writeBoolean(true);
+      currentTaskAttemptID.write(out);
+    } else {
+      out.writeBoolean(false);
+    }
+    out.writeInt(startIndex);
+    out.writeInt(maxEvents);
+    out.writeLong(requestId);
+    Text.writeString(out, containerIdentifier);
+  }
+
+  @Override
+  public void readFields(DataInput in) throws IOException {
+    if (in.readBoolean()) {
+      int eventsCount = in.readInt();
+      events = new ArrayList<TezEvent>(eventsCount);
+      for (int i = 0; i < eventsCount; ++i) {
+        TezEvent e = new TezEvent();
+        e.readFields(in);
+        events.add(e);
+      }
+    }
+    if (in.readBoolean()) {
+      currentTaskAttemptID = new TezTaskAttemptID();
+      currentTaskAttemptID.readFields(in);
+    } else {
+      currentTaskAttemptID = null;
+    }
+    startIndex = in.readInt();
+    maxEvents = in.readInt();
+    requestId = in.readLong();
+    containerIdentifier = Text.readString(in);
+  }
+
+  @Override
+  public String toString() {
+    return "{ "
+        + " containerId=" + containerIdentifier
+        + ", requestId=" + requestId
+        + ", startIndex=" + startIndex
+        + ", maxEventsToGet=" + maxEvents
+        + ", taskAttemptId" + currentTaskAttemptID
+        + ", eventCount=" + (events != null ? events.size() : 0)
+        + " }";
+  }
+}


[15/50] [abbrv] git commit: Rename *.new* packages back to what they should be, remove dead code from the old packages - mapreduce module - tez-engine module (part of TEZ-398). (sseth)

Posted by ss...@apache.org.
 Rename *.new* packages back to what they should be, remove dead code
 from the old packages - mapreduce module - tez-engine module (part of
 TEZ-398). (sseth)


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

Branch: refs/heads/master
Commit: b4950f98a7a0c62d4c6d53ab75bfb857e8f6b551
Parents: 3d60945
Author: Siddharth Seth <ss...@apache.org>
Authored: Mon Sep 23 10:44:14 2013 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Mon Sep 23 10:44:14 2013 -0700

----------------------------------------------------------------------
 .../apache/hadoop/mapred/YarnTezDagChild.java   |   24 +-
 .../dag/app/TaskAttemptListenerImpTezDag.java   |    6 +-
 .../java/org/apache/tez/dag/app/dag/Task.java   |    2 +-
 .../java/org/apache/tez/dag/app/dag/Vertex.java |    4 +-
 .../dag/event/TaskAttemptEventStatusUpdate.java |    2 +-
 .../dag/app/dag/event/TaskEventAddTezEvent.java |    2 +-
 .../app/dag/event/VertexEventRouteEvent.java    |    2 +-
 .../org/apache/tez/dag/app/dag/impl/Edge.java   |   10 +-
 .../tez/dag/app/dag/impl/TaskAttemptImpl.java   |    4 +-
 .../apache/tez/dag/app/dag/impl/TaskImpl.java   |    2 +-
 .../apache/tez/dag/app/dag/impl/VertexImpl.java |   12 +-
 .../app/rm/AMSchedulerEventTALaunchRequest.java |    2 +-
 .../rm/container/AMContainerEventAssignTA.java  |    2 +-
 .../dag/app/rm/container/AMContainerImpl.java   |    2 +-
 .../dag/app/rm/container/AMContainerTask.java   |    2 +-
 .../tez/dag/app/dag/impl/TestTaskAttempt.java   |    2 +-
 .../tez/dag/app/rm/TestContainerReuse.java      |    7 +-
 .../org/apache/tez/common/ContainerTask.java    |    2 +-
 .../apache/tez/common/RunningTaskContext.java   |   70 -
 .../apache/tez/common/TezEngineTaskContext.java |  141 --
 .../org/apache/tez/common/TezTaskReporter.java  |  116 --
 .../tez/common/TezTaskUmbilicalProtocol.java    |   14 +-
 .../org/apache/tez/engine/api/KVReader.java     |   81 +
 .../org/apache/tez/engine/api/KVWriter.java     |   40 +
 .../api/events/TaskAttemptCompletedEvent.java   |   28 +
 .../api/events/TaskAttemptFailedEvent.java      |   35 +
 .../api/events/TaskStatusUpdateEvent.java       |   70 +
 .../tez/engine/api/impl/EventMetaData.java      |  152 ++
 .../apache/tez/engine/api/impl/EventType.java   |   29 +
 .../apache/tez/engine/api/impl/InputSpec.java   |   88 +
 .../apache/tez/engine/api/impl/OutputSpec.java  |   87 +
 .../apache/tez/engine/api/impl/TaskSpec.java    |  146 ++
 .../apache/tez/engine/api/impl/TezEvent.java    |  248 +++
 .../engine/api/impl/TezHeartbeatRequest.java    |  137 ++
 .../engine/api/impl/TezHeartbeatResponse.java   |  105 ++
 .../engine/api/impl/TezInputContextImpl.java    |   84 +
 .../engine/api/impl/TezOutputContextImpl.java   |   85 +
 .../api/impl/TezProcessorContextImpl.java       |   86 +
 .../tez/engine/api/impl/TezTaskContextImpl.java |  145 ++
 .../tez/engine/api/impl/TezUmbilical.java       |   36 +
 .../broadcast/input/BroadcastInputManager.java  |    2 +-
 .../broadcast/input/BroadcastKVReader.java      |    2 +-
 .../broadcast/output/FileBasedKVWriter.java     |    4 +-
 .../tez/engine/common/TezEngineUtils.java       |    4 +-
 .../tez/engine/common/combine/CombineInput.java |  176 --
 .../engine/common/combine/CombineOutput.java    |   55 -
 .../common/localshuffle/LocalShuffle.java       |    4 +-
 .../engine/common/shuffle/impl/MapOutput.java   |    2 +-
 .../common/shuffle/impl/MergeManager.java       |    2 +-
 .../tez/engine/common/sort/SortingOutput.java   |   32 -
 .../engine/common/sort/impl/ExternalSorter.java |    2 +-
 .../common/task/impl/CombineValuesIterator.java |   51 -
 .../newoutput/TezLocalTaskOutputFiles.java      |  249 ---
 .../task/local/newoutput/TezTaskOutput.java     |  165 --
 .../local/newoutput/TezTaskOutputFiles.java     |  246 ---
 .../local/output/TezLocalTaskOutputFiles.java   |   58 +-
 .../common/task/local/output/TezTaskOutput.java |   45 +-
 .../task/local/output/TezTaskOutputFiles.java   |   65 +-
 .../engine/lib/input/ShuffledMergedInput.java   |    2 +-
 .../engine/lib/oldinput/LocalMergedInput.java   |   67 -
 .../lib/oldinput/OldShuffledMergedInput.java    |   74 -
 .../lib/oldoutput/OldInMemorySortedOutput.java  |   58 -
 .../oldoutput/OldLocalOnFileSorterOutput.java   |   38 -
 .../lib/oldoutput/OldOnFileSortedOutput.java    |   62 -
 .../engine/lib/output/InMemorySortedOutput.java |    2 +-
 .../lib/output/LocalOnFileSorterOutput.java     |    2 +-
 .../engine/lib/output/OnFileSortedOutput.java   |    2 +-
 .../lib/output/OnFileUnorderedKVOutput.java     |    2 +-
 .../org/apache/tez/engine/newapi/KVReader.java  |   79 -
 .../org/apache/tez/engine/newapi/KVWriter.java  |   38 -
 .../events/TaskAttemptCompletedEvent.java       |   28 -
 .../newapi/events/TaskAttemptFailedEvent.java   |   35 -
 .../newapi/events/TaskStatusUpdateEvent.java    |   70 -
 .../tez/engine/newapi/impl/EventMetaData.java   |  152 --
 .../tez/engine/newapi/impl/EventType.java       |   29 -
 .../tez/engine/newapi/impl/InputSpec.java       |   88 -
 .../tez/engine/newapi/impl/OutputSpec.java      |   87 -
 .../apache/tez/engine/newapi/impl/TaskSpec.java |  146 --
 .../apache/tez/engine/newapi/impl/TezEvent.java |  248 ---
 .../engine/newapi/impl/TezHeartbeatRequest.java |  137 --
 .../newapi/impl/TezHeartbeatResponse.java       |  105 --
 .../engine/newapi/impl/TezInputContextImpl.java |   84 -
 .../newapi/impl/TezOutputContextImpl.java       |   85 -
 .../newapi/impl/TezProcessorContextImpl.java    |   86 -
 .../engine/newapi/impl/TezTaskContextImpl.java  |  145 --
 .../tez/engine/newapi/impl/TezUmbilical.java    |   36 -
 .../LogicalIOProcessorRuntimeTask.java          |   20 +-
 .../tez/engine/newruntime/RuntimeTask.java      |    6 +-
 .../apache/tez/engine/runtime/RuntimeUtils.java |  164 --
 .../engine/shuffle/common/DiskFetchedInput.java |    2 +-
 .../org/apache/tez/engine/task/RuntimeTask.java |   92 -
 .../mapred/LocalClientProtocolProviderTez.java  |  108 +-
 .../hadoop/mapred/LocalJobRunnerMetricsTez.java |  196 +-
 .../apache/hadoop/mapred/LocalJobRunnerTez.java | 1753 +++++++++---------
 .../apache/tez/mapreduce/input/SimpleInput.java |    2 +-
 .../tez/mapreduce/output/SimpleOutput.java      |    2 +-
 .../mapreduce/processor/map/MapProcessor.java   |    4 +-
 .../processor/reduce/ReduceProcessor.java       |    4 +-
 .../tez/mapreduce/TestUmbilicalProtocol.java    |    4 +-
 .../tez/mapreduce/processor/MapUtils.java       |   43 +-
 .../processor/map/TestMapProcessor.java         |   45 +-
 .../processor/reduce/TestReduceProcessor.java   |   76 +-
 102 files changed, 2984 insertions(+), 4800 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-dag/src/main/java/org/apache/hadoop/mapred/YarnTezDagChild.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/hadoop/mapred/YarnTezDagChild.java b/tez-dag/src/main/java/org/apache/hadoop/mapred/YarnTezDagChild.java
index 31898a3..f32fa6b 100644
--- a/tez-dag/src/main/java/org/apache/hadoop/mapred/YarnTezDagChild.java
+++ b/tez-dag/src/main/java/org/apache/hadoop/mapred/YarnTezDagChild.java
@@ -67,23 +67,23 @@ import org.apache.tez.dag.api.TezConfiguration;
 import org.apache.tez.dag.api.TezException;
 import org.apache.tez.dag.records.TezTaskAttemptID;
 import org.apache.tez.dag.records.TezVertexID;
+import org.apache.tez.engine.api.events.TaskAttemptCompletedEvent;
+import org.apache.tez.engine.api.events.TaskAttemptFailedEvent;
+import org.apache.tez.engine.api.events.TaskStatusUpdateEvent;
+import org.apache.tez.engine.api.impl.EventMetaData;
+import org.apache.tez.engine.api.impl.InputSpec;
+import org.apache.tez.engine.api.impl.OutputSpec;
+import org.apache.tez.engine.api.impl.TaskSpec;
+import org.apache.tez.engine.api.impl.TezEvent;
+import org.apache.tez.engine.api.impl.TezHeartbeatRequest;
+import org.apache.tez.engine.api.impl.TezHeartbeatResponse;
+import org.apache.tez.engine.api.impl.TezUmbilical;
+import org.apache.tez.engine.api.impl.EventMetaData.EventProducerConsumerType;
 import org.apache.tez.engine.common.objectregistry.ObjectLifeCycle;
 import org.apache.tez.engine.common.objectregistry.ObjectRegistryImpl;
 import org.apache.tez.engine.common.objectregistry.ObjectRegistryModule;
 import org.apache.tez.engine.common.security.JobTokenIdentifier;
 import org.apache.tez.engine.common.security.TokenCache;
-import org.apache.tez.engine.newapi.events.TaskAttemptCompletedEvent;
-import org.apache.tez.engine.newapi.events.TaskAttemptFailedEvent;
-import org.apache.tez.engine.newapi.events.TaskStatusUpdateEvent;
-import org.apache.tez.engine.newapi.impl.EventMetaData;
-import org.apache.tez.engine.newapi.impl.EventMetaData.EventProducerConsumerType;
-import org.apache.tez.engine.newapi.impl.InputSpec;
-import org.apache.tez.engine.newapi.impl.OutputSpec;
-import org.apache.tez.engine.newapi.impl.TaskSpec;
-import org.apache.tez.engine.newapi.impl.TezEvent;
-import org.apache.tez.engine.newapi.impl.TezHeartbeatRequest;
-import org.apache.tez.engine.newapi.impl.TezHeartbeatResponse;
-import org.apache.tez.engine.newapi.impl.TezUmbilical;
 import org.apache.tez.engine.newruntime.LogicalIOProcessorRuntimeTask;
 import org.apache.tez.mapreduce.input.SimpleInputLegacy;
 import org.apache.tez.mapreduce.output.SimpleOutput;

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java b/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java
index 2be9c5f..36486c9 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java
@@ -52,10 +52,10 @@ import org.apache.tez.dag.app.rm.container.AMContainerTask;
 import org.apache.tez.dag.app.security.authorize.MRAMPolicyProvider;
 import org.apache.tez.dag.records.TezTaskAttemptID;
 import org.apache.tez.dag.records.TezVertexID;
+import org.apache.tez.engine.api.impl.TezEvent;
+import org.apache.tez.engine.api.impl.TezHeartbeatRequest;
+import org.apache.tez.engine.api.impl.TezHeartbeatResponse;
 import org.apache.tez.engine.common.security.JobTokenSecretManager;
-import org.apache.tez.engine.newapi.impl.TezEvent;
-import org.apache.tez.engine.newapi.impl.TezHeartbeatRequest;
-import org.apache.tez.engine.newapi.impl.TezHeartbeatResponse;
 import org.apache.tez.engine.records.OutputContext;
 import org.apache.tez.engine.records.TezDependentTaskCompletionEvent;
 import org.apache.tez.engine.records.TezTaskDependencyCompletionEventsUpdate;

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-dag/src/main/java/org/apache/tez/dag/app/dag/Task.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/Task.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/Task.java
index 0947a41..088a195 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/Task.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/Task.java
@@ -26,7 +26,7 @@ import org.apache.tez.dag.api.oldrecords.TaskReport;
 import org.apache.tez.dag.api.oldrecords.TaskState;
 import org.apache.tez.dag.records.TezTaskAttemptID;
 import org.apache.tez.dag.records.TezTaskID;
-import org.apache.tez.engine.newapi.impl.TezEvent;
+import org.apache.tez.engine.api.impl.TezEvent;
 
 /**
  * Read only view of Task.

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-dag/src/main/java/org/apache/tez/dag/app/dag/Vertex.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/Vertex.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/Vertex.java
index 85240e7..42ff8de 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/Vertex.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/Vertex.java
@@ -32,8 +32,8 @@ import org.apache.tez.dag.app.dag.impl.Edge;
 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.engine.newapi.impl.InputSpec;
-import org.apache.tez.engine.newapi.impl.OutputSpec;
+import org.apache.tez.engine.api.impl.InputSpec;
+import org.apache.tez.engine.api.impl.OutputSpec;
 import org.apache.tez.engine.records.TezDependentTaskCompletionEvent;
 
 

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskAttemptEventStatusUpdate.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskAttemptEventStatusUpdate.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskAttemptEventStatusUpdate.java
index 3a8c489..0b8db76 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskAttemptEventStatusUpdate.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskAttemptEventStatusUpdate.java
@@ -25,7 +25,7 @@ import org.apache.tez.common.counters.DAGCounter;
 import org.apache.tez.common.counters.TezCounters;
 import org.apache.tez.dag.api.oldrecords.TaskAttemptState;
 import org.apache.tez.dag.records.TezTaskAttemptID;
-import org.apache.tez.engine.newapi.events.TaskStatusUpdateEvent;
+import org.apache.tez.engine.api.events.TaskStatusUpdateEvent;
 
 public class TaskAttemptEventStatusUpdate extends TaskAttemptEvent {
   

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskEventAddTezEvent.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskEventAddTezEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskEventAddTezEvent.java
index 51f6d53..4154bd0 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskEventAddTezEvent.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskEventAddTezEvent.java
@@ -19,7 +19,7 @@
 package org.apache.tez.dag.app.dag.event;
 
 import org.apache.tez.dag.records.TezTaskID;
-import org.apache.tez.engine.newapi.impl.TezEvent;
+import org.apache.tez.engine.api.impl.TezEvent;
 
 public class TaskEventAddTezEvent extends TaskEvent {
 

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexEventRouteEvent.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexEventRouteEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexEventRouteEvent.java
index c851ae0..37478cb 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexEventRouteEvent.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexEventRouteEvent.java
@@ -21,7 +21,7 @@ package org.apache.tez.dag.app.dag.event;
 import java.util.List;
 
 import org.apache.tez.dag.records.TezVertexID;
-import org.apache.tez.engine.newapi.impl.TezEvent;
+import org.apache.tez.engine.api.impl.TezEvent;
 
 public class VertexEventRouteEvent extends VertexEvent {
   

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/Edge.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/Edge.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/Edge.java
index d565978..3605857 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/Edge.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/Edge.java
@@ -31,14 +31,14 @@ import org.apache.tez.dag.app.dag.Vertex;
 import org.apache.tez.dag.app.dag.event.TaskEventAddTezEvent;
 import org.apache.tez.dag.records.TezTaskAttemptID;
 import org.apache.tez.dag.records.TezTaskID;
+import org.apache.tez.engine.api.impl.EventMetaData;
+import org.apache.tez.engine.api.impl.InputSpec;
+import org.apache.tez.engine.api.impl.OutputSpec;
+import org.apache.tez.engine.api.impl.TezEvent;
+import org.apache.tez.engine.api.impl.EventMetaData.EventProducerConsumerType;
 import org.apache.tez.engine.newapi.events.DataMovementEvent;
 import org.apache.tez.engine.newapi.events.InputFailedEvent;
 import org.apache.tez.engine.newapi.events.InputReadErrorEvent;
-import org.apache.tez.engine.newapi.impl.EventMetaData;
-import org.apache.tez.engine.newapi.impl.EventMetaData.EventProducerConsumerType;
-import org.apache.tez.engine.newapi.impl.InputSpec;
-import org.apache.tez.engine.newapi.impl.OutputSpec;
-import org.apache.tez.engine.newapi.impl.TezEvent;
 
 public class Edge {
 

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/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 1ae9dcd..f2b2776 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
@@ -89,8 +89,8 @@ import org.apache.tez.dag.records.TezTaskAttemptID;
 import org.apache.tez.dag.records.TezTaskID;
 import org.apache.tez.dag.records.TezVertexID;
 import org.apache.tez.dag.utils.TezBuilderUtils;
-import org.apache.tez.engine.newapi.events.TaskStatusUpdateEvent;
-import org.apache.tez.engine.newapi.impl.TaskSpec;
+import org.apache.tez.engine.api.events.TaskStatusUpdateEvent;
+import org.apache.tez.engine.api.impl.TaskSpec;
 
 import com.google.common.annotations.VisibleForTesting;
 

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/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 92a1859..13fa915 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
@@ -75,7 +75,7 @@ import org.apache.tez.dag.history.events.TaskStartedEvent;
 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.engine.newapi.impl.TezEvent;
+import org.apache.tez.engine.api.impl.TezEvent;
 import org.apache.tez.engine.records.TezDependentTaskCompletionEvent;
 
 import com.google.common.annotations.VisibleForTesting;

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/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 0bcba4f..1ec1225 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
@@ -106,14 +106,14 @@ 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.engine.api.events.TaskAttemptFailedEvent;
+import org.apache.tez.engine.api.events.TaskStatusUpdateEvent;
+import org.apache.tez.engine.api.impl.EventMetaData;
+import org.apache.tez.engine.api.impl.InputSpec;
+import org.apache.tez.engine.api.impl.OutputSpec;
+import org.apache.tez.engine.api.impl.TezEvent;
 import org.apache.tez.engine.newapi.events.DataMovementEvent;
 import org.apache.tez.engine.newapi.events.InputFailedEvent;
-import org.apache.tez.engine.newapi.events.TaskAttemptFailedEvent;
-import org.apache.tez.engine.newapi.events.TaskStatusUpdateEvent;
-import org.apache.tez.engine.newapi.impl.EventMetaData;
-import org.apache.tez.engine.newapi.impl.InputSpec;
-import org.apache.tez.engine.newapi.impl.OutputSpec;
-import org.apache.tez.engine.newapi.impl.TezEvent;
 import org.apache.tez.engine.records.TezDependentTaskCompletionEvent;
 
 import com.google.common.annotations.VisibleForTesting;

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-dag/src/main/java/org/apache/tez/dag/app/rm/AMSchedulerEventTALaunchRequest.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/AMSchedulerEventTALaunchRequest.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/AMSchedulerEventTALaunchRequest.java
index 1c30b0b..14edd96 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/AMSchedulerEventTALaunchRequest.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/AMSchedulerEventTALaunchRequest.java
@@ -22,7 +22,7 @@ import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.tez.dag.app.ContainerContext;
 import org.apache.tez.dag.app.dag.TaskAttempt;
 import org.apache.tez.dag.records.TezTaskAttemptID;
-import org.apache.tez.engine.newapi.impl.TaskSpec;
+import org.apache.tez.engine.api.impl.TaskSpec;
 
 public class AMSchedulerEventTALaunchRequest extends AMSchedulerEvent {
 

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerEventAssignTA.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerEventAssignTA.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerEventAssignTA.java
index dd178fb..76e80f5 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerEventAssignTA.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerEventAssignTA.java
@@ -19,7 +19,7 @@ package org.apache.tez.dag.app.rm.container;
 
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.tez.dag.records.TezTaskAttemptID;
-import org.apache.tez.engine.newapi.impl.TaskSpec;
+import org.apache.tez.engine.api.impl.TaskSpec;
 
 public class AMContainerEventAssignTA extends AMContainerEvent {
 

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerImpl.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerImpl.java
index eccf92a..94dd580 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerImpl.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerImpl.java
@@ -54,7 +54,7 @@ import org.apache.tez.dag.app.rm.NMCommunicatorLaunchRequestEvent;
 import org.apache.tez.dag.app.rm.NMCommunicatorStopRequestEvent;
 import org.apache.tez.dag.records.TezTaskAttemptID;
 //import org.apache.tez.dag.app.dag.event.TaskAttemptEventDiagnosticsUpdate;
-import org.apache.tez.engine.newapi.impl.TaskSpec;
+import org.apache.tez.engine.api.impl.TaskSpec;
 
 @SuppressWarnings("rawtypes")
 public class AMContainerImpl implements AMContainer {

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerTask.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerTask.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerTask.java
index be1c08e..c0ef524 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerTask.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerTask.java
@@ -18,7 +18,7 @@
 
 package org.apache.tez.dag.app.rm.container;
 
-import org.apache.tez.engine.newapi.impl.TaskSpec;
+import org.apache.tez.engine.api.impl.TaskSpec;
 
 public class AMContainerTask {
   private final boolean shouldDie;

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/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 b5e283b..676e747 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
@@ -82,7 +82,7 @@ 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.engine.newapi.impl.TaskSpec;
+import org.apache.tez.engine.api.impl.TaskSpec;
 import org.junit.Test;
 import org.mockito.ArgumentCaptor;
 

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestContainerReuse.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestContainerReuse.java b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestContainerReuse.java
index 408f88a..3a6e008 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestContainerReuse.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestContainerReuse.java
@@ -29,7 +29,6 @@ import org.apache.hadoop.yarn.client.api.AMRMClient;
 import org.apache.hadoop.yarn.client.api.async.AMRMClientAsync;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.util.RackResolver;
-import org.apache.tez.common.TezEngineTaskContext;
 import org.apache.tez.dag.api.InputDescriptor;
 import org.apache.tez.dag.api.OutputDescriptor;
 import org.apache.tez.dag.api.ProcessorDescriptor;
@@ -57,9 +56,9 @@ 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.engine.newapi.impl.InputSpec;
-import org.apache.tez.engine.newapi.impl.OutputSpec;
-import org.apache.tez.engine.newapi.impl.TaskSpec;
+import org.apache.tez.engine.api.impl.InputSpec;
+import org.apache.tez.engine.api.impl.OutputSpec;
+import org.apache.tez.engine.api.impl.TaskSpec;
 import org.junit.Test;
 
 import com.google.common.collect.Lists;

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/common/ContainerTask.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/common/ContainerTask.java b/tez-engine/src/main/java/org/apache/tez/common/ContainerTask.java
index 3c18d9f..e90f7fa 100644
--- a/tez-engine/src/main/java/org/apache/tez/common/ContainerTask.java
+++ b/tez-engine/src/main/java/org/apache/tez/common/ContainerTask.java
@@ -22,7 +22,7 @@ import java.io.DataOutput;
 import java.io.IOException;
 
 import org.apache.hadoop.io.Writable;
-import org.apache.tez.engine.newapi.impl.TaskSpec;
+import org.apache.tez.engine.api.impl.TaskSpec;
 
 public class ContainerTask implements Writable {
 

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/common/RunningTaskContext.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/common/RunningTaskContext.java b/tez-engine/src/main/java/org/apache/tez/common/RunningTaskContext.java
deleted file mode 100644
index aac4095..0000000
--- a/tez-engine/src/main/java/org/apache/tez/common/RunningTaskContext.java
+++ /dev/null
@@ -1,70 +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.common;
-
-import java.io.IOException;
-
-import javax.crypto.SecretKey;
-
-import org.apache.hadoop.util.Progress;
-import org.apache.tez.engine.api.Partitioner;
-import org.apache.tez.engine.api.Processor;
-
-public class RunningTaskContext {
-  
-  protected SecretKey jobTokenSecret;
-  protected TezTaskReporter reporter;
-  protected Partitioner partitioner;
-  protected Processor combineProcessor;
-  protected TezTaskStatus status;
-  protected Progress progress = new Progress();
-
-  public Progress getProgress() {
-    return progress;
-  }
-
-  public void setJobTokenSecret(SecretKey jobTokenSecret) {
-    this.jobTokenSecret = jobTokenSecret;
-  }
-
-  public TezTaskStatus getStatus() {
-    return status;
-  }
-
-  public TezTaskReporter getTaskReporter() {
-    return reporter;
-  }
-
-  // TODO Doesn't belong here.
-  public Processor getCombineProcessor() {
-    return combineProcessor;
-  }
-
-  // TODO Doesn't belong here.
-  public Partitioner getPartitioner() {
-    return partitioner;
-  }
-
-  // TODO Doesn't belong here.
-  public SecretKey getJobTokenSecret() {
-    return jobTokenSecret;
-  }
-  
-  public void statusUpdate() throws IOException, InterruptedException {
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/common/TezEngineTaskContext.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/common/TezEngineTaskContext.java b/tez-engine/src/main/java/org/apache/tez/common/TezEngineTaskContext.java
deleted file mode 100644
index c012928..0000000
--- a/tez-engine/src/main/java/org/apache/tez/common/TezEngineTaskContext.java
+++ /dev/null
@@ -1,141 +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.common;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.tez.dag.api.DagTypeConverters;
-import org.apache.tez.dag.api.ProcessorDescriptor;
-import org.apache.tez.dag.api.records.DAGProtos.TezEntityDescriptorProto;
-import org.apache.tez.dag.records.TezTaskAttemptID;
-
-public class TezEngineTaskContext extends TezTaskContext {
-
-  // These two could be replaced by a TezConfiguration / DagSpec.
-  private List<InputSpec> inputSpecList;
-  private List<OutputSpec> outputSpecList;
-  private ProcessorDescriptor processorDescriptor;
-  
-  public TezEngineTaskContext() {
-    super();
-  }
-
-  public TezEngineTaskContext(TezTaskAttemptID taskAttemptID, String user,
-      String jobName, String vertexName, ProcessorDescriptor processorDescriptor,
-      List<InputSpec> inputSpecList, List<OutputSpec> outputSpecList) {
-    super(taskAttemptID, user, jobName, vertexName);
-    this.inputSpecList = inputSpecList;
-    this.outputSpecList = outputSpecList;
-    if (this.inputSpecList == null) {
-      inputSpecList = new ArrayList<InputSpec>(0);
-    }
-    if (this.outputSpecList == null) {
-      outputSpecList = new ArrayList<OutputSpec>(0);
-    }
-    this.inputSpecList = inputSpecList;
-    this.outputSpecList = outputSpecList;
-    this.processorDescriptor = processorDescriptor;
-  }
-
-  public String getRuntimeName() {
-    // FIXME. Add this to the DAG configuration, and fetch from there.
-    return "org.apache.tez.mapreduce.task.MRRuntimeTask";
-  }
-
-  public String getProcessorName() {
-    return processorDescriptor.getClassName();
-  }
-  
-  public byte[] getProcessorUserPayload() {
-    return processorDescriptor.getUserPayload();
-  }
-  
-  public List<InputSpec> getInputSpecList() {
-    return this.inputSpecList;
-  }
-  
-  public List<OutputSpec> getOutputSpecList() {
-    return this.outputSpecList;
-  }
-  
-  @Override
-  public void write(DataOutput out) throws IOException {
-    super.write(out);
-    byte[] procDesc = 
-        DagTypeConverters.convertToDAGPlan(processorDescriptor).toByteArray();
-    out.writeInt(procDesc.length);
-    out.write(procDesc);
-    out.writeInt(inputSpecList.size());
-    for (InputSpec inputSpec : inputSpecList) {
-      inputSpec.write(out);
-    }
-    out.writeInt(outputSpecList.size());
-    for (OutputSpec outputSpec : outputSpecList) {
-      outputSpec.write(out);
-    }
-  }
-  
-  @Override
-  public void readFields(DataInput in) throws IOException {
-    super.readFields(in);
-    int procDescLength = in.readInt();
-    // TODO at least 3 buffer copies here. Need to convert this to full PB
-    // TEZ-305
-    byte[] procDescBytes = new byte[procDescLength];
-    in.readFully(procDescBytes);
-    processorDescriptor = DagTypeConverters.convertProcessorDescriptorFromDAGPlan(
-        TezEntityDescriptorProto.parseFrom(procDescBytes)); 
-    int numInputSpecs = in.readInt();
-    inputSpecList = new ArrayList<InputSpec>(numInputSpecs);
-    for (int i = 0; i < numInputSpecs; i++) {
-      InputSpec inputSpec = new InputSpec();
-      inputSpec.readFields(in);
-      inputSpecList.add(inputSpec);
-    }
-    int numOutputSpecs = in.readInt();
-    outputSpecList = new ArrayList<OutputSpec>(numOutputSpecs);
-    for (int i = 0; i < numOutputSpecs; i++) {
-      OutputSpec outputSpec = new OutputSpec();
-      outputSpec.readFields(in);
-      outputSpecList.add(outputSpec);
-    }
-  }
-
-  @Override
-  public String toString() {
-    StringBuffer sb = new StringBuffer();
-    sb.append("processorName=" + getProcessorName()
-        + ", inputSpecListSize=" + inputSpecList.size()
-        + ", outputSpecListSize=" + outputSpecList.size());
-    sb.append(", inputSpecList=[");
-    for (InputSpec i : inputSpecList) {
-      sb.append("{" + i.toString() + "}, ");
-    }
-    sb.append("], outputSpecList=[");
-    for (OutputSpec i : outputSpecList) {
-      sb.append("{" + i.toString() + "}, ");
-    }
-    sb.append("]");
-    return sb.toString();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/common/TezTaskReporter.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/common/TezTaskReporter.java b/tez-engine/src/main/java/org/apache/tez/common/TezTaskReporter.java
deleted file mode 100644
index 1931e31..0000000
--- a/tez-engine/src/main/java/org/apache/tez/common/TezTaskReporter.java
+++ /dev/null
@@ -1,116 +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.common;
-
-import java.io.IOException;
-
-import org.apache.hadoop.ipc.ProtocolSignature;
-import org.apache.hadoop.util.Progressable;
-import org.apache.tez.common.counters.TezCounter;
-import org.apache.tez.dag.records.TezTaskAttemptID;
-import org.apache.tez.engine.api.Master;
-import org.apache.tez.engine.records.TezTaskDependencyCompletionEventsUpdate;
-
-public interface TezTaskReporter extends Progressable, Master {
-
-  public void setStatus(String status);
-
-  public float getProgress();
-
-  public void setProgress(float progress);
-  
-  public void progress();
-
-  public TezCounter getCounter(String group, String name);
-
-  public TezCounter getCounter(Enum<?> name);
-
-  public void incrCounter(String group, String counter, long amount);
-
-  public void incrCounter(Enum<?> key, long amount);
-
-  public void reportFatalError(TezTaskAttemptID taskAttemptId, 
-      Throwable exception, String logMsg);
-
-  public final TezTaskReporter NULL = new TezTaskReporter() {
-
-    @Override
-    public TezTaskDependencyCompletionEventsUpdate getDependentTasksCompletionEvents(
-        int fromEventIdx, int maxEventsToFetch,
-        TezTaskAttemptID reduce) {
-      return null;
-    }
-    
-    @Override
-    public void setStatus(String status) {
-    }
-    
-    @Override
-    public void setProgress(float progress) {
-    }
-    
-    @Override
-    public void progress() {
-    }
-    
-    @Override
-    public void incrCounter(Enum<?> key, long amount) {
-    }
-    
-    @Override
-    public void incrCounter(String group, String counter, long amount) {
-    }
-    
-    @Override
-    public float getProgress() {
-      return 0.0f;
-    }
-    
-    @Override
-    public TezCounter getCounter(Enum<?> name) {
-      return null;
-    }
-    
-    @Override
-    public TezCounter getCounter(String group, String name) {
-      return null;
-    }
-
-    @Override
-    public void reportFatalError(TezTaskAttemptID taskAttemptId,
-        Throwable exception, String logMsg) {
-      // TODO Auto-generated method stub
-      
-    }
-
-    @Override
-    public long getProtocolVersion(String protocol, long clientVersion)
-        throws IOException {
-      // TODO TEZAM3
-      return 0;
-    }
-
-    @Override
-    public ProtocolSignature getProtocolSignature(String protocol,
-        long clientVersion, int clientMethodsHash) throws IOException {
-      // TODO TEZAM3
-      return null;
-    }
-  };
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/common/TezTaskUmbilicalProtocol.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/common/TezTaskUmbilicalProtocol.java b/tez-engine/src/main/java/org/apache/tez/common/TezTaskUmbilicalProtocol.java
index 28991a8..7d81b4c 100644
--- a/tez-engine/src/main/java/org/apache/tez/common/TezTaskUmbilicalProtocol.java
+++ b/tez-engine/src/main/java/org/apache/tez/common/TezTaskUmbilicalProtocol.java
@@ -22,13 +22,15 @@ import java.io.IOException;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.ipc.ProtocolInfo;
+import org.apache.hadoop.ipc.VersionedProtocol;
 import org.apache.tez.common.records.ProceedToCompletionResponse;
 import org.apache.tez.dag.api.TezException;
 import org.apache.tez.dag.records.TezTaskAttemptID;
-import org.apache.tez.engine.api.Master;
-import org.apache.tez.engine.newapi.impl.TezHeartbeatRequest;
-import org.apache.tez.engine.newapi.impl.TezHeartbeatResponse;
+import org.apache.tez.engine.api.impl.TezHeartbeatRequest;
+import org.apache.tez.engine.api.impl.TezHeartbeatResponse;
 import org.apache.tez.engine.records.OutputContext;
+import org.apache.tez.engine.records.TezTaskDependencyCompletionEventsUpdate;
 
 /** Protocol that task child process uses to contact its parent process.  The
  * parent is a daemon which which polls the central master for a new map or
@@ -36,7 +38,8 @@ import org.apache.tez.engine.records.OutputContext;
  * and parent is via this protocol. */
 @InterfaceAudience.Private
 @InterfaceStability.Stable
-public interface TezTaskUmbilicalProtocol extends Master {
+@ProtocolInfo(protocolName = "TezTaskUmbilicalProtocol", protocolVersion = 1)
+public interface TezTaskUmbilicalProtocol extends VersionedProtocol {
 
   public static final long versionID = 19L;
 
@@ -68,4 +71,7 @@ public interface TezTaskUmbilicalProtocol extends Master {
   public TezHeartbeatResponse heartbeat(TezHeartbeatRequest request)
       throws IOException, TezException;
 
+  public TezTaskDependencyCompletionEventsUpdate getDependentTasksCompletionEvents(
+      int fromEventIdx, int maxEventsToFetch,
+      TezTaskAttemptID taskAttemptId);
 }

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/api/KVReader.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/api/KVReader.java b/tez-engine/src/main/java/org/apache/tez/engine/api/KVReader.java
new file mode 100644
index 0000000..150b598
--- /dev/null
+++ b/tez-engine/src/main/java/org/apache/tez/engine/api/KVReader.java
@@ -0,0 +1,81 @@
+/**
+ * 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.engine.api;
+
+import java.io.IOException;
+
+import org.apache.tez.engine.newapi.Reader;
+
+/**
+ * A key/value(s) pair based {@link Reader}.
+ * 
+ * Example usage
+ * <code>
+ * while (kvReader.next()) {
+ *   KVRecord kvRecord = getCurrentKV();
+ *   Object key =  kvRecord.getKey();
+ *   Iterable values = kvRecord.getValues();
+ * </code>
+ *
+ */
+public interface KVReader extends Reader {
+
+  /**
+   * Moves to the next key/values(s) pair
+   * 
+   * @return true if another key/value(s) pair exists, false if there are no more.
+   * @throws IOException
+   *           if an error occurs
+   */
+  public boolean next() throws IOException;
+
+  /**
+   * Return the current key/value(s) pair. Use moveToNext() to advance.
+   * @return
+   * @throws IOException
+   */
+  public KVRecord getCurrentKV() throws IOException;
+  
+  // TODO NEWTEZ Move this to getCurrentKey and getCurrentValue independently. Otherwise usage pattern seems to be getCurrentKV.getKey, getCurrentKV.getValue
+  
+  // TODO NEWTEZ KVRecord which does not need to return a list!
+  // TODO NEWTEZ Parameterize this
+  /**
+   * Represents a key and an associated set of values
+   *
+   */
+  public static class KVRecord {
+
+    private Object key;
+    private Iterable<Object> values;
+
+    public KVRecord(Object key, Iterable<Object> values) {
+      this.key = key;
+      this.values = values;
+    }
+
+    public Object getKey() {
+      return this.key;
+    }
+
+    public Iterable<Object> getValues() {
+      return this.values;
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/api/KVWriter.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/api/KVWriter.java b/tez-engine/src/main/java/org/apache/tez/engine/api/KVWriter.java
new file mode 100644
index 0000000..079d488
--- /dev/null
+++ b/tez-engine/src/main/java/org/apache/tez/engine/api/KVWriter.java
@@ -0,0 +1,40 @@
+/**
+ * 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.engine.api;
+
+import java.io.IOException;
+
+import org.apache.tez.engine.newapi.Writer;
+
+/**
+ * A key/value(s) pair based {@link Writer}
+ */
+public interface KVWriter extends Writer {
+  /**
+   * Writes a key/value pair.
+   * 
+   * @param key
+   *          the key to write
+   * @param value
+   *          the value to write
+   * @throws IOException
+   *           if an error occurs
+   */
+  public void write(Object key, Object value) throws IOException;
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/api/events/TaskAttemptCompletedEvent.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/api/events/TaskAttemptCompletedEvent.java b/tez-engine/src/main/java/org/apache/tez/engine/api/events/TaskAttemptCompletedEvent.java
new file mode 100644
index 0000000..3a90f56
--- /dev/null
+++ b/tez-engine/src/main/java/org/apache/tez/engine/api/events/TaskAttemptCompletedEvent.java
@@ -0,0 +1,28 @@
+/**
+ * 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.engine.api.events;
+
+import org.apache.tez.engine.newapi.Event;
+
+public class TaskAttemptCompletedEvent extends Event {
+
+  public TaskAttemptCompletedEvent() {
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/api/events/TaskAttemptFailedEvent.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/api/events/TaskAttemptFailedEvent.java b/tez-engine/src/main/java/org/apache/tez/engine/api/events/TaskAttemptFailedEvent.java
new file mode 100644
index 0000000..bd0bc04
--- /dev/null
+++ b/tez-engine/src/main/java/org/apache/tez/engine/api/events/TaskAttemptFailedEvent.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.engine.api.events;
+
+import org.apache.tez.engine.newapi.Event;
+
+public class TaskAttemptFailedEvent extends Event {
+
+  private final String diagnostics;
+
+  public TaskAttemptFailedEvent(String diagnostics) {
+    this.diagnostics = diagnostics;
+  }
+
+  public String getDiagnostics() {
+    return diagnostics;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/api/events/TaskStatusUpdateEvent.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/api/events/TaskStatusUpdateEvent.java b/tez-engine/src/main/java/org/apache/tez/engine/api/events/TaskStatusUpdateEvent.java
new file mode 100644
index 0000000..c0d77da
--- /dev/null
+++ b/tez-engine/src/main/java/org/apache/tez/engine/api/events/TaskStatusUpdateEvent.java
@@ -0,0 +1,70 @@
+/**
+ * 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.engine.api.events;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hadoop.io.Writable;
+import org.apache.tez.common.counters.TezCounters;
+import org.apache.tez.engine.newapi.Event;
+
+public class TaskStatusUpdateEvent extends Event implements Writable {
+
+  private TezCounters tezCounters;
+  private float progress;
+
+  public TaskStatusUpdateEvent() {
+  }
+
+  public TaskStatusUpdateEvent(TezCounters tezCounters, float progress) {
+    this.tezCounters = tezCounters;
+    this.progress = progress;
+  }
+
+  public TezCounters getCounters() {
+    return tezCounters;
+  }
+
+  public float getProgress() {
+    return progress;
+  }
+
+  @Override
+  public void write(DataOutput out) throws IOException {
+    out.writeFloat(progress);
+    if (tezCounters != null) {
+      out.writeBoolean(true);
+      tezCounters.write(out);
+    } else {
+      out.writeBoolean(false);
+    }
+  }
+
+  @Override
+  public void readFields(DataInput in) throws IOException {
+    progress = in.readFloat();
+    if (in.readBoolean()) {
+      tezCounters = new TezCounters();
+      tezCounters.readFields(in);
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/api/impl/EventMetaData.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/api/impl/EventMetaData.java b/tez-engine/src/main/java/org/apache/tez/engine/api/impl/EventMetaData.java
new file mode 100644
index 0000000..64df7bb
--- /dev/null
+++ b/tez-engine/src/main/java/org/apache/tez/engine/api/impl/EventMetaData.java
@@ -0,0 +1,152 @@
+/**
+ * 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.engine.api.impl;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hadoop.io.Writable;
+import org.apache.tez.dag.records.TezTaskAttemptID;
+
+/**
+ * Class that encapsulates all the information to identify the unique
+ * object that either generated an Event or is the recipient of an Event.
+ */
+public class EventMetaData implements Writable {
+
+  public static enum EventProducerConsumerType {
+    INPUT,
+    PROCESSOR,
+    OUTPUT,
+    SYSTEM
+  }
+
+  /**
+   * Producer Type ( one of Input/Output/Processor ) that generated the Event
+   * or Consumer Type that will consume the Event.
+   */
+  private EventProducerConsumerType producerConsumerType;
+
+  /**
+   * Name of the vertex where the event was generated.
+   */
+  private String taskVertexName;
+
+  /**
+   * Name of the vertex to which the Input or Output is connected to.
+   */
+  private String edgeVertexName;
+
+  /**
+   * i'th physical input/output that this event maps to.
+   */
+  private int index;
+
+  /**
+   * Task Attempt ID
+   */
+  private TezTaskAttemptID taskAttemptID;
+
+  public EventMetaData() {
+  }
+
+  public EventMetaData(EventProducerConsumerType generator,
+      String taskVertexName, String edgeVertexName,
+      TezTaskAttemptID taskAttemptID) {
+    this.producerConsumerType = generator;
+    this.taskVertexName = taskVertexName;
+    this.edgeVertexName = edgeVertexName;
+    this.taskAttemptID = taskAttemptID;
+  }
+
+  public EventProducerConsumerType getEventGenerator() {
+    return producerConsumerType;
+  }
+
+  public TezTaskAttemptID getTaskAttemptID() {
+    return taskAttemptID;
+  }
+
+  public String getTaskVertexName() {
+    return taskVertexName;
+  }
+
+  public String getEdgeVertexName() {
+    return edgeVertexName;
+  }
+
+  @Override
+  public void write(DataOutput out) throws IOException {
+    out.writeInt(producerConsumerType.ordinal());
+    if (taskVertexName != null) {
+      out.writeBoolean(true);
+      out.writeUTF(taskVertexName);
+    } else {
+      out.writeBoolean(false);
+    }
+    if (edgeVertexName != null) {
+      out.writeBoolean(true);
+      out.writeUTF(edgeVertexName);
+    } else {
+      out.writeBoolean(false);
+    }
+    if(taskAttemptID != null) {
+      out.writeBoolean(true);
+      taskAttemptID.write(out);
+    } else {
+      out.writeBoolean(false);
+    }
+    
+    out.writeInt(index);
+  }
+
+  @Override
+  public void readFields(DataInput in) throws IOException {
+    producerConsumerType = EventProducerConsumerType.values()[in.readInt()];
+    if (in.readBoolean()) {
+      taskVertexName = in.readUTF();
+    }
+    if (in.readBoolean()) {
+      edgeVertexName = in.readUTF();
+    }
+    if (in.readBoolean()) {
+      taskAttemptID = new TezTaskAttemptID();
+      taskAttemptID.readFields(in);
+    }
+    index = in.readInt();
+  }
+
+  public int getIndex() {
+    return index;
+  }
+
+  public void setIndex(int index) {
+    this.index = index;
+  }
+
+  @Override
+  public String toString() {
+    return "{ producerConsumerType=" + producerConsumerType
+        + ", taskVertexName=" + taskVertexName
+        + ", edgeVertexName=" + edgeVertexName
+        + ", taskAttemptId=" + taskAttemptID
+        + ", index=" + index + " }";
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/api/impl/EventType.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/api/impl/EventType.java b/tez-engine/src/main/java/org/apache/tez/engine/api/impl/EventType.java
new file mode 100644
index 0000000..52fc10d
--- /dev/null
+++ b/tez-engine/src/main/java/org/apache/tez/engine/api/impl/EventType.java
@@ -0,0 +1,29 @@
+/**
+ * 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.engine.api.impl;
+
+public enum EventType {
+  TASK_ATTEMPT_COMPLETED_EVENT,
+  TASK_ATTEMPT_FAILED_EVENT,
+  DATA_MOVEMENT_EVENT,
+  INPUT_READ_ERROR_EVENT,
+  INPUT_FAILED_EVENT,
+  INTPUT_INFORMATION_EVENT,
+  TASK_STATUS_UPDATE_EVENT
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/api/impl/InputSpec.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/api/impl/InputSpec.java b/tez-engine/src/main/java/org/apache/tez/engine/api/impl/InputSpec.java
new file mode 100644
index 0000000..a9ef333
--- /dev/null
+++ b/tez-engine/src/main/java/org/apache/tez/engine/api/impl/InputSpec.java
@@ -0,0 +1,88 @@
+/**
+ * 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.engine.api.impl;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hadoop.io.Writable;
+import org.apache.tez.dag.api.DagTypeConverters;
+import org.apache.tez.dag.api.InputDescriptor;
+import org.apache.tez.dag.api.records.DAGProtos.TezEntityDescriptorProto;
+
+public class InputSpec implements Writable {
+
+  private String sourceVertexName;
+  private InputDescriptor inputDescriptor;
+  private int physicalEdgeCount;
+
+  public InputSpec() {
+  }
+
+  public InputSpec(String sourceVertexName, InputDescriptor inputDescriptor,
+      int physicalEdgeCount) {
+    this.sourceVertexName = sourceVertexName;
+    this.inputDescriptor = inputDescriptor;
+    this.physicalEdgeCount = physicalEdgeCount;
+  }
+
+  public String getSourceVertexName() {
+    return sourceVertexName;
+  }
+
+  public InputDescriptor getInputDescriptor() {
+    return inputDescriptor;
+  }
+
+  public int getPhysicalEdgeCount() {
+    return physicalEdgeCount;
+  }
+
+  @Override
+  public void write(DataOutput out) throws IOException {
+    // TODONEWTEZ convert to PB
+    out.writeUTF(sourceVertexName);
+    out.writeInt(physicalEdgeCount);
+    byte[] inputDescBytes =
+        DagTypeConverters.convertToDAGPlan(inputDescriptor).toByteArray();
+    out.writeInt(inputDescBytes.length);
+    out.write(inputDescBytes);
+  }
+
+  @Override
+  public void readFields(DataInput in) throws IOException {
+    sourceVertexName = in.readUTF();
+    physicalEdgeCount = in.readInt();
+    int inputDescLen = in.readInt();
+    byte[] inputDescBytes = new byte[inputDescLen];
+    in.readFully(inputDescBytes);
+    inputDescriptor =
+        DagTypeConverters.convertInputDescriptorFromDAGPlan(
+            TezEntityDescriptorProto.parseFrom(inputDescBytes));
+  }
+
+  public String toString() {
+    return "{ sourceVertexName=" + sourceVertexName
+        + ", physicalEdgeCount" + physicalEdgeCount
+        + ", inputClassName=" + inputDescriptor.getClassName()
+        + " }";
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/api/impl/OutputSpec.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/api/impl/OutputSpec.java b/tez-engine/src/main/java/org/apache/tez/engine/api/impl/OutputSpec.java
new file mode 100644
index 0000000..3a1d5d8
--- /dev/null
+++ b/tez-engine/src/main/java/org/apache/tez/engine/api/impl/OutputSpec.java
@@ -0,0 +1,87 @@
+/**
+ * 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.engine.api.impl;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hadoop.io.Writable;
+import org.apache.tez.dag.api.DagTypeConverters;
+import org.apache.tez.dag.api.OutputDescriptor;
+import org.apache.tez.dag.api.records.DAGProtos.TezEntityDescriptorProto;
+
+public class OutputSpec implements Writable {
+
+  private String destinationVertexName;
+  private OutputDescriptor outputDescriptor;
+  private int physicalEdgeCount;
+
+  public OutputSpec() {
+  }
+
+  public OutputSpec(String destinationVertexName,
+      OutputDescriptor outputDescriptor, int physicalEdgeCount) {
+    this.destinationVertexName = destinationVertexName;
+    this.outputDescriptor = outputDescriptor;
+    this.physicalEdgeCount = physicalEdgeCount;
+  }
+
+  public String getDestinationVertexName() {
+    return destinationVertexName;
+  }
+
+  public OutputDescriptor getOutputDescriptor() {
+    return outputDescriptor;
+  }
+
+  public int getPhysicalEdgeCount() {
+    return physicalEdgeCount;
+  }
+
+  @Override
+  public void write(DataOutput out) throws IOException {
+    // TODONEWTEZ convert to PB
+    out.writeUTF(destinationVertexName);
+    out.writeInt(physicalEdgeCount);
+    byte[] inputDescBytes =
+        DagTypeConverters.convertToDAGPlan(outputDescriptor).toByteArray();
+    out.writeInt(inputDescBytes.length);
+    out.write(inputDescBytes);
+  }
+
+  @Override
+  public void readFields(DataInput in) throws IOException {
+    destinationVertexName = in.readUTF();
+    physicalEdgeCount = in.readInt();
+    int inputDescLen = in.readInt();
+    byte[] inputDescBytes = new byte[inputDescLen];
+    in.readFully(inputDescBytes);
+    outputDescriptor =
+        DagTypeConverters.convertOutputDescriptorFromDAGPlan(
+            TezEntityDescriptorProto.parseFrom(inputDescBytes));
+  }
+
+  public String toString() {
+    return "{ destinationVertexName=" + destinationVertexName
+        + ", physicalEdgeCount" + physicalEdgeCount
+        + ", outputClassName=" + outputDescriptor.getClassName()
+        + " }";
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TaskSpec.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TaskSpec.java b/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TaskSpec.java
new file mode 100644
index 0000000..6527777
--- /dev/null
+++ b/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TaskSpec.java
@@ -0,0 +1,146 @@
+/**
+ * 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.engine.api.impl;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.io.Writable;
+import org.apache.tez.dag.api.DagTypeConverters;
+import org.apache.tez.dag.api.ProcessorDescriptor;
+import org.apache.tez.dag.api.records.DAGProtos.TezEntityDescriptorProto;
+import org.apache.tez.dag.records.TezTaskAttemptID;
+
+public class TaskSpec implements Writable {
+
+  private TezTaskAttemptID taskAttemptId;
+  private String vertexName;
+  private String user;
+  private ProcessorDescriptor processorDescriptor;
+  private List<InputSpec> inputSpecList;
+  private List<OutputSpec> outputSpecList;
+
+  public TaskSpec() {
+  }
+
+  // TODO NEWTEZ Remove user
+  public TaskSpec(TezTaskAttemptID taskAttemptID, String user,
+      String vertexName, ProcessorDescriptor processorDescriptor,
+      List<InputSpec> inputSpecList, List<OutputSpec> outputSpecList) {
+    this.taskAttemptId = taskAttemptID;
+    this.vertexName = vertexName;
+    this.user = user;
+    this.processorDescriptor = processorDescriptor;
+    this.inputSpecList = inputSpecList;
+    this.outputSpecList = outputSpecList;
+  }
+
+  public String getVertexName() {
+    return vertexName;
+  }
+
+  public TezTaskAttemptID getTaskAttemptID() {
+    return taskAttemptId;
+  }
+
+  public String getUser() {
+    return user;
+  }
+
+  public ProcessorDescriptor getProcessorDescriptor() {
+    return processorDescriptor;
+  }
+
+  public List<InputSpec> getInputs() {
+    return inputSpecList;
+  }
+
+  public List<OutputSpec> getOutputs() {
+    return outputSpecList;
+  }
+
+  @Override
+  public void write(DataOutput out) throws IOException {
+    taskAttemptId.write(out);
+    out.writeUTF(vertexName);
+    byte[] procDesc =
+        DagTypeConverters.convertToDAGPlan(processorDescriptor).toByteArray();
+    out.writeInt(procDesc.length);
+    out.write(procDesc);
+    out.writeInt(inputSpecList.size());
+    for (InputSpec inputSpec : inputSpecList) {
+      inputSpec.write(out);
+    }
+    out.writeInt(outputSpecList.size());
+    for (OutputSpec outputSpec : outputSpecList) {
+      outputSpec.write(out);
+    }
+  }
+
+  @Override
+  public void readFields(DataInput in) throws IOException {
+    taskAttemptId = new TezTaskAttemptID();
+    taskAttemptId.readFields(in);
+    vertexName = in.readUTF();
+    int procDescLength = in.readInt();
+    // TODO at least 3 buffer copies here. Need to convert this to full PB
+    // TEZ-305
+    byte[] procDescBytes = new byte[procDescLength];
+    in.readFully(procDescBytes);
+    processorDescriptor =
+        DagTypeConverters.convertProcessorDescriptorFromDAGPlan(
+            TezEntityDescriptorProto.parseFrom(procDescBytes));
+    int numInputSpecs = in.readInt();
+    inputSpecList = new ArrayList<InputSpec>(numInputSpecs);
+    for (int i = 0; i < numInputSpecs; i++) {
+      InputSpec inputSpec = new InputSpec();
+      inputSpec.readFields(in);
+      inputSpecList.add(inputSpec);
+    }
+    int numOutputSpecs = in.readInt();
+    outputSpecList = new ArrayList<OutputSpec>(numOutputSpecs);
+    for (int i = 0; i < numOutputSpecs; i++) {
+      OutputSpec outputSpec = new OutputSpec();
+      outputSpec.readFields(in);
+      outputSpecList.add(outputSpec);
+    }
+  }
+
+  @Override
+  public String toString() {
+    StringBuffer sb = new StringBuffer();
+    sb.append("TaskAttemptID:" + taskAttemptId);
+    sb.append("processorName=" + processorDescriptor.getClassName()
+        + ", inputSpecListSize=" + inputSpecList.size()
+        + ", outputSpecListSize=" + outputSpecList.size());
+    sb.append(", inputSpecList=[");
+    for (InputSpec i : inputSpecList) {
+      sb.append("{" + i.toString() + "}, ");
+    }
+    sb.append("], outputSpecList=[");
+    for (OutputSpec i : outputSpecList) {
+      sb.append("{" + i.toString() + "}, ");
+    }
+    sb.append("]");
+    return sb.toString();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezEvent.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezEvent.java b/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezEvent.java
new file mode 100644
index 0000000..9d0228d
--- /dev/null
+++ b/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezEvent.java
@@ -0,0 +1,248 @@
+/**
+ * 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.engine.api.impl;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hadoop.io.Writable;
+import org.apache.tez.dag.api.TezUncheckedException;
+import org.apache.tez.engine.api.events.TaskAttemptCompletedEvent;
+import org.apache.tez.engine.api.events.TaskAttemptFailedEvent;
+import org.apache.tez.engine.api.events.TaskStatusUpdateEvent;
+import org.apache.tez.engine.api.events.EventProtos.DataMovementEventProto;
+import org.apache.tez.engine.api.events.EventProtos.InputFailedEventProto;
+import org.apache.tez.engine.api.events.EventProtos.InputInformationEventProto;
+import org.apache.tez.engine.api.events.EventProtos.InputReadErrorEventProto;
+import org.apache.tez.engine.api.events.SystemEventProtos.TaskAttemptCompletedEventProto;
+import org.apache.tez.engine.api.events.SystemEventProtos.TaskAttemptFailedEventProto;
+import org.apache.tez.engine.newapi.Event;
+import org.apache.tez.engine.newapi.events.DataMovementEvent;
+import org.apache.tez.engine.newapi.events.InputFailedEvent;
+import org.apache.tez.engine.newapi.events.InputInformationEvent;
+import org.apache.tez.engine.newapi.events.InputReadErrorEvent;
+
+import com.google.protobuf.ByteString;
+
+public class TezEvent implements Writable {
+
+  private EventType eventType;
+
+  private Event event;
+
+  private EventMetaData sourceInfo;
+
+  private EventMetaData destinationInfo;
+
+  public TezEvent() {
+  }
+
+  public TezEvent(Event event, EventMetaData sourceInfo) {
+    this.event = event;
+    this.setSourceInfo(sourceInfo);
+    if (event instanceof DataMovementEvent) {
+      eventType = EventType.DATA_MOVEMENT_EVENT;
+    } else if (event instanceof InputReadErrorEvent) {
+      eventType = EventType.INPUT_READ_ERROR_EVENT;
+    } else if (event instanceof TaskAttemptFailedEvent) {
+      eventType = EventType.TASK_ATTEMPT_FAILED_EVENT;
+    } else if (event instanceof TaskAttemptCompletedEvent) {
+      eventType = EventType.TASK_ATTEMPT_COMPLETED_EVENT;
+    } else if (event instanceof InputInformationEvent) {
+      eventType = EventType.INTPUT_INFORMATION_EVENT;
+    } else if (event instanceof InputFailedEvent) {
+      eventType = EventType.INPUT_FAILED_EVENT;
+    } else if (event instanceof TaskStatusUpdateEvent) {
+      eventType = EventType.TASK_STATUS_UPDATE_EVENT;
+    } else {
+      throw new TezUncheckedException("Unknown event, event="
+          + event.getClass().getName());
+    }
+  }
+
+  public Event getEvent() {
+    return event;
+  }
+
+  public EventMetaData getSourceInfo() {
+    return sourceInfo;
+  }
+
+  public void setSourceInfo(EventMetaData sourceInfo) {
+    this.sourceInfo = sourceInfo;
+  }
+
+  public EventMetaData getDestinationInfo() {
+    return destinationInfo;
+  }
+
+  public void setDestinationInfo(EventMetaData destinationInfo) {
+    this.destinationInfo = destinationInfo;
+  }
+
+  public EventType getEventType() {
+    return eventType;
+  }
+
+  private void serializeEvent(DataOutput out) throws IOException {
+    if (event == null) {
+      out.writeBoolean(false);
+      return;
+    }
+    out.writeBoolean(true);
+    out.writeInt(eventType.ordinal());
+    if (eventType.equals(EventType.TASK_STATUS_UPDATE_EVENT)) {
+      // TODO NEWTEZ convert to PB
+      TaskStatusUpdateEvent sEvt = (TaskStatusUpdateEvent) event;
+      sEvt.write(out);
+    } else {
+      byte[] eventBytes = null;
+      switch (eventType) {
+      case DATA_MOVEMENT_EVENT:
+        DataMovementEvent dmEvt = (DataMovementEvent) event;
+        eventBytes = DataMovementEventProto.newBuilder()
+          .setSourceIndex(dmEvt.getSourceIndex())
+          .setTargetIndex(dmEvt.getTargetIndex())
+          .setUserPayload(ByteString.copyFrom(dmEvt.getUserPayload()))
+          .build().toByteArray();
+        break;
+      case INPUT_READ_ERROR_EVENT:
+        InputReadErrorEvent ideEvt = (InputReadErrorEvent) event;
+        eventBytes = InputReadErrorEventProto.newBuilder()
+            .setIndex(ideEvt.getIndex())
+            .setDiagnostics(ideEvt.getDiagnostics())
+            .build().toByteArray();
+        break;
+      case TASK_ATTEMPT_FAILED_EVENT:
+        TaskAttemptFailedEvent tfEvt = (TaskAttemptFailedEvent) event;
+        eventBytes = TaskAttemptFailedEventProto.newBuilder()
+            .setDiagnostics(tfEvt.getDiagnostics())
+            .build().toByteArray();
+        break;
+      case TASK_ATTEMPT_COMPLETED_EVENT:
+        eventBytes = TaskAttemptCompletedEventProto.newBuilder()
+            .build().toByteArray();
+        break;
+      case INPUT_FAILED_EVENT:
+        InputFailedEvent ifEvt = (InputFailedEvent) event;
+        eventBytes = InputFailedEventProto.newBuilder()
+            .setSourceIndex(ifEvt.getSourceIndex())
+            .setTargetIndex(ifEvt.getTargetIndex())
+            .setVersion(ifEvt.getVersion()).build().toByteArray();
+      case INTPUT_INFORMATION_EVENT:
+        InputInformationEvent iEvt = (InputInformationEvent) event;
+        eventBytes = InputInformationEventProto.newBuilder()
+            .setUserPayload(ByteString.copyFrom(iEvt.getUserPayload()))
+            .build().toByteArray();
+      default:
+        throw new TezUncheckedException("Unknown TezEvent"
+           + ", type=" + eventType);
+      }
+      out.writeInt(eventBytes.length);
+      out.write(eventBytes);
+    }
+  }
+
+  private void deserializeEvent(DataInput in) throws IOException {
+    if (!in.readBoolean()) {
+      event = null;
+      return;
+    }
+    eventType = EventType.values()[in.readInt()];
+    if (eventType.equals(EventType.TASK_STATUS_UPDATE_EVENT)) {
+      // TODO NEWTEZ convert to PB
+      event = new TaskStatusUpdateEvent();
+      ((TaskStatusUpdateEvent)event).readFields(in);
+    } else {
+      int eventBytesLen = in.readInt();
+      byte[] eventBytes = new byte[eventBytesLen];
+      in.readFully(eventBytes);
+      switch (eventType) {
+      case DATA_MOVEMENT_EVENT:
+        DataMovementEventProto dmProto =
+            DataMovementEventProto.parseFrom(eventBytes);
+        event = new DataMovementEvent(dmProto.getSourceIndex(),
+            dmProto.getTargetIndex(),
+            dmProto.getUserPayload().toByteArray());
+        break;
+      case INPUT_READ_ERROR_EVENT:
+        InputReadErrorEventProto ideProto =
+            InputReadErrorEventProto.parseFrom(eventBytes);
+        event = new InputReadErrorEvent(ideProto.getDiagnostics(),
+            ideProto.getIndex(), ideProto.getVersion());
+        break;
+      case TASK_ATTEMPT_FAILED_EVENT:
+        TaskAttemptFailedEventProto tfProto =
+            TaskAttemptFailedEventProto.parseFrom(eventBytes);
+        event = new TaskAttemptFailedEvent(tfProto.getDiagnostics());
+        break;
+      case TASK_ATTEMPT_COMPLETED_EVENT:
+        event = new TaskAttemptCompletedEvent();
+        break;
+      case INPUT_FAILED_EVENT:
+        InputFailedEventProto ifProto =
+            InputFailedEventProto.parseFrom(eventBytes);
+        event = new InputFailedEvent(ifProto.getSourceIndex(),
+            ifProto.getTargetIndex(), ifProto.getVersion());
+        break;
+      case INTPUT_INFORMATION_EVENT:
+        InputInformationEventProto infoProto =
+            InputInformationEventProto.parseFrom(eventBytes);
+        event = new InputInformationEvent(
+            infoProto.getUserPayload().toByteArray());
+        break;
+      default:
+        throw new TezUncheckedException("Unknown TezEvent"
+           + ", type=" + eventType);
+      }
+    }
+  }
+
+  @Override
+  public void write(DataOutput out) throws IOException {
+    serializeEvent(out);
+    if (sourceInfo != null) {
+      out.writeBoolean(true);
+      sourceInfo.write(out);
+    } else {
+      out.writeBoolean(false);
+    }
+    if (destinationInfo != null) {
+      out.writeBoolean(true);
+      destinationInfo.write(out);
+    } else {
+      out.writeBoolean(false);
+    }
+  }
+
+  @Override
+  public void readFields(DataInput in) throws IOException {
+    deserializeEvent(in);
+    if (in.readBoolean()) {
+      sourceInfo = new EventMetaData();
+      sourceInfo.readFields(in);
+    }
+    if (in.readBoolean()) {
+      destinationInfo = new EventMetaData();
+      destinationInfo.readFields(in);
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezHeartbeatRequest.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezHeartbeatRequest.java b/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezHeartbeatRequest.java
new file mode 100644
index 0000000..dc1a447
--- /dev/null
+++ b/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezHeartbeatRequest.java
@@ -0,0 +1,137 @@
+/**
+ * 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.engine.api.impl;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.Writable;
+import org.apache.tez.dag.records.TezTaskAttemptID;
+
+
+public class TezHeartbeatRequest implements Writable {
+
+  private String containerIdentifier;
+  private List<TezEvent> events;
+  private TezTaskAttemptID currentTaskAttemptID;
+  private int startIndex;
+  private int maxEvents;
+  private long requestId;
+
+  public TezHeartbeatRequest() {
+  }
+
+  public TezHeartbeatRequest(long requestId, List<TezEvent> events,
+      String containerIdentifier, TezTaskAttemptID taskAttemptID,
+      int startIndex, int maxEvents) {
+    this.containerIdentifier = containerIdentifier;
+    this.requestId = requestId;
+    this.events = Collections.unmodifiableList(events);
+    this.startIndex = startIndex;
+    this.maxEvents = maxEvents;
+    this.currentTaskAttemptID = taskAttemptID;
+  }
+
+  public String getContainerIdentifier() {
+    return containerIdentifier;
+  }
+
+  public List<TezEvent> getEvents() {
+    return events;
+  }
+
+  public int getStartIndex() {
+    return startIndex;
+  }
+
+  public int getMaxEvents() {
+    return maxEvents;
+  }
+
+  public long getRequestId() {
+    return requestId;
+  }
+
+  public TezTaskAttemptID getCurrentTaskAttemptID() {
+    return currentTaskAttemptID;
+  }
+
+  @Override
+  public void write(DataOutput out) throws IOException {
+    if (events != null) {
+      out.writeBoolean(true);
+      out.writeInt(events.size());
+      for (TezEvent e : events) {
+        e.write(out);
+      }
+    } else {
+      out.writeBoolean(false);
+    }
+    if (currentTaskAttemptID != null) {
+      out.writeBoolean(true);
+      currentTaskAttemptID.write(out);
+    } else {
+      out.writeBoolean(false);
+    }
+    out.writeInt(startIndex);
+    out.writeInt(maxEvents);
+    out.writeLong(requestId);
+    Text.writeString(out, containerIdentifier);
+  }
+
+  @Override
+  public void readFields(DataInput in) throws IOException {
+    if (in.readBoolean()) {
+      int eventsCount = in.readInt();
+      events = new ArrayList<TezEvent>(eventsCount);
+      for (int i = 0; i < eventsCount; ++i) {
+        TezEvent e = new TezEvent();
+        e.readFields(in);
+        events.add(e);
+      }
+    }
+    if (in.readBoolean()) {
+      currentTaskAttemptID = new TezTaskAttemptID();
+      currentTaskAttemptID.readFields(in);
+    } else {
+      currentTaskAttemptID = null;
+    }
+    startIndex = in.readInt();
+    maxEvents = in.readInt();
+    requestId = in.readLong();
+    containerIdentifier = Text.readString(in);
+  }
+
+  @Override
+  public String toString() {
+    return "{ "
+        + " containerId=" + containerIdentifier
+        + ", requestId=" + requestId
+        + ", startIndex=" + startIndex
+        + ", maxEventsToGet=" + maxEvents
+        + ", taskAttemptId" + currentTaskAttemptID
+        + ", eventCount=" + (events != null ? events.size() : 0)
+        + " }";
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezHeartbeatResponse.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezHeartbeatResponse.java b/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezHeartbeatResponse.java
new file mode 100644
index 0000000..22ae7eb
--- /dev/null
+++ b/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezHeartbeatResponse.java
@@ -0,0 +1,105 @@
+/**
+ * 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.engine.api.impl;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.hadoop.io.Writable;
+
+public class TezHeartbeatResponse implements Writable {
+
+  private long lastRequestId;
+  private boolean shouldDie = false;
+  private List<TezEvent> events;
+
+  public TezHeartbeatResponse() {
+  }
+
+  public TezHeartbeatResponse(List<TezEvent> events) {
+    this.events = Collections.unmodifiableList(events);
+  }
+
+  public List<TezEvent> getEvents() {
+    return events;
+  }
+
+  public boolean shouldDie() {
+    return shouldDie;
+  }
+
+  public long getLastRequestId() {
+    return lastRequestId;
+  }
+
+  public void setEvents(List<TezEvent> events) {
+    this.events = Collections.unmodifiableList(events);
+  }
+
+  public void setLastRequestId(long lastRequestId ) {
+    this.lastRequestId = lastRequestId;
+  }
+
+  public void setShouldDie() {
+    this.shouldDie = true;
+  }
+
+  @Override
+  public void write(DataOutput out) throws IOException {
+    out.writeLong(lastRequestId);
+    out.writeBoolean(shouldDie);
+    if(events != null) {
+      out.writeBoolean(true);
+      out.writeInt(events.size());
+      for (TezEvent e : events) {
+        e.write(out);
+      }
+    } else {
+      out.writeBoolean(false);
+    }
+  }
+
+  @Override
+  public void readFields(DataInput in) throws IOException {
+    lastRequestId = in.readLong();
+    shouldDie = in.readBoolean();
+    if(in.readBoolean()) {
+      int eventCount = in.readInt();
+      events = new ArrayList<TezEvent>(eventCount);
+      for (int i = 0; i < eventCount; ++i) {
+        TezEvent e = new TezEvent();
+        e.readFields(in);
+        events.add(e);
+      }
+    }
+  }
+
+  @Override
+  public String toString() {
+    return "{ "
+        + " lastRequestId=" + lastRequestId
+        + ", shouldDie=" + shouldDie
+        + ", eventCount=" + (events != null ? events.size() : 0)
+        + " }";
+  }
+}


[37/50] [abbrv] Rename tez-engine-api to tez-runtime-api and tez-engine is split into 2: - tez-engine-library for user-visible Input/Output/Processor implementations - tez-engine-internals for framework internals

Posted by ss...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/engine/shuffle/common/Fetcher.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/shuffle/common/Fetcher.java b/tez-engine/src/main/java/org/apache/tez/engine/shuffle/common/Fetcher.java
deleted file mode 100644
index a353416..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/shuffle/common/Fetcher.java
+++ /dev/null
@@ -1,608 +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.engine.shuffle.common;
-
-import java.io.DataInputStream;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.net.HttpURLConnection;
-import java.net.MalformedURLException;
-import java.net.URL;
-import java.net.URLConnection;
-import java.security.GeneralSecurityException;
-import java.util.Arrays;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.Callable;
-import java.util.concurrent.atomic.AtomicInteger;
-
-import javax.crypto.SecretKey;
-import javax.net.ssl.HttpsURLConnection;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.io.IOUtils;
-import org.apache.hadoop.io.compress.CompressionCodec;
-import org.apache.hadoop.io.compress.Decompressor;
-import org.apache.hadoop.security.ssl.SSLFactory;
-import org.apache.hadoop.yarn.api.records.ApplicationId;
-import org.apache.tez.common.TezJobConfig;
-import org.apache.tez.engine.common.InputAttemptIdentifier;
-import org.apache.tez.engine.common.security.SecureShuffleUtils;
-import org.apache.tez.engine.common.shuffle.impl.ShuffleHeader;
-import org.apache.tez.engine.common.sort.impl.IFileInputStream;
-import org.apache.tez.engine.shuffle.common.FetchedInput.Type;
-
-import com.google.common.base.Preconditions;
-
-/**
- * Responsible for fetching inputs served by the ShuffleHandler for a single
- * host. Construct using {@link FetcherBuilder}
- */
-public class Fetcher implements Callable<FetchResult> {
-
-  private static final Log LOG = LogFactory.getLog(Fetcher.class);
-
-  private static final int UNIT_CONNECT_TIMEOUT = 60 * 1000;
-  private static final AtomicInteger fetcherIdGen = new AtomicInteger(0);
-
-  // Configurable fields.
-  private CompressionCodec codec;
-  private Decompressor decompressor;
-  private int connectionTimeout;
-  private int readTimeout;
-
-  private final SecretKey shuffleSecret;
-  private final Configuration conf;
-
-  private final FetcherCallback fetcherCallback;
-  private final FetchedInputAllocator inputManager;
-  private final ApplicationId appId;
-
-  private static boolean sslShuffle;
-  private static SSLFactory sslFactory;
-  private static boolean sslFactoryInited;
-
-  private final int fetcherIdentifier;
-  
-  // Parameters to track work.
-  private List<InputAttemptIdentifier> srcAttempts;
-  private String host;
-  private int port;
-  private int partition;
-
-  // Maps from the pathComponents (unique per srcTaskId) to the specific taskId
-  private Map<String, InputAttemptIdentifier> pathToAttemptMap;
-  private Set<InputAttemptIdentifier> remaining;
-
-  private URL url;
-  private String encHash;
-  private String msgToEncode;
-
-  private Fetcher(FetcherCallback fetcherCallback,
-      FetchedInputAllocator inputManager, ApplicationId appId, SecretKey shuffleSecret,
-      Configuration conf) {
-    this.fetcherCallback = fetcherCallback;
-    this.inputManager = inputManager;
-    this.shuffleSecret = shuffleSecret;
-    this.appId = appId;
-    this.conf = conf;
-
-    this.fetcherIdentifier = fetcherIdGen.getAndIncrement();
-    
-    // TODO NEWTEZ Ideally, move this out from here into a static initializer block.
-    synchronized (Fetcher.class) {
-      if (!sslFactoryInited) {
-        sslFactoryInited = true;
-        sslShuffle = conf.getBoolean(
-            TezJobConfig.TEZ_ENGINE_SHUFFLE_ENABLE_SSL,
-            TezJobConfig.DEFAULT_TEZ_ENGINE_SHUFFLE_ENABLE_SSL);
-        if (sslShuffle) {
-          sslFactory = new SSLFactory(SSLFactory.Mode.CLIENT, conf);
-          try {
-            sslFactory.init();
-          } catch (Exception ex) {
-            sslFactory.destroy();
-            throw new RuntimeException(ex);
-          }
-        }
-      }
-    }
-  }
-
-  @Override
-  public FetchResult call() throws Exception {
-    if (srcAttempts.size() == 0) {
-      return new FetchResult(host, port, partition, srcAttempts);
-    }
-
-    for (InputAttemptIdentifier in : srcAttempts) {
-      pathToAttemptMap.put(in.getPathComponent(), in);
-    }
-
-    remaining = new HashSet<InputAttemptIdentifier>(srcAttempts);
-
-    HttpURLConnection connection;
-    try {
-      connection = connectToShuffleHandler(host, port, partition, srcAttempts);
-    } catch (IOException e) {
-      // ioErrs.increment(1);
-      // If connect did not succeed, just mark all the maps as failed,
-      // indirectly penalizing the host
-      for (Iterator<InputAttemptIdentifier> leftIter = remaining.iterator(); leftIter
-          .hasNext();) {
-        fetcherCallback.fetchFailed(host, leftIter.next(), true);
-        leftIter.remove();
-      }
-      return new FetchResult(host, port, partition, remaining);
-    }
-
-    DataInputStream input;
-
-    try {
-      input = new DataInputStream(connection.getInputStream());
-      validateConnectionResponse(connection, url, msgToEncode, encHash);
-    } catch (IOException e) {
-      // ioErrs.increment(1);
-      // If we got a read error at this stage, it implies there was a problem
-      // with the first map, typically lost map. So, penalize only that map
-      // and add the rest
-      InputAttemptIdentifier firstAttempt = srcAttempts.get(0);
-      fetcherCallback.fetchFailed(host, firstAttempt, false);
-      remaining.remove(firstAttempt);
-      return new FetchResult(host, port, partition, remaining);
-    }
-
-    // By this point, the connection is setup and the response has been
-    // validated.
-
-    // Loop through available map-outputs and fetch them
-    // On any error, faildTasks is not null and we exit
-    // after putting back the remaining maps to the
-    // yet_to_be_fetched list and marking the failed tasks.
-    InputAttemptIdentifier[] failedInputs = null;
-    while (!remaining.isEmpty() && failedInputs == null) {
-      failedInputs = fetchInputs(input);
-    }
-
-    if (failedInputs != null && failedInputs.length > 0) {
-      LOG.warn("copyInputs failed for tasks " + Arrays.toString(failedInputs));
-      for (InputAttemptIdentifier left : failedInputs) {
-        fetcherCallback.fetchFailed(host, left, false);
-        remaining.remove(left);
-      }
-    }
-
-    IOUtils.cleanup(LOG, input);
-
-    // Sanity check
-    if (failedInputs == null && !remaining.isEmpty()) {
-      throw new IOException("server didn't return all expected map outputs: "
-          + remaining.size() + " left.");
-    }
-
-    return new FetchResult(host, port, partition, remaining);
-
-  }
-
-  private InputAttemptIdentifier[] fetchInputs(DataInputStream input) {
-    FetchedInput fetchedInput = null;
-    InputAttemptIdentifier srcAttemptId = null;
-    long decompressedLength = -1;
-    long compressedLength = -1;
-
-    try {
-      long startTime = System.currentTimeMillis();
-      int responsePartition = -1;
-      // Read the shuffle header
-      try {
-        ShuffleHeader header = new ShuffleHeader();
-        header.readFields(input);
-        String pathComponent = header.getMapId();
-
-        srcAttemptId = pathToAttemptMap.get(pathComponent);
-        compressedLength = header.getCompressedLength();
-        decompressedLength = header.getUncompressedLength();
-        responsePartition = header.getPartition();
-      } catch (IllegalArgumentException e) {
-        // badIdErrs.increment(1);
-        LOG.warn("Invalid src id ", e);
-        // Don't know which one was bad, so consider all of them as bad
-        return remaining.toArray(new InputAttemptIdentifier[remaining.size()]);
-      }
-
-      // Do some basic sanity verification
-      if (!verifySanity(compressedLength, decompressedLength,
-          responsePartition, srcAttemptId)) {
-        return new InputAttemptIdentifier[] { srcAttemptId };
-      }
-
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("header: " + srcAttemptId + ", len: " + compressedLength
-            + ", decomp len: " + decompressedLength);
-      }
-
-      // Get the location for the map output - either in-memory or on-disk
-      fetchedInput = inputManager.allocate(decompressedLength, srcAttemptId);
-
-      // TODO NEWTEZ No concept of WAIT at the moment.
-      // // Check if we can shuffle *now* ...
-      // if (fetchedInput.getType() == FetchedInput.WAIT) {
-      // LOG.info("fetcher#" + id +
-      // " - MergerManager returned Status.WAIT ...");
-      // //Not an error but wait to process data.
-      // return EMPTY_ATTEMPT_ID_ARRAY;
-      // }
-
-      // Go!
-      LOG.info("fetcher" + " about to shuffle output of srcAttempt "
-          + fetchedInput.getInputAttemptIdentifier() + " decomp: "
-          + decompressedLength + " len: " + compressedLength + " to "
-          + fetchedInput.getType());
-
-      if (fetchedInput.getType() == Type.MEMORY) {
-        shuffleToMemory((MemoryFetchedInput) fetchedInput, input,
-            (int) decompressedLength, (int) compressedLength);
-      } else {
-        shuffleToDisk((DiskFetchedInput) fetchedInput, input, compressedLength);
-      }
-
-      // Inform the shuffle scheduler
-      long endTime = System.currentTimeMillis();
-      fetcherCallback.fetchSucceeded(host, srcAttemptId, fetchedInput,
-          compressedLength, (endTime - startTime));
-
-      // Note successful shuffle
-      remaining.remove(srcAttemptId);
-      // metrics.successFetch();
-      return null;
-    } catch (IOException ioe) {
-      // ioErrs.increment(1);
-      if (srcAttemptId == null || fetchedInput == null) {
-        LOG.info("fetcher" + " failed to read map header" + srcAttemptId
-            + " decomp: " + decompressedLength + ", " + compressedLength, ioe);
-        if (srcAttemptId == null) {
-          return remaining
-              .toArray(new InputAttemptIdentifier[remaining.size()]);
-        } else {
-          return new InputAttemptIdentifier[] { srcAttemptId };
-        }
-      }
-      LOG.warn("Failed to shuffle output of " + srcAttemptId + " from " + host,
-          ioe);
-
-      // Inform the shuffle-scheduler
-      try {
-        fetchedInput.abort();
-      } catch (IOException e) {
-        LOG.info("Failure to cleanup fetchedInput: " + fetchedInput);
-      }
-      // metrics.failedFetch();
-      return new InputAttemptIdentifier[] { srcAttemptId };
-    }
-  }
-
-  @SuppressWarnings("resource")
-  private void shuffleToMemory(MemoryFetchedInput fetchedInput,
-      InputStream input, int decompressedLength, int compressedLength)
-      throws IOException {
-    IFileInputStream checksumIn = new IFileInputStream(input, compressedLength,
-        conf);
-
-    input = checksumIn;
-
-    // Are map-outputs compressed?
-    if (codec != null) {
-      decompressor.reset();
-      input = codec.createInputStream(input, decompressor);
-    }
-    // Copy map-output into an in-memory buffer
-    byte[] shuffleData = fetchedInput.getBytes();
-
-    try {
-      IOUtils.readFully(input, shuffleData, 0, shuffleData.length);
-      // metrics.inputBytes(shuffleData.length);
-      LOG.info("Read " + shuffleData.length + " bytes from input for "
-          + fetchedInput.getInputAttemptIdentifier());
-    } catch (IOException ioe) {
-      // Close the streams
-      IOUtils.cleanup(LOG, input);
-      // Re-throw
-      throw ioe;
-    }
-  }
-
-  private void shuffleToDisk(DiskFetchedInput fetchedInput, InputStream input,
-      long compressedLength) throws IOException {
-    // Copy data to local-disk
-    OutputStream output = fetchedInput.getOutputStream();
-    long bytesLeft = compressedLength;
-    try {
-      final int BYTES_TO_READ = 64 * 1024;
-      byte[] buf = new byte[BYTES_TO_READ];
-      while (bytesLeft > 0) {
-        int n = input.read(buf, 0, (int) Math.min(bytesLeft, BYTES_TO_READ));
-        if (n < 0) {
-          throw new IOException("read past end of stream reading "
-              + fetchedInput.getInputAttemptIdentifier());
-        }
-        output.write(buf, 0, n);
-        bytesLeft -= n;
-        // metrics.inputBytes(n);
-      }
-
-      LOG.info("Read " + (compressedLength - bytesLeft)
-          + " bytes from input for " + fetchedInput.getInputAttemptIdentifier());
-
-      output.close();
-    } catch (IOException ioe) {
-      // Close the streams
-      IOUtils.cleanup(LOG, input, output);
-
-      // Re-throw
-      throw ioe;
-    }
-
-    // Sanity check
-    if (bytesLeft != 0) {
-      throw new IOException("Incomplete input received for "
-          + fetchedInput.getInputAttemptIdentifier() + " from " + host + " ("
-          + bytesLeft + " bytes missing of " + compressedLength + ")");
-    }
-  }
-
-  /**
-   * Do some basic verification on the input received -- Being defensive
-   * 
-   * @param compressedLength
-   * @param decompressedLength
-   * @param fetchPartition
-   * @param remaining
-   * @param mapId
-   * @return true/false, based on if the verification succeeded or not
-   */
-  private boolean verifySanity(long compressedLength, long decompressedLength,
-      int fetchPartition, InputAttemptIdentifier srcAttemptId) {
-    if (compressedLength < 0 || decompressedLength < 0) {
-      // wrongLengthErrs.increment(1);
-      LOG.warn(" invalid lengths in input header: id: " + srcAttemptId
-          + " len: " + compressedLength + ", decomp len: " + decompressedLength);
-      return false;
-    }
-
-    if (fetchPartition != this.partition) {
-      // wrongReduceErrs.increment(1);
-      LOG.warn(" data for the wrong reduce map: " + srcAttemptId + " len: "
-          + compressedLength + " decomp len: " + decompressedLength
-          + " for reduce " + fetchPartition);
-      return false;
-    }
-
-    // Sanity check
-    if (!remaining.contains(srcAttemptId)) {
-      // wrongMapErrs.increment(1);
-      LOG.warn("Invalid input. Received output for " + srcAttemptId);
-      return false;
-    }
-    return true;
-  }
-
-  private HttpURLConnection connectToShuffleHandler(String host, int port,
-      int partition, List<InputAttemptIdentifier> inputs) throws IOException {
-    try {
-      this.url = constructInputURL(host, port, partition, inputs);
-      HttpURLConnection connection = openConnection(url);
-
-      // generate hash of the url
-      this.msgToEncode = SecureShuffleUtils.buildMsgFrom(url);
-      this.encHash = SecureShuffleUtils.hashFromString(msgToEncode,
-          shuffleSecret);
-
-      // put url hash into http header
-      connection.addRequestProperty(SecureShuffleUtils.HTTP_HEADER_URL_HASH,
-          encHash);
-      // set the read timeout
-      connection.setReadTimeout(readTimeout);
-      // put shuffle version into http header
-      connection.addRequestProperty(ShuffleHeader.HTTP_HEADER_NAME,
-          ShuffleHeader.DEFAULT_HTTP_HEADER_NAME);
-      connection.addRequestProperty(ShuffleHeader.HTTP_HEADER_VERSION,
-          ShuffleHeader.DEFAULT_HTTP_HEADER_VERSION);
-
-      connect(connection, connectionTimeout);
-      return connection;
-    } catch (IOException e) {
-      LOG.warn("Failed to connect to " + host + " with " + srcAttempts.size()
-          + " inputs", e);
-      throw e;
-    }
-  }
-
-  private void validateConnectionResponse(HttpURLConnection connection,
-      URL url, String msgToEncode, String encHash) throws IOException {
-    int rc = connection.getResponseCode();
-    if (rc != HttpURLConnection.HTTP_OK) {
-      throw new IOException("Got invalid response code " + rc + " from " + url
-          + ": " + connection.getResponseMessage());
-    }
-
-    if (!ShuffleHeader.DEFAULT_HTTP_HEADER_NAME.equals(connection
-        .getHeaderField(ShuffleHeader.HTTP_HEADER_NAME))
-        || !ShuffleHeader.DEFAULT_HTTP_HEADER_VERSION.equals(connection
-            .getHeaderField(ShuffleHeader.HTTP_HEADER_VERSION))) {
-      throw new IOException("Incompatible shuffle response version");
-    }
-
-    // get the replyHash which is HMac of the encHash we sent to the server
-    String replyHash = connection
-        .getHeaderField(SecureShuffleUtils.HTTP_HEADER_REPLY_URL_HASH);
-    if (replyHash == null) {
-      throw new IOException("security validation of TT Map output failed");
-    }
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("url=" + msgToEncode + ";encHash=" + encHash + ";replyHash="
-          + replyHash);
-    }
-    // verify that replyHash is HMac of encHash
-    SecureShuffleUtils.verifyReply(replyHash, encHash, shuffleSecret);
-    LOG.info("for url=" + msgToEncode + " sent hash and receievd reply");
-  }
-
-  protected HttpURLConnection openConnection(URL url) throws IOException {
-    HttpURLConnection conn = (HttpURLConnection) url.openConnection();
-    if (sslShuffle) {
-      HttpsURLConnection httpsConn = (HttpsURLConnection) conn;
-      try {
-        httpsConn.setSSLSocketFactory(sslFactory.createSSLSocketFactory());
-      } catch (GeneralSecurityException ex) {
-        throw new IOException(ex);
-      }
-      httpsConn.setHostnameVerifier(sslFactory.getHostnameVerifier());
-    }
-    return conn;
-  }
-
-  /**
-   * The connection establishment is attempted multiple times and is given up
-   * only on the last failure. Instead of connecting with a timeout of X, we try
-   * connecting with a timeout of x < X but multiple times.
-   */
-  private void connect(URLConnection connection, int connectionTimeout)
-      throws IOException {
-    int unit = 0;
-    if (connectionTimeout < 0) {
-      throw new IOException("Invalid timeout " + "[timeout = "
-          + connectionTimeout + " ms]");
-    } else if (connectionTimeout > 0) {
-      unit = Math.min(UNIT_CONNECT_TIMEOUT, connectionTimeout);
-    }
-    // set the connect timeout to the unit-connect-timeout
-    connection.setConnectTimeout(unit);
-    while (true) {
-      try {
-        connection.connect();
-        break;
-      } catch (IOException ioe) {
-        // update the total remaining connect-timeout
-        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) {
-          throw ioe;
-        }
-
-        // reset the connect timeout for the last try
-        if (connectionTimeout < unit) {
-          unit = connectionTimeout;
-          // reset the connect time out for the final connect
-          connection.setConnectTimeout(unit);
-        }
-      }
-    }
-  }
-
-  private URL constructInputURL(String host, int port, int partition,
-      List<InputAttemptIdentifier> inputs) throws MalformedURLException {
-    StringBuilder url = ShuffleUtils.constructBaseURIForShuffleHandler(host,
-        port, partition, appId);
-    boolean first = true;
-    for (InputAttemptIdentifier input : inputs) {
-      if (first) {
-        first = false;
-        url.append(input.getPathComponent());
-      } else {
-        url.append(",").append(input.getPathComponent());
-      }
-    }
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("InputFetch URL for: " + host + " : " + url.toString());
-    }
-    return new URL(url.toString());
-  }
-
-  /**
-   * Builder for the construction of Fetchers
-   */
-  public static class FetcherBuilder {
-    private Fetcher fetcher;
-    private boolean workAssigned = false;
-
-    public FetcherBuilder(FetcherCallback fetcherCallback,
-        FetchedInputAllocator inputManager, ApplicationId appId,
-        SecretKey shuffleSecret, Configuration conf) {
-      this.fetcher = new Fetcher(fetcherCallback, inputManager, appId,
-          shuffleSecret, conf);
-    }
-
-    public FetcherBuilder setCompressionParameters(CompressionCodec codec,
-        Decompressor decompressor) {
-      fetcher.codec = codec;
-      fetcher.decompressor = decompressor;
-      return this;
-    }
-
-    public FetcherBuilder setConnectionParameters(int connectionTimeout,
-        int readTimeout) {
-      fetcher.connectionTimeout = connectionTimeout;
-      fetcher.readTimeout = readTimeout;
-      return this;
-    }
-
-    public FetcherBuilder assignWork(String host, int port, int partition,
-        List<InputAttemptIdentifier> inputs) {
-      fetcher.host = host;
-      fetcher.port = port;
-      fetcher.partition = partition;
-      fetcher.srcAttempts = inputs;
-      workAssigned = true;
-      return this;
-    }
-
-    public Fetcher build() {
-      Preconditions.checkState(workAssigned == true,
-          "Cannot build a fetcher withot assigning work to it");
-      return fetcher;
-    }
-  }
-
-  @Override
-  public int hashCode() {
-    return fetcherIdentifier;
-  }
-
-  @Override
-  public boolean equals(Object obj) {
-    if (this == obj)
-      return true;
-    if (obj == null)
-      return false;
-    if (getClass() != obj.getClass())
-      return false;
-    Fetcher other = (Fetcher) obj;
-    if (fetcherIdentifier != other.fetcherIdentifier)
-      return false;
-    return true;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/engine/shuffle/common/FetcherCallback.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/shuffle/common/FetcherCallback.java b/tez-engine/src/main/java/org/apache/tez/engine/shuffle/common/FetcherCallback.java
deleted file mode 100644
index f0b7cd2..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/shuffle/common/FetcherCallback.java
+++ /dev/null
@@ -1,31 +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.engine.shuffle.common;
-
-import java.io.IOException;
-
-import org.apache.tez.engine.common.InputAttemptIdentifier;
-
-public interface FetcherCallback {
-
-  public void fetchSucceeded(String host, InputAttemptIdentifier srcAttemptIdentifier, FetchedInput fetchedInput, long fetchedBytes, long copyDuration) throws IOException;
-  
-  public void fetchFailed(String host, InputAttemptIdentifier srcAttemptIdentifier, boolean connectFailed);
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/engine/shuffle/common/InputHost.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/shuffle/common/InputHost.java b/tez-engine/src/main/java/org/apache/tez/engine/shuffle/common/InputHost.java
deleted file mode 100644
index 4862b76..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/shuffle/common/InputHost.java
+++ /dev/null
@@ -1,90 +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.engine.shuffle.common;
-
-import java.util.ArrayList;
-import java.util.LinkedList;
-import java.util.List;
-
-import org.apache.hadoop.yarn.api.records.ApplicationId;
-import org.apache.tez.engine.common.InputAttemptIdentifier;
-
-public class InputHost {
-
-  private final String host;
-  private final int port;
-
-  private final List<InputAttemptIdentifier> inputs = new LinkedList<InputAttemptIdentifier>();
-
-  public InputHost(String hostName, int port, ApplicationId appId) {
-    this.host = hostName;
-    this.port = port;
-  }
-
-  public String getHost() {
-    return this.host;
-  }
-
-  public int getPort() {
-    return this.port;
-  }
-
-  public synchronized int getNumPendingInputs() {
-    return inputs.size();
-  }
-  
-  public synchronized void addKnownInput(InputAttemptIdentifier srcAttempt) {
-    inputs.add(srcAttempt);
-  }
-
-  public synchronized List<InputAttemptIdentifier> clearAndGetPendingInputs() {
-    List<InputAttemptIdentifier> inputsCopy = new ArrayList<InputAttemptIdentifier>(
-        inputs);
-    inputs.clear();
-    return inputsCopy;
-  }
-
-  @Override
-  public int hashCode() {
-    final int prime = 31;
-    int result = 1;
-    result = prime * result + ((host == null) ? 0 : host.hashCode());
-    result = prime * result + port;
-    return result;
-  }
-
-  @Override
-  public boolean equals(Object obj) {
-    if (this == obj)
-      return true;
-    if (obj == null)
-      return false;
-    if (getClass() != obj.getClass())
-      return false;
-    InputHost other = (InputHost) obj;
-    if (host == null) {
-      if (other.host != null)
-        return false;
-    } else if (!host.equals(other.host))
-      return false;
-    if (port != other.port)
-      return false;
-    return true;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/engine/shuffle/common/MemoryFetchedInput.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/shuffle/common/MemoryFetchedInput.java b/tez-engine/src/main/java/org/apache/tez/engine/shuffle/common/MemoryFetchedInput.java
deleted file mode 100644
index 59d288e..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/shuffle/common/MemoryFetchedInput.java
+++ /dev/null
@@ -1,89 +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.engine.shuffle.common;
-
-import java.io.ByteArrayInputStream;
-import java.io.InputStream;
-import java.io.OutputStream;
-
-import org.apache.hadoop.io.BoundedByteArrayOutputStream;
-import org.apache.tez.engine.common.InputAttemptIdentifier;
-
-import com.google.common.base.Preconditions;
-
-public class MemoryFetchedInput extends FetchedInput {
-
-  private BoundedByteArrayOutputStream byteStream;
-
-  public MemoryFetchedInput(long size,
-      InputAttemptIdentifier inputAttemptIdentifier,
-      FetchedInputCallback callbackHandler) {
-    super(Type.MEMORY, size, inputAttemptIdentifier, callbackHandler);
-    this.byteStream = new BoundedByteArrayOutputStream((int) size);
-  }
-
-  @Override
-  public OutputStream getOutputStream() {
-    return byteStream;
-  }
-
-  @Override
-  public InputStream getInputStream() {
-    return new ByteArrayInputStream(byteStream.getBuffer());
-  }
-
-  public byte[] getBytes() {
-    return byteStream.getBuffer();
-  }
-  
-  @Override
-  public void commit() {
-    if (state == State.PENDING) {
-      state = State.COMMITTED;
-      notifyFetchComplete();
-    }
-  }
-
-  @Override
-  public void abort() {
-    if (state == State.PENDING) {
-      state = State.ABORTED;
-      notifyFetchFailure();
-    }
-  }
-  
-  @Override
-  public void free() {
-    Preconditions.checkState(
-        state == State.COMMITTED || state == State.ABORTED,
-        "FetchedInput can only be freed after it is committed or aborted");
-    if (state == State.COMMITTED) {
-      state = State.FREED;
-      this.byteStream = null;
-      notifyFreedResource();
-    }
-  }
-
-  @Override
-  public String toString() {
-    return "MemoryFetchedInput [inputAttemptIdentifier="
-        + inputAttemptIdentifier + ", size=" + size + ", type=" + type
-        + ", id=" + id + ", state=" + state + "]";
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/engine/shuffle/common/ShuffleUtils.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/shuffle/common/ShuffleUtils.java b/tez-engine/src/main/java/org/apache/tez/engine/shuffle/common/ShuffleUtils.java
deleted file mode 100644
index 84d270d..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/shuffle/common/ShuffleUtils.java
+++ /dev/null
@@ -1,81 +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.engine.shuffle.common;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-
-import javax.crypto.SecretKey;
-
-import org.apache.hadoop.io.DataInputByteBuffer;
-import org.apache.hadoop.io.DataOutputBuffer;
-import org.apache.hadoop.security.token.Token;
-import org.apache.hadoop.yarn.api.records.ApplicationId;
-import org.apache.tez.engine.common.security.JobTokenIdentifier;
-import org.apache.tez.engine.common.security.JobTokenSecretManager;
-
-public class ShuffleUtils {
-
-  public static String SHUFFLE_HANDLER_SERVICE_ID = "mapreduce.shuffle";
-
-  public static SecretKey getJobTokenSecretFromTokenBytes(ByteBuffer meta)
-      throws IOException {
-    DataInputByteBuffer in = new DataInputByteBuffer();
-    in.reset(meta);
-    Token<JobTokenIdentifier> jt = new Token<JobTokenIdentifier>();
-    jt.readFields(in);
-    SecretKey sk = JobTokenSecretManager.createSecretKey(jt.getPassword());
-    return sk;
-  }
-
-  public static ByteBuffer convertJobTokenToBytes(
-      Token<JobTokenIdentifier> jobToken) throws IOException {
-    DataOutputBuffer dob = new DataOutputBuffer();
-    jobToken.write(dob);
-    ByteBuffer bb = ByteBuffer.wrap(dob.getData(), 0, dob.getLength());
-    return bb;
-  }
-
-  public static int deserializeShuffleProviderMetaData(ByteBuffer meta)
-      throws IOException {
-    DataInputByteBuffer in = new DataInputByteBuffer();
-    try {
-      in.reset(meta);
-      int port = in.readInt();
-      return port;
-    } finally {
-      in.close();
-    }
-  }
-  
-  // TODO NEWTEZ handle ssl shuffle
-  public static StringBuilder constructBaseURIForShuffleHandler(String host, int port, int partition, ApplicationId appId) {
-    StringBuilder sb = new StringBuilder("http://");
-    sb.append(host);
-    sb.append(":");
-    sb.append(String.valueOf(port));
-    sb.append("/");
-    sb.append("mapOutput?job=");
-    sb.append(appId.toString().replace("application", "job"));
-    sb.append("&reduce=");
-    sb.append(String.valueOf(partition));
-    sb.append("&map=");
-    return sb;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/proto/Events.proto
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/proto/Events.proto b/tez-engine/src/main/proto/Events.proto
deleted file mode 100644
index fa9cb2c..0000000
--- a/tez-engine/src/main/proto/Events.proto
+++ /dev/null
@@ -1,28 +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.
- */
-
-option java_package = "org.apache.tez.engine.api.events";
-option java_outer_classname = "SystemEventProtos";
-option java_generate_equals_and_hash = true;
-
-message TaskAttemptFailedEventProto {
-  optional string diagnostics = 1;
-}
-
-message TaskAttemptCompletedEventProto {
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/proto/ShufflePayloads.proto
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/proto/ShufflePayloads.proto b/tez-engine/src/main/proto/ShufflePayloads.proto
deleted file mode 100644
index f831de2..0000000
--- a/tez-engine/src/main/proto/ShufflePayloads.proto
+++ /dev/null
@@ -1,33 +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.
- */
-
-option java_package = "org.apache.tez.engine.common.shuffle.newimpl";
-option java_outer_classname = "ShuffleUserPayloads";
-option java_generate_equals_and_hash = true;
-
-message DataMovementEventPayloadProto {
-  optional bool output_generated = 1;
-  optional string host = 2;
-  optional int32 port = 3;
-  optional string path_component = 4;
-  optional int32 run_duration = 5;
-} 
-
-message InputInformationEventPayloadProto {
-  optional int32 partition_range = 1;
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/test/java/org/apache/tez/engine/common/objectregistry/TestObjectRegistry.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/test/java/org/apache/tez/engine/common/objectregistry/TestObjectRegistry.java b/tez-engine/src/test/java/org/apache/tez/engine/common/objectregistry/TestObjectRegistry.java
deleted file mode 100644
index 7276782..0000000
--- a/tez-engine/src/test/java/org/apache/tez/engine/common/objectregistry/TestObjectRegistry.java
+++ /dev/null
@@ -1,56 +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.engine.common.objectregistry;
-
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
-
-import com.google.inject.Guice;
-import com.google.inject.Injector;
-
-public class TestObjectRegistry {
-
-  @SuppressWarnings("unused")
-  @Before
-  public void setup() {
-    Injector injector = Guice.createInjector(new ObjectRegistryModule());
-  }
-
-  @Test
-  public void testBasicCRUD() {
-    ObjectRegistry objectRegistry =
-        ObjectRegistryFactory.getObjectRegistry();
-    Assert.assertNotNull(objectRegistry);
-
-    Assert.assertNull(objectRegistry.get("foo"));
-    Assert.assertFalse(objectRegistry.delete("foo"));
-    Integer one = new Integer(1);
-    Integer two_1 = new Integer(2);
-    Integer two_2 = new Integer(3);
-    Assert.assertNull(objectRegistry.add(ObjectLifeCycle.DAG, "one", one));
-    Assert.assertEquals(one, objectRegistry.get("one"));
-    Assert.assertNull(objectRegistry.add(ObjectLifeCycle.DAG, "two", two_1));
-    Assert.assertNotNull(objectRegistry.add(ObjectLifeCycle.SESSION, "two", two_2));
-    Assert.assertNotEquals(two_1, objectRegistry.get("two"));
-    Assert.assertEquals(two_2, objectRegistry.get("two"));
-    Assert.assertTrue(objectRegistry.delete("one"));
-    Assert.assertFalse(objectRegistry.delete("one"));
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-mapreduce-examples/src/main/java/org/apache/tez/mapreduce/examples/MRRSleepJob.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce-examples/src/main/java/org/apache/tez/mapreduce/examples/MRRSleepJob.java b/tez-mapreduce-examples/src/main/java/org/apache/tez/mapreduce/examples/MRRSleepJob.java
index 05675b5..e98b45f 100644
--- a/tez-mapreduce-examples/src/main/java/org/apache/tez/mapreduce/examples/MRRSleepJob.java
+++ b/tez-mapreduce-examples/src/main/java/org/apache/tez/mapreduce/examples/MRRSleepJob.java
@@ -77,17 +77,17 @@ import org.apache.tez.dag.api.EdgeProperty.DataSourceType;
 import org.apache.tez.dag.api.EdgeProperty.SchedulingType;
 import org.apache.tez.dag.api.client.DAGClient;
 import org.apache.tez.dag.api.client.DAGStatus;
-import org.apache.tez.engine.common.objectregistry.ObjectLifeCycle;
-import org.apache.tez.engine.common.objectregistry.ObjectRegistry;
-import org.apache.tez.engine.common.objectregistry.ObjectRegistryFactory;
-import org.apache.tez.engine.lib.input.ShuffledMergedInputLegacy;
-import org.apache.tez.engine.lib.output.OnFileSortedOutput;
 import org.apache.tez.mapreduce.hadoop.InputSplitInfo;
 import org.apache.tez.mapreduce.hadoop.MRHelpers;
 import org.apache.tez.mapreduce.hadoop.MultiStageMRConfToTezTranslator;
 import org.apache.tez.mapreduce.hadoop.MultiStageMRConfigUtil;
 import org.apache.tez.mapreduce.processor.map.MapProcessor;
 import org.apache.tez.mapreduce.processor.reduce.ReduceProcessor;
+import org.apache.tez.runtime.common.objectregistry.ObjectLifeCycle;
+import org.apache.tez.runtime.common.objectregistry.ObjectRegistry;
+import org.apache.tez.runtime.common.objectregistry.ObjectRegistryFactory;
+import org.apache.tez.runtime.library.input.ShuffledMergedInputLegacy;
+import org.apache.tez.runtime.library.output.OnFileSortedOutput;
 
 import com.google.common.annotations.VisibleForTesting;
 

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-mapreduce-examples/src/main/java/org/apache/tez/mapreduce/examples/OrderedWordCount.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce-examples/src/main/java/org/apache/tez/mapreduce/examples/OrderedWordCount.java b/tez-mapreduce-examples/src/main/java/org/apache/tez/mapreduce/examples/OrderedWordCount.java
index ec419c1..cad79f5 100644
--- a/tez-mapreduce-examples/src/main/java/org/apache/tez/mapreduce/examples/OrderedWordCount.java
+++ b/tez-mapreduce-examples/src/main/java/org/apache/tez/mapreduce/examples/OrderedWordCount.java
@@ -70,14 +70,14 @@ import org.apache.tez.dag.api.EdgeProperty.DataSourceType;
 import org.apache.tez.dag.api.EdgeProperty.SchedulingType;
 import org.apache.tez.dag.api.client.DAGClient;
 import org.apache.tez.dag.api.client.DAGStatus;
-import org.apache.tez.engine.lib.input.ShuffledMergedInputLegacy;
-import org.apache.tez.engine.lib.output.OnFileSortedOutput;
 import org.apache.tez.mapreduce.hadoop.InputSplitInfo;
 import org.apache.tez.mapreduce.hadoop.MRHelpers;
 import org.apache.tez.mapreduce.hadoop.MRJobConfig;
 import org.apache.tez.mapreduce.hadoop.MultiStageMRConfToTezTranslator;
 import org.apache.tez.mapreduce.processor.map.MapProcessor;
 import org.apache.tez.mapreduce.processor.reduce.ReduceProcessor;
+import org.apache.tez.runtime.library.input.ShuffledMergedInputLegacy;
+import org.apache.tez.runtime.library.output.OnFileSortedOutput;
 
 /**
  * An MRR job built on top of word count to return words sorted by

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-mapreduce-tests/src/test/java/org/apache/tez/mapreduce/TestMRRJobsDAGApi.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce-tests/src/test/java/org/apache/tez/mapreduce/TestMRRJobsDAGApi.java b/tez-mapreduce-tests/src/test/java/org/apache/tez/mapreduce/TestMRRJobsDAGApi.java
index 7e662cb..7280a1f 100644
--- a/tez-mapreduce-tests/src/test/java/org/apache/tez/mapreduce/TestMRRJobsDAGApi.java
+++ b/tez-mapreduce-tests/src/test/java/org/apache/tez/mapreduce/TestMRRJobsDAGApi.java
@@ -68,8 +68,6 @@ import org.apache.tez.dag.api.EdgeProperty.SchedulingType;
 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.engine.lib.input.ShuffledMergedInputLegacy;
-import org.apache.tez.engine.lib.output.OnFileSortedOutput;
 import org.apache.tez.mapreduce.examples.MRRSleepJob;
 import org.apache.tez.mapreduce.examples.MRRSleepJob.ISleepReducer;
 import org.apache.tez.mapreduce.examples.MRRSleepJob.MRRSleepJobPartitioner;
@@ -82,6 +80,8 @@ import org.apache.tez.mapreduce.hadoop.MRJobConfig;
 import org.apache.tez.mapreduce.hadoop.MultiStageMRConfToTezTranslator;
 import org.apache.tez.mapreduce.processor.map.MapProcessor;
 import org.apache.tez.mapreduce.processor.reduce.ReduceProcessor;
+import org.apache.tez.runtime.library.input.ShuffledMergedInputLegacy;
+import org.apache.tez.runtime.library.output.OnFileSortedOutput;
 import org.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.BeforeClass;

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-mapreduce/pom.xml
----------------------------------------------------------------------
diff --git a/tez-mapreduce/pom.xml b/tez-mapreduce/pom.xml
index 2c6b78e..aa3d915 100644
--- a/tez-mapreduce/pom.xml
+++ b/tez-mapreduce/pom.xml
@@ -35,7 +35,12 @@
     </dependency>
     <dependency>
       <groupId>org.apache.tez</groupId>
-      <artifactId>tez-engine</artifactId>
+      <artifactId>tez-runtime-library</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.tez</groupId>
+      <artifactId>tez-runtime-internals</artifactId>
+      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.hadoop</groupId>

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-mapreduce/src/main/java/org/apache/hadoop/mapred/LocalJobRunnerTez.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/hadoop/mapred/LocalJobRunnerTez.java b/tez-mapreduce/src/main/java/org/apache/hadoop/mapred/LocalJobRunnerTez.java
index f2b0a38..22d4a75 100644
--- a/tez-mapreduce/src/main/java/org/apache/hadoop/mapred/LocalJobRunnerTez.java
+++ b/tez-mapreduce/src/main/java/org/apache/hadoop/mapred/LocalJobRunnerTez.java
@@ -853,7 +853,7 @@
 //    LOG.info(TezJobConfig.LOCAL_DIRS + " for child : " + taskAttemptID +
 //        " is " + childMapredLocalDir);
 //    conf.set(TezJobConfig.LOCAL_DIRS, childMapredLocalDir.toString());
-//    conf.setClass(Constants.TEZ_ENGINE_TASK_OUTPUT_MANAGER,
+//    conf.setClass(Constants.TEZ_RUNTIME_TASK_OUTPUT_MANAGER,
 //        TezLocalTaskOutputFiles.class, TezTaskOutput.class);
 //  }
 //

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/combine/MRCombiner.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/combine/MRCombiner.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/combine/MRCombiner.java
index 199bbfe..3bc8da2 100644
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/combine/MRCombiner.java
+++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/combine/MRCombiner.java
@@ -42,18 +42,18 @@ import org.apache.hadoop.util.Progress;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.tez.common.TezUtils;
 import org.apache.tez.common.counters.TezCounter;
-import org.apache.tez.engine.api.TezInputContext;
-import org.apache.tez.engine.api.TezOutputContext;
-import org.apache.tez.engine.api.TezTaskContext;
-import org.apache.tez.engine.common.ConfigUtils;
-import org.apache.tez.engine.common.ValuesIterator;
-import org.apache.tez.engine.common.combine.Combiner;
-import org.apache.tez.engine.common.sort.impl.IFile.Writer;
-import org.apache.tez.engine.common.sort.impl.TezRawKeyValueIterator;
 import org.apache.tez.mapreduce.hadoop.MRConfig;
 import org.apache.tez.mapreduce.hadoop.MRJobConfig;
 import org.apache.tez.mapreduce.hadoop.mapred.MRCounters;
 import org.apache.tez.mapreduce.processor.MRTaskReporter;
+import org.apache.tez.runtime.api.TezInputContext;
+import org.apache.tez.runtime.api.TezOutputContext;
+import org.apache.tez.runtime.api.TezTaskContext;
+import org.apache.tez.runtime.library.common.ConfigUtils;
+import org.apache.tez.runtime.library.common.ValuesIterator;
+import org.apache.tez.runtime.library.common.combine.Combiner;
+import org.apache.tez.runtime.library.common.sort.impl.TezRawKeyValueIterator;
+import org.apache.tez.runtime.library.common.sort.impl.IFile.Writer;
 
 @SuppressWarnings({"rawtypes", "unchecked"})
 public class MRCombiner implements Combiner {

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/DeprecatedKeys.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/DeprecatedKeys.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/DeprecatedKeys.java
index 21a3983..dac92ed 100644
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/DeprecatedKeys.java
+++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/DeprecatedKeys.java
@@ -23,9 +23,10 @@ import java.util.HashMap;
 import java.util.Map;
 
 import org.apache.hadoop.conf.Configuration;
-import org.apache.tez.common.Constants;
 import org.apache.tez.common.TezJobConfig;
 import org.apache.tez.dag.api.TezConfiguration;
+import org.apache.tez.runtime.library.common.Constants;
+
 import com.google.common.collect.Maps;
 
 public class DeprecatedKeys {
@@ -50,15 +51,15 @@ public class DeprecatedKeys {
   
   
   /**
-   * Keys used by the engine.
+   * Keys used by the Tez Runtime.
    */
-  private static Map<String, String> mrParamToEngineParamMap =
+  private static Map<String, String> mrParamToTezRuntimeParamMap =
       new HashMap<String, String>();
 
   
  
   static {
-    populateMRToEngineParamMap();
+    populateMRToTezRuntimeParamMap();
     populateMRToDagParamMap();
     populateMultiStageParamMap();
     addDeprecatedKeys();
@@ -70,32 +71,32 @@ public class DeprecatedKeys {
     multiStageParamMap.put(
         MRJobConfig.KEY_COMPARATOR,
         getDeprecationMap(
-            TezJobConfig.TEZ_ENGINE_INTERMEDIATE_INPUT_KEY_COMPARATOR_CLASS,
-            TezJobConfig.TEZ_ENGINE_INTERMEDIATE_OUTPUT_KEY_COMPARATOR_CLASS));
+            TezJobConfig.TEZ_RUNTIME_INTERMEDIATE_INPUT_KEY_COMPARATOR_CLASS,
+            TezJobConfig.TEZ_RUNTIME_INTERMEDIATE_OUTPUT_KEY_COMPARATOR_CLASS));
     
     multiStageParamMap.put(
         MRJobConfig.MAP_OUTPUT_KEY_CLASS,
         getDeprecationMap(
-            TezJobConfig.TEZ_ENGINE_INTERMEDIATE_INPUT_KEY_CLASS,
-            TezJobConfig.TEZ_ENGINE_INTERMEDIATE_OUTPUT_KEY_CLASS));
+            TezJobConfig.TEZ_RUNTIME_INTERMEDIATE_INPUT_KEY_CLASS,
+            TezJobConfig.TEZ_RUNTIME_INTERMEDIATE_OUTPUT_KEY_CLASS));
     
     multiStageParamMap.put(
         MRJobConfig.MAP_OUTPUT_VALUE_CLASS,
         getDeprecationMap(
-            TezJobConfig.TEZ_ENGINE_INTERMEDIATE_INPUT_VALUE_CLASS,
-            TezJobConfig.TEZ_ENGINE_INTERMEDIATE_OUTPUT_VALUE_CLASS));
+            TezJobConfig.TEZ_RUNTIME_INTERMEDIATE_INPUT_VALUE_CLASS,
+            TezJobConfig.TEZ_RUNTIME_INTERMEDIATE_OUTPUT_VALUE_CLASS));
     
     multiStageParamMap.put(
         MRJobConfig.MAP_OUTPUT_COMPRESS,
         getDeprecationMap(
-            TezJobConfig.TEZ_ENGINE_INTERMEDIATE_INPUT_IS_COMPRESSED,
-            TezJobConfig.TEZ_ENGINE_INTERMEDIATE_OUTPUT_SHOULD_COMPRESS));
+            TezJobConfig.TEZ_RUNTIME_INTERMEDIATE_INPUT_IS_COMPRESSED,
+            TezJobConfig.TEZ_RUNTIME_INTERMEDIATE_OUTPUT_SHOULD_COMPRESS));
     
     multiStageParamMap.put(
         MRJobConfig.MAP_OUTPUT_COMPRESS_CODEC,
         getDeprecationMap(
-            TezJobConfig.TEZ_ENGINE_INTERMEDIATE_INPUT_COMPRESS_CODEC,
-            TezJobConfig.TEZ_ENGINE_INTERMEDIATE_OUTPUT_COMPRESS_CODEC));
+            TezJobConfig.TEZ_RUNTIME_INTERMEDIATE_INPUT_COMPRESS_CODEC,
+            TezJobConfig.TEZ_RUNTIME_INTERMEDIATE_OUTPUT_COMPRESS_CODEC));
   }
   
   private static Map<MultiStageKeys, String> getDeprecationMap(String inputKey, String outputKey) {
@@ -130,23 +131,23 @@ public class DeprecatedKeys {
   public static void init() {
   }
   
-  private static void populateMRToEngineParamMap() {
+  private static void populateMRToTezRuntimeParamMap() {
     
-    registerMRToEngineKeyTranslation(MRConfig.MAPRED_IFILE_READAHEAD, TezJobConfig.TEZ_ENGINE_IFILE_READAHEAD);
+    registerMRToRuntimeKeyTranslation(MRConfig.MAPRED_IFILE_READAHEAD, TezJobConfig.TEZ_RUNTIME_IFILE_READAHEAD);
 
-    registerMRToEngineKeyTranslation(MRConfig.MAPRED_IFILE_READAHEAD_BYTES, TezJobConfig.TEZ_ENGINE_IFILE_READAHEAD_BYTES);
+    registerMRToRuntimeKeyTranslation(MRConfig.MAPRED_IFILE_READAHEAD_BYTES, TezJobConfig.TEZ_RUNTIME_IFILE_READAHEAD_BYTES);
     
-    registerMRToEngineKeyTranslation(MRJobConfig.RECORDS_BEFORE_PROGRESS, TezJobConfig.RECORDS_BEFORE_PROGRESS);
+    registerMRToRuntimeKeyTranslation(MRJobConfig.RECORDS_BEFORE_PROGRESS, TezJobConfig.RECORDS_BEFORE_PROGRESS);
 
-    registerMRToEngineKeyTranslation(MRJobConfig.IO_SORT_FACTOR, TezJobConfig.TEZ_ENGINE_IO_SORT_FACTOR);
+    registerMRToRuntimeKeyTranslation(MRJobConfig.IO_SORT_FACTOR, TezJobConfig.TEZ_RUNTIME_IO_SORT_FACTOR);
     
-    registerMRToEngineKeyTranslation(MRJobConfig.MAP_SORT_SPILL_PERCENT, TezJobConfig.TEZ_ENGINE_SORT_SPILL_PERCENT);
+    registerMRToRuntimeKeyTranslation(MRJobConfig.MAP_SORT_SPILL_PERCENT, TezJobConfig.TEZ_RUNTIME_SORT_SPILL_PERCENT);
     
-    registerMRToEngineKeyTranslation(MRJobConfig.IO_SORT_MB, TezJobConfig.TEZ_ENGINE_IO_SORT_MB);
+    registerMRToRuntimeKeyTranslation(MRJobConfig.IO_SORT_MB, TezJobConfig.TEZ_RUNTIME_IO_SORT_MB);
     
-    registerMRToEngineKeyTranslation(MRJobConfig.INDEX_CACHE_MEMORY_LIMIT, TezJobConfig.TEZ_ENGINE_INDEX_CACHE_MEMORY_LIMIT_BYTES);
+    registerMRToRuntimeKeyTranslation(MRJobConfig.INDEX_CACHE_MEMORY_LIMIT, TezJobConfig.TEZ_RUNTIME_INDEX_CACHE_MEMORY_LIMIT_BYTES);
     
-    registerMRToEngineKeyTranslation(MRJobConfig.MAP_COMBINE_MIN_SPILLS, TezJobConfig.TEZ_ENGINE_COMBINE_MIN_SPILLS);
+    registerMRToRuntimeKeyTranslation(MRJobConfig.MAP_COMBINE_MIN_SPILLS, TezJobConfig.TEZ_RUNTIME_COMBINE_MIN_SPILLS);
     
     // Counter replacement will work in this manner, as long as TezCounters
     // extends MRCounters and is used directly by the Mapper/Reducer.
@@ -154,56 +155,56 @@ public class DeprecatedKeys {
     // may break.
     // Framework counters, like FILESYSTEM will likely be incompatible since
     // they enum key belongs to a different package.
-    registerMRToEngineKeyTranslation(MRJobConfig.COUNTERS_MAX_KEY, TezJobConfig.COUNTERS_MAX_KEY);
+    registerMRToRuntimeKeyTranslation(MRJobConfig.COUNTERS_MAX_KEY, TezJobConfig.COUNTERS_MAX_KEY);
     
-    registerMRToEngineKeyTranslation(MRJobConfig.COUNTER_GROUP_NAME_MAX_KEY, TezJobConfig.COUNTER_GROUP_NAME_MAX_KEY);
+    registerMRToRuntimeKeyTranslation(MRJobConfig.COUNTER_GROUP_NAME_MAX_KEY, TezJobConfig.COUNTER_GROUP_NAME_MAX_KEY);
     
-    registerMRToEngineKeyTranslation(MRJobConfig.COUNTER_NAME_MAX_KEY, TezJobConfig.COUNTER_NAME_MAX_KEY);
+    registerMRToRuntimeKeyTranslation(MRJobConfig.COUNTER_NAME_MAX_KEY, TezJobConfig.COUNTER_NAME_MAX_KEY);
     
-    registerMRToEngineKeyTranslation(MRJobConfig.COUNTER_GROUPS_MAX_KEY, TezJobConfig.COUNTER_GROUPS_MAX_KEY);
+    registerMRToRuntimeKeyTranslation(MRJobConfig.COUNTER_GROUPS_MAX_KEY, TezJobConfig.COUNTER_GROUPS_MAX_KEY);
     
-    registerMRToEngineKeyTranslation(MRJobConfig.REDUCE_MEMORY_TOTAL_BYTES, Constants.TEZ_ENGINE_TASK_MEMORY);
+    registerMRToRuntimeKeyTranslation(MRJobConfig.REDUCE_MEMORY_TOTAL_BYTES, Constants.TEZ_RUNTIME_TASK_MEMORY);
     
-    registerMRToEngineKeyTranslation(MRJobConfig.SHUFFLE_PARALLEL_COPIES, TezJobConfig.TEZ_ENGINE_SHUFFLE_PARALLEL_COPIES);
+    registerMRToRuntimeKeyTranslation(MRJobConfig.SHUFFLE_PARALLEL_COPIES, TezJobConfig.TEZ_RUNTIME_SHUFFLE_PARALLEL_COPIES);
     
-    registerMRToEngineKeyTranslation(MRJobConfig.SHUFFLE_FETCH_FAILURES, TezJobConfig.TEZ_ENGINE_SHUFFLE_FETCH_FAILURES);
+    registerMRToRuntimeKeyTranslation(MRJobConfig.SHUFFLE_FETCH_FAILURES, TezJobConfig.TEZ_RUNTIME_SHUFFLE_FETCH_FAILURES);
     
-    registerMRToEngineKeyTranslation(MRJobConfig.SHUFFLE_NOTIFY_READERROR, TezJobConfig.TEZ_ENGINE_SHUFFLE_NOTIFY_READERROR);
+    registerMRToRuntimeKeyTranslation(MRJobConfig.SHUFFLE_NOTIFY_READERROR, TezJobConfig.TEZ_RUNTIME_SHUFFLE_NOTIFY_READERROR);
     
-    registerMRToEngineKeyTranslation(MRJobConfig.SHUFFLE_CONNECT_TIMEOUT, TezJobConfig.TEZ_ENGINE_SHUFFLE_CONNECT_TIMEOUT);
+    registerMRToRuntimeKeyTranslation(MRJobConfig.SHUFFLE_CONNECT_TIMEOUT, TezJobConfig.TEZ_RUNTIME_SHUFFLE_CONNECT_TIMEOUT);
     
-    registerMRToEngineKeyTranslation(MRJobConfig.SHUFFLE_READ_TIMEOUT, TezJobConfig.TEZ_ENGINE_SHUFFLE_READ_TIMEOUT);
+    registerMRToRuntimeKeyTranslation(MRJobConfig.SHUFFLE_READ_TIMEOUT, TezJobConfig.TEZ_RUNTIME_SHUFFLE_READ_TIMEOUT);
     
-    registerMRToEngineKeyTranslation(MRConfig.SHUFFLE_SSL_ENABLED_KEY, TezJobConfig.TEZ_ENGINE_SHUFFLE_ENABLE_SSL);
+    registerMRToRuntimeKeyTranslation(MRConfig.SHUFFLE_SSL_ENABLED_KEY, TezJobConfig.TEZ_RUNTIME_SHUFFLE_ENABLE_SSL);
     
-    registerMRToEngineKeyTranslation(MRJobConfig.SHUFFLE_INPUT_BUFFER_PERCENT, TezJobConfig.TEZ_ENGINE_SHUFFLE_INPUT_BUFFER_PERCENT);
+    registerMRToRuntimeKeyTranslation(MRJobConfig.SHUFFLE_INPUT_BUFFER_PERCENT, TezJobConfig.TEZ_RUNTIME_SHUFFLE_INPUT_BUFFER_PERCENT);
     
-    registerMRToEngineKeyTranslation(MRJobConfig.SHUFFLE_MEMORY_LIMIT_PERCENT, TezJobConfig.TEZ_ENGINE_SHUFFLE_MEMORY_LIMIT_PERCENT);
+    registerMRToRuntimeKeyTranslation(MRJobConfig.SHUFFLE_MEMORY_LIMIT_PERCENT, TezJobConfig.TEZ_RUNTIME_SHUFFLE_MEMORY_LIMIT_PERCENT);
     
-    registerMRToEngineKeyTranslation(MRJobConfig.SHUFFLE_MERGE_PERCENT, TezJobConfig.TEZ_ENGINE_SHUFFLE_MERGE_PERCENT);
+    registerMRToRuntimeKeyTranslation(MRJobConfig.SHUFFLE_MERGE_PERCENT, TezJobConfig.TEZ_RUNTIME_SHUFFLE_MERGE_PERCENT);
     
-    registerMRToEngineKeyTranslation(MRJobConfig.REDUCE_MEMTOMEM_THRESHOLD, TezJobConfig.TEZ_ENGINE_SHUFFLE_MEMTOMEM_SEGMENTS);
+    registerMRToRuntimeKeyTranslation(MRJobConfig.REDUCE_MEMTOMEM_THRESHOLD, TezJobConfig.TEZ_RUNTIME_SHUFFLE_MEMTOMEM_SEGMENTS);
     
-    registerMRToEngineKeyTranslation(MRJobConfig.REDUCE_MEMTOMEM_ENABLED, TezJobConfig.TEZ_ENGINE_SHUFFLE_ENABLE_MEMTOMEM);
+    registerMRToRuntimeKeyTranslation(MRJobConfig.REDUCE_MEMTOMEM_ENABLED, TezJobConfig.TEZ_RUNTIME_SHUFFLE_ENABLE_MEMTOMEM);
     
-    registerMRToEngineKeyTranslation(MRJobConfig.REDUCE_INPUT_BUFFER_PERCENT, TezJobConfig.TEZ_ENGINE_INPUT_BUFFER_PERCENT);
+    registerMRToRuntimeKeyTranslation(MRJobConfig.REDUCE_INPUT_BUFFER_PERCENT, TezJobConfig.TEZ_RUNTIME_INPUT_BUFFER_PERCENT);
 
-    registerMRToEngineKeyTranslation(MRJobConfig.MAPREDUCE_JOB_CREDENTIALS_BINARY, TezJobConfig.DAG_CREDENTIALS_BINARY);
+    registerMRToRuntimeKeyTranslation(MRJobConfig.MAPREDUCE_JOB_CREDENTIALS_BINARY, TezJobConfig.DAG_CREDENTIALS_BINARY);
     
-    registerMRToEngineKeyTranslation("map.sort.class", TezJobConfig.TEZ_ENGINE_INTERNAL_SORTER_CLASS);
+    registerMRToRuntimeKeyTranslation("map.sort.class", TezJobConfig.TEZ_RUNTIME_INTERNAL_SORTER_CLASS);
     
-    registerMRToEngineKeyTranslation(MRJobConfig.GROUP_COMPARATOR_CLASS, TezJobConfig.TEZ_ENGINE_GROUP_COMPARATOR_CLASS);
+    registerMRToRuntimeKeyTranslation(MRJobConfig.GROUP_COMPARATOR_CLASS, TezJobConfig.TEZ_RUNTIME_GROUP_COMPARATOR_CLASS);
     
-    registerMRToEngineKeyTranslation(MRJobConfig.GROUP_COMPARATOR_CLASS, TezJobConfig.TEZ_ENGINE_INTERMEDIATE_INPUT_KEY_SECONDARY_COMPARATOR_CLASS);
+    registerMRToRuntimeKeyTranslation(MRJobConfig.GROUP_COMPARATOR_CLASS, TezJobConfig.TEZ_RUNTIME_INTERMEDIATE_INPUT_KEY_SECONDARY_COMPARATOR_CLASS);
 
   }
   
   private static void addDeprecatedKeys() {
   }
 
-  private static void registerMRToEngineKeyTranslation(String mrKey,
+  private static void registerMRToRuntimeKeyTranslation(String mrKey,
       String tezKey) {
-    mrParamToEngineParamMap.put(mrKey, tezKey);
+    mrParamToTezRuntimeParamMap.put(mrKey, tezKey);
   }
   
   @SuppressWarnings("unused")
@@ -215,8 +216,8 @@ public class DeprecatedKeys {
     return Collections.unmodifiableMap(mrParamToDAGParamMap);
   }
 
-  public static Map<String, String> getMRToEngineParamMap() {
-    return Collections.unmodifiableMap(mrParamToEngineParamMap);
+  public static Map<String, String> getMRToTezRuntimeParamMap() {
+    return Collections.unmodifiableMap(mrParamToTezRuntimeParamMap);
   }
 
   // TODO Ideally, multi-stage should not be exposed.

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/MRHelpers.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/MRHelpers.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/MRHelpers.java
index b0ed6ab..c39ca4a 100644
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/MRHelpers.java
+++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/MRHelpers.java
@@ -367,16 +367,16 @@ public class MRHelpers {
     // the AM anyway.
 
     // TODO eventually ACLs
-    conf.set(TezJobConfig.TEZ_ENGINE_PARTITIONER_CLASS, MRPartitioner.class.getName());
+    conf.set(TezJobConfig.TEZ_RUNTIME_PARTITIONER_CLASS, MRPartitioner.class.getName());
     
     boolean useNewApi = conf.getBoolean("mapred.mapper.new-api", false);
     if (useNewApi) {
       if (conf.get(MRJobConfig.COMBINE_CLASS_ATTR) != null) {
-        conf.set(TezJobConfig.TEZ_ENGINE_COMBINER_CLASS, MRCombiner.class.getName());
+        conf.set(TezJobConfig.TEZ_RUNTIME_COMBINER_CLASS, MRCombiner.class.getName());
       }
     } else {
       if (conf.get("mapred.combiner.class") != null) {
-        conf.set(TezJobConfig.TEZ_ENGINE_COMBINER_CLASS, MRCombiner.class.getName());
+        conf.set(TezJobConfig.TEZ_RUNTIME_COMBINER_CLASS, MRCombiner.class.getName());
       }
     }
     

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/MultiStageMRConfToTezTranslator.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/MultiStageMRConfToTezTranslator.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/MultiStageMRConfToTezTranslator.java
index d888c42..b07b04b 100644
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/MultiStageMRConfToTezTranslator.java
+++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/MultiStageMRConfToTezTranslator.java
@@ -221,7 +221,7 @@ public class MultiStageMRConfToTezTranslator {
     int numStages = numIntermediateStages + (hasFinalReduceStage ? 2 : 1);
 
     // Setup Tez partitioner class
-    conf.set(TezJobConfig.TEZ_ENGINE_PARTITIONER_CLASS,
+    conf.set(TezJobConfig.TEZ_RUNTIME_PARTITIONER_CLASS,
         MRPartitioner.class.getName());
     
     // Setup Tez Combiner class if required.
@@ -229,11 +229,11 @@ public class MultiStageMRConfToTezTranslator {
     boolean useNewApi = conf.getBoolean("mapred.mapper.new-api", false);
     if (useNewApi) {
       if (conf.get(MRJobConfig.COMBINE_CLASS_ATTR) != null) {
-        conf.set(TezJobConfig.TEZ_ENGINE_COMBINER_CLASS, MRCombiner.class.getName());
+        conf.set(TezJobConfig.TEZ_RUNTIME_COMBINER_CLASS, MRCombiner.class.getName());
       }
     } else {
       if (conf.get("mapred.combiner.class") != null) {
-        conf.set(TezJobConfig.TEZ_ENGINE_COMBINER_CLASS, MRCombiner.class.getName());
+        conf.set(TezJobConfig.TEZ_RUNTIME_COMBINER_CLASS, MRCombiner.class.getName());
       }
     }
 
@@ -259,7 +259,7 @@ public class MultiStageMRConfToTezTranslator {
   }
 
   private static void processDirectConversion(Configuration conf) {
-    for (Entry<String, String> dep : DeprecatedKeys.getMRToEngineParamMap()
+    for (Entry<String, String> dep : DeprecatedKeys.getMRToTezRuntimeParamMap()
         .entrySet()) {
       if (conf.get(dep.getKey()) != null) {
         // TODO Deprecation reason does not seem to reflect in the config ?
@@ -336,7 +336,7 @@ public class MultiStageMRConfToTezTranslator {
       Configuration baseConf, String stage) {
     JobConf jobConf = new JobConf(baseConf);
     // Don't clobber explicit tez config.
-    if (conf.get(TezJobConfig.TEZ_ENGINE_INTERMEDIATE_OUTPUT_KEY_CLASS) == null) {
+    if (conf.get(TezJobConfig.TEZ_RUNTIME_INTERMEDIATE_OUTPUT_KEY_CLASS) == null) {
       // If this is set, but the comparator is not set, and their types differ -
       // the job will break.
       if (conf.get(MRJobConfig.MAP_OUTPUT_KEY_CLASS) == null) {
@@ -352,7 +352,7 @@ public class MultiStageMRConfToTezTranslator {
       }
     }
 
-    if (conf.get(TezJobConfig.TEZ_ENGINE_INTERMEDIATE_OUTPUT_VALUE_CLASS) == null) {
+    if (conf.get(TezJobConfig.TEZ_RUNTIME_INTERMEDIATE_OUTPUT_VALUE_CLASS) == null) {
       if (conf.get(MRJobConfig.MAP_OUTPUT_VALUE_CLASS) == null) {
         conf.set(MRJobConfig.MAP_OUTPUT_VALUE_CLASS, jobConf
             .getMapOutputValueClass().getName());

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/MultiStageMRConfigUtil.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/MultiStageMRConfigUtil.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/MultiStageMRConfigUtil.java
index 0bcd45e..2378f58 100644
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/MultiStageMRConfigUtil.java
+++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/MultiStageMRConfigUtil.java
@@ -57,7 +57,7 @@ public class MultiStageMRConfigUtil {
   public static Configuration getAndRemoveBasicNonIntermediateStageConf(
       Configuration baseConf) {
     Configuration newConf = new Configuration(false);
-    for (String key : DeprecatedKeys.getMRToEngineParamMap().keySet()) {
+    for (String key : DeprecatedKeys.getMRToTezRuntimeParamMap().keySet()) {
       if (baseConf.get(key) != null) {
         newConf.set(key, baseConf.get(key));
         baseConf.unset(key);

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/mapred/MRReporter.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/mapred/MRReporter.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/mapred/MRReporter.java
index 2a926d7..635af90 100644
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/mapred/MRReporter.java
+++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/mapred/MRReporter.java
@@ -3,9 +3,9 @@ package org.apache.tez.mapreduce.hadoop.mapred;
 import org.apache.hadoop.mapred.Counters.Counter;
 import org.apache.hadoop.mapred.InputSplit;
 import org.apache.hadoop.mapred.Reporter;
-import org.apache.tez.engine.api.TezProcessorContext;
-import org.apache.tez.engine.api.TezTaskContext;
 import org.apache.tez.mapreduce.common.Utils;
+import org.apache.tez.runtime.api.TezProcessorContext;
+import org.apache.tez.runtime.api.TezTaskContext;
 
 public class MRReporter implements Reporter {
 

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/mapreduce/MapContextImpl.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/mapreduce/MapContextImpl.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/mapreduce/MapContextImpl.java
index dcdb3ff..2d27c4b 100644
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/mapreduce/MapContextImpl.java
+++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/mapreduce/MapContextImpl.java
@@ -30,7 +30,7 @@ import org.apache.hadoop.mapreduce.OutputCommitter;
 import org.apache.hadoop.mapreduce.RecordReader;
 import org.apache.hadoop.mapreduce.RecordWriter;
 import org.apache.hadoop.mapreduce.TaskAttemptID;
-import org.apache.tez.engine.api.TezTaskContext;
+import org.apache.tez.runtime.api.TezTaskContext;
 
 /**
  * The context that is given to the {@link Mapper}.

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/mapreduce/TaskAttemptContextImpl.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/mapreduce/TaskAttemptContextImpl.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/mapreduce/TaskAttemptContextImpl.java
index 4035c71..be65be7 100644
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/mapreduce/TaskAttemptContextImpl.java
+++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/mapreduce/TaskAttemptContextImpl.java
@@ -24,8 +24,8 @@ import org.apache.hadoop.mapreduce.Counter;
 import org.apache.hadoop.mapreduce.TaskAttemptID;
 import org.apache.hadoop.mapreduce.TaskID;
 import org.apache.hadoop.mapreduce.TaskType;
-import org.apache.tez.engine.api.TezTaskContext;
 import org.apache.tez.mapreduce.common.Utils;
+import org.apache.tez.runtime.api.TezTaskContext;
 
 // NOTE: NEWTEZ: This is a copy of org.apache.tez.mapreduce.hadoop.mapred (not mapreduce). mapred likely does not need it's own copy of this class.
 // Meant for use by the "mapreduce" API

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/mapreduce/TaskInputOutputContextImpl.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/mapreduce/TaskInputOutputContextImpl.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/mapreduce/TaskInputOutputContextImpl.java
index 05ea89c..5b5c8ec 100644
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/mapreduce/TaskInputOutputContextImpl.java
+++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/mapreduce/TaskInputOutputContextImpl.java
@@ -29,7 +29,7 @@ import org.apache.hadoop.mapreduce.RecordWriter;
 import org.apache.hadoop.mapreduce.Reducer;
 import org.apache.hadoop.mapreduce.TaskAttemptID;
 import org.apache.hadoop.mapreduce.TaskInputOutputContext;
-import org.apache.tez.engine.api.TezTaskContext;
+import org.apache.tez.runtime.api.TezTaskContext;
 
 /**
  * A context object that allows input and output from the task. It is only

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/MRInput.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/MRInput.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/MRInput.java
index 6066d93..b9f2242 100644
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/MRInput.java
+++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/MRInput.java
@@ -46,13 +46,14 @@ import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.tez.common.TezUtils;
 import org.apache.tez.common.counters.TaskCounter;
 import org.apache.tez.common.counters.TezCounter;
-import org.apache.tez.engine.api.Event;
-import org.apache.tez.engine.api.KVReader;
-import org.apache.tez.engine.api.LogicalInput;
-import org.apache.tez.engine.api.TezInputContext;
 import org.apache.tez.mapreduce.common.Utils;
 import org.apache.tez.mapreduce.hadoop.mapred.MRReporter;
 import org.apache.tez.mapreduce.hadoop.mapreduce.TaskAttemptContextImpl;
+import org.apache.tez.runtime.api.Event;
+import org.apache.tez.runtime.api.Input;
+import org.apache.tez.runtime.api.LogicalInput;
+import org.apache.tez.runtime.api.TezInputContext;
+import org.apache.tez.runtime.library.api.KVReader;
 
 import com.google.common.base.Preconditions;
 

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/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 e6bdbe6..11184e4 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
@@ -26,16 +26,16 @@ import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.tez.common.TezUtils;
 import org.apache.tez.common.counters.TaskCounter;
 import org.apache.tez.common.counters.TezCounter;
-import org.apache.tez.engine.api.Event;
-import org.apache.tez.engine.api.KVWriter;
-import org.apache.tez.engine.api.LogicalOutput;
-import org.apache.tez.engine.api.TezOutputContext;
 import org.apache.tez.mapreduce.common.Utils;
 import org.apache.tez.mapreduce.hadoop.MRConfig;
 import org.apache.tez.mapreduce.hadoop.MRJobConfig;
 import org.apache.tez.mapreduce.hadoop.mapred.MRReporter;
 import org.apache.tez.mapreduce.hadoop.mapreduce.TaskAttemptContextImpl;
 import org.apache.tez.mapreduce.processor.MRTaskReporter;
+import org.apache.tez.runtime.api.Event;
+import org.apache.tez.runtime.api.LogicalOutput;
+import org.apache.tez.runtime.api.TezOutputContext;
+import org.apache.tez.runtime.library.api.KVWriter;
 
 public class MROutput implements LogicalOutput {
 

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/partition/MRPartitioner.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/partition/MRPartitioner.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/partition/MRPartitioner.java
index d061ad5..224900e 100644
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/partition/MRPartitioner.java
+++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/partition/MRPartitioner.java
@@ -24,11 +24,11 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.tez.common.TezJobConfig;
-import org.apache.tez.engine.common.ConfigUtils;
 import org.apache.tez.mapreduce.hadoop.MRJobConfig;
+import org.apache.tez.runtime.library.common.ConfigUtils;
 
 @SuppressWarnings({ "rawtypes", "unchecked" })
-public class MRPartitioner implements org.apache.tez.engine.api.Partitioner {
+public class MRPartitioner implements org.apache.tez.runtime.library.api.Partitioner {
 
   static final Log LOG = LogFactory.getLog(MRPartitioner.class);
 
@@ -40,7 +40,7 @@ public class MRPartitioner implements org.apache.tez.engine.api.Partitioner {
 
   public MRPartitioner(Configuration conf) {
     this.useNewApi = ConfigUtils.useNewApi(conf);
-    this.partitions = conf.getInt(TezJobConfig.TEZ_ENGINE_NUM_EXPECTED_PARTITIONS, 1);
+    this.partitions = conf.getInt(TezJobConfig.TEZ_RUNTIME_NUM_EXPECTED_PARTITIONS, 1);
 
     if (useNewApi) {
       if (partitions > 1) {
@@ -85,4 +85,4 @@ public class MRPartitioner implements org.apache.tez.engine.api.Partitioner {
       return oldPartitioner.getPartition(key, value, numPartitions);
     }
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/MRTask.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/MRTask.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/MRTask.java
index fac1454..5471c55 100644
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/MRTask.java
+++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/MRTask.java
@@ -62,7 +62,6 @@ import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.DiskChecker.DiskErrorException;
 import org.apache.hadoop.yarn.api.ApplicationConstants.Environment;
 import org.apache.hadoop.yarn.util.ResourceCalculatorProcessTree;
-import org.apache.tez.common.Constants;
 import org.apache.tez.common.TezJobConfig;
 import org.apache.tez.common.TezTaskStatus.State;
 import org.apache.tez.common.TezUtils;
@@ -70,11 +69,6 @@ import org.apache.tez.common.counters.TaskCounter;
 import org.apache.tez.common.counters.TezCounter;
 import org.apache.tez.common.counters.TezCounters;
 import org.apache.tez.dag.records.TezDAGID;
-import org.apache.tez.engine.api.LogicalOutput;
-import org.apache.tez.engine.api.TezProcessorContext;
-import org.apache.tez.engine.common.security.JobTokenIdentifier;
-import org.apache.tez.engine.common.security.TokenCache;
-import org.apache.tez.engine.common.sort.impl.TezRawKeyValueIterator;
 import org.apache.tez.mapreduce.hadoop.DeprecatedKeys;
 import org.apache.tez.mapreduce.hadoop.IDConverter;
 import org.apache.tez.mapreduce.hadoop.MRConfig;
@@ -82,6 +76,12 @@ import org.apache.tez.mapreduce.hadoop.MRJobConfig;
 import org.apache.tez.mapreduce.hadoop.mapred.TaskAttemptContextImpl;
 import org.apache.tez.mapreduce.hadoop.mapreduce.JobContextImpl;
 import org.apache.tez.mapreduce.output.MROutput;
+import org.apache.tez.runtime.api.LogicalOutput;
+import org.apache.tez.runtime.api.TezProcessorContext;
+import org.apache.tez.runtime.library.common.Constants;
+import org.apache.tez.runtime.library.common.security.JobTokenIdentifier;
+import org.apache.tez.runtime.library.common.security.TokenCache;
+import org.apache.tez.runtime.library.common.sort.impl.TezRawKeyValueIterator;
 
 @SuppressWarnings("deprecation")
 public abstract class MRTask {
@@ -155,7 +155,7 @@ public abstract class MRTask {
     } else {
       this.jobConf = new JobConf(conf);
     }
-    jobConf.set(Constants.TEZ_ENGINE_TASK_ATTEMPT_ID,
+    jobConf.set(Constants.TEZ_RUNTIME_TASK_ATTEMPT_ID,
         taskAttemptId.toString());
     jobConf.setInt(MRJobConfig.APPLICATION_ATTEMPT_ID,
         context.getDAGAttemptNumber());
@@ -687,7 +687,4 @@ public abstract class MRTask {
     return taskAttemptId;
   }
 
-  public TezProcessorContext getTezEngineTaskContext() {
-    return processorContext;
-  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/MRTaskReporter.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/MRTaskReporter.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/MRTaskReporter.java
index 85139ed..74a34af 100644
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/MRTaskReporter.java
+++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/MRTaskReporter.java
@@ -24,12 +24,12 @@ import org.apache.hadoop.mapred.Counters;
 import org.apache.hadoop.mapred.InputSplit;
 import org.apache.hadoop.mapred.Reporter;
 import org.apache.tez.common.counters.TezCounter;
-import org.apache.tez.engine.api.TezInputContext;
-import org.apache.tez.engine.api.TezOutputContext;
-import org.apache.tez.engine.api.TezProcessorContext;
-import org.apache.tez.engine.api.TezTaskContext;
 import org.apache.tez.mapreduce.hadoop.mapred.MRCounters;
 import org.apache.tez.mapreduce.hadoop.mapred.MRReporter;
+import org.apache.tez.runtime.api.TezInputContext;
+import org.apache.tez.runtime.api.TezOutputContext;
+import org.apache.tez.runtime.api.TezProcessorContext;
+import org.apache.tez.runtime.api.TezTaskContext;
 
 @InterfaceAudience.Private
 @InterfaceStability.Unstable

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/map/MapProcessor.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/map/MapProcessor.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/map/MapProcessor.java
index e4b990a..b7ecddd 100644
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/map/MapProcessor.java
+++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/map/MapProcessor.java
@@ -35,20 +35,20 @@ import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.tez.common.counters.TaskCounter;
 import org.apache.tez.common.counters.TezCounter;
 import org.apache.tez.dag.api.TezException;
-import org.apache.tez.engine.api.Event;
-import org.apache.tez.engine.api.KVReader;
-import org.apache.tez.engine.api.KVWriter;
-import org.apache.tez.engine.api.LogicalIOProcessor;
-import org.apache.tez.engine.api.LogicalInput;
-import org.apache.tez.engine.api.LogicalOutput;
-import org.apache.tez.engine.api.TezProcessorContext;
-import org.apache.tez.engine.lib.output.OnFileSortedOutput;
 import org.apache.tez.mapreduce.hadoop.mapreduce.MapContextImpl;
 import org.apache.tez.mapreduce.input.MRInput;
 import org.apache.tez.mapreduce.input.MRInputLegacy;
 import org.apache.tez.mapreduce.output.MROutput;
 import org.apache.tez.mapreduce.processor.MRTask;
 import org.apache.tez.mapreduce.processor.MRTaskReporter;
+import org.apache.tez.runtime.api.Event;
+import org.apache.tez.runtime.api.LogicalIOProcessor;
+import org.apache.tez.runtime.api.LogicalInput;
+import org.apache.tez.runtime.api.LogicalOutput;
+import org.apache.tez.runtime.api.TezProcessorContext;
+import org.apache.tez.runtime.library.api.KVReader;
+import org.apache.tez.runtime.library.api.KVWriter;
+import org.apache.tez.runtime.library.output.OnFileSortedOutput;
 
 @SuppressWarnings({ "unchecked", "rawtypes" })
 public class MapProcessor extends MRTask implements LogicalIOProcessor {

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/reduce/ReduceProcessor.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/reduce/ReduceProcessor.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/reduce/ReduceProcessor.java
index 19acb39..1ba76f6 100644
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/reduce/ReduceProcessor.java
+++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/reduce/ReduceProcessor.java
@@ -37,20 +37,20 @@ import org.apache.hadoop.util.Progressable;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.tez.common.counters.TaskCounter;
 import org.apache.tez.common.counters.TezCounter;
-import org.apache.tez.engine.api.Event;
-import org.apache.tez.engine.api.KVReader;
-import org.apache.tez.engine.api.KVWriter;
-import org.apache.tez.engine.api.LogicalIOProcessor;
-import org.apache.tez.engine.api.LogicalInput;
-import org.apache.tez.engine.api.LogicalOutput;
-import org.apache.tez.engine.api.TezProcessorContext;
-import org.apache.tez.engine.common.ConfigUtils;
-import org.apache.tez.engine.common.sort.impl.TezRawKeyValueIterator;
-import org.apache.tez.engine.lib.input.ShuffledMergedInputLegacy;
-import org.apache.tez.engine.lib.output.OnFileSortedOutput;
 import org.apache.tez.mapreduce.output.MROutput;
 import org.apache.tez.mapreduce.processor.MRTask;
 import org.apache.tez.mapreduce.processor.MRTaskReporter;
+import org.apache.tez.runtime.api.Event;
+import org.apache.tez.runtime.api.LogicalIOProcessor;
+import org.apache.tez.runtime.api.LogicalInput;
+import org.apache.tez.runtime.api.LogicalOutput;
+import org.apache.tez.runtime.api.TezProcessorContext;
+import org.apache.tez.runtime.library.api.KVReader;
+import org.apache.tez.runtime.library.api.KVWriter;
+import org.apache.tez.runtime.library.common.ConfigUtils;
+import org.apache.tez.runtime.library.common.sort.impl.TezRawKeyValueIterator;
+import org.apache.tez.runtime.library.input.ShuffledMergedInputLegacy;
+import org.apache.tez.runtime.library.output.OnFileSortedOutput;
 
 
 @SuppressWarnings({ "unchecked", "rawtypes" })

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/TestUmbilical.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/TestUmbilical.java b/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/TestUmbilical.java
index 9de2ed1..08b66eb 100644
--- a/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/TestUmbilical.java
+++ b/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/TestUmbilical.java
@@ -24,9 +24,9 @@ import java.util.Collection;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.tez.dag.records.TezTaskAttemptID;
-import org.apache.tez.engine.api.impl.EventMetaData;
-import org.apache.tez.engine.api.impl.TezEvent;
-import org.apache.tez.engine.api.impl.TezUmbilical;
+import org.apache.tez.runtime.api.impl.EventMetaData;
+import org.apache.tez.runtime.api.impl.TezEvent;
+import org.apache.tez.runtime.api.impl.TezUmbilical;
 
 public class TestUmbilical implements TezUmbilical {
 

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/hadoop/TestConfigTranslationMRToTez.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/hadoop/TestConfigTranslationMRToTez.java b/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/hadoop/TestConfigTranslationMRToTez.java
index b75f01e..5e3d201 100644
--- a/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/hadoop/TestConfigTranslationMRToTez.java
+++ b/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/hadoop/TestConfigTranslationMRToTez.java
@@ -29,7 +29,7 @@ import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.mapred.JobConf;
-import org.apache.tez.engine.common.ConfigUtils;
+import org.apache.tez.runtime.library.common.ConfigUtils;
 import org.junit.Test;
 
 public class TestConfigTranslationMRToTez {


[07/50] [abbrv] TEZ-444. Rename *.new* packages back to what they should be, remove dead code from the old packages - mapreduce module (part of TEZ-398). (sseth)

Posted by ss...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/3d609458/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/MRTask.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/MRTask.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/MRTask.java
index 0653cc8..af087c9 100644
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/MRTask.java
+++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/MRTask.java
@@ -19,69 +19,75 @@
 package org.apache.tez.mapreduce.processor;
 
 import java.io.IOException;
+import java.net.URI;
 import java.text.NumberFormat;
 import java.util.ArrayList;
 import java.util.HashMap;
+import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.FutureTask;
+import java.util.Map.Entry;
 import java.util.concurrent.atomic.AtomicBoolean;
 
+import javax.crypto.SecretKey;
+
 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.fs.FileSystem;
+import org.apache.hadoop.fs.LocalDirAllocator;
 import org.apache.hadoop.fs.FileSystem.Statistics;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.DataInputBuffer;
 import org.apache.hadoop.io.RawComparator;
-import org.apache.hadoop.mapred.FileOutputCommitter;
-import org.apache.hadoop.mapred.FileOutputFormat;
+import org.apache.hadoop.mapred.FileAlreadyExistsException;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.JobContext;
+import org.apache.hadoop.mapred.MapOutputFile;
 import org.apache.hadoop.mapred.RawKeyValueIterator;
 import org.apache.hadoop.mapred.TaskAttemptContext;
+import org.apache.hadoop.mapred.TaskAttemptID;
+import org.apache.hadoop.mapred.TaskID;
 import org.apache.hadoop.mapreduce.OutputCommitter;
-import org.apache.hadoop.mapreduce.OutputFormat;
-import org.apache.hadoop.mapreduce.lib.input.FileInputFormatCounter;
-import org.apache.hadoop.mapreduce.lib.output.FileOutputFormatCounter;
+import org.apache.hadoop.mapreduce.TaskType;
+import org.apache.hadoop.mapreduce.filecache.DistributedCache;
 import org.apache.hadoop.mapreduce.lib.reduce.WrappedReducer;
+import org.apache.hadoop.mapreduce.security.token.JobTokenSecretManager;
 import org.apache.hadoop.mapreduce.task.ReduceContextImpl;
+import org.apache.hadoop.security.Credentials;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.util.Progress;
-import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.util.DiskChecker.DiskErrorException;
+import org.apache.hadoop.yarn.api.ApplicationConstants.Environment;
 import org.apache.hadoop.yarn.util.ResourceCalculatorProcessTree;
 import org.apache.tez.common.Constants;
-import org.apache.tez.common.RunningTaskContext;
-import org.apache.tez.common.TezEngineTaskContext;
-import org.apache.tez.common.TezTaskStatus;
-import org.apache.tez.common.TezTaskUmbilicalProtocol;
-import org.apache.tez.common.TezTaskStatus.Phase;
+import org.apache.tez.common.TezJobConfig;
 import org.apache.tez.common.TezTaskStatus.State;
+import org.apache.tez.common.TezUtils;
 import org.apache.tez.common.counters.TaskCounter;
 import org.apache.tez.common.counters.TezCounter;
 import org.apache.tez.common.counters.TezCounters;
-import org.apache.tez.common.records.ProceedToCompletionResponse;
 import org.apache.tez.dag.records.TezDAGID;
-import org.apache.tez.dag.records.TezTaskAttemptID;
-import org.apache.tez.engine.api.Input;
-import org.apache.tez.engine.api.Master;
-import org.apache.tez.engine.api.Output;
+import org.apache.tez.engine.common.security.JobTokenIdentifier;
+import org.apache.tez.engine.common.security.TokenCache;
 import org.apache.tez.engine.common.sort.impl.TezRawKeyValueIterator;
+import org.apache.tez.engine.newapi.LogicalOutput;
+import org.apache.tez.engine.newapi.TezProcessorContext;
 import org.apache.tez.engine.records.OutputContext;
-import org.apache.tez.mapreduce.combine.MRCombiner;
+import org.apache.tez.mapreduce.hadoop.DeprecatedKeys;
 import org.apache.tez.mapreduce.hadoop.IDConverter;
 import org.apache.tez.mapreduce.hadoop.MRConfig;
-import org.apache.tez.mapreduce.hadoop.MRTaskStatus;
+import org.apache.tez.mapreduce.hadoop.MRJobConfig;
 import org.apache.tez.mapreduce.hadoop.mapred.TaskAttemptContextImpl;
 import org.apache.tez.mapreduce.hadoop.mapreduce.JobContextImpl;
-import org.apache.tez.mapreduce.hadoop.mapreduce.TezNullOutputCommitter;
-import org.apache.tez.mapreduce.partition.MRPartitioner;
+import org.apache.tez.mapreduce.output.SimpleOutput;
+import org.apache.tez.mapreduce.task.impl.YarnOutputFiles;
 
-public abstract class MRTask extends RunningTaskContext {
+@SuppressWarnings("deprecation")
+public abstract class MRTask {
 
   static final Log LOG = LogFactory.getLog(MRTask.class);
 
@@ -91,12 +97,16 @@ public abstract class MRTask extends RunningTaskContext {
   protected OutputCommitter committer;
 
   // Current counters
-  transient TezCounters counters = new TezCounters();
+  transient TezCounters counters;
   protected GcTimeUpdater gcUpdater;
   private ResourceCalculatorProcessTree pTree;
   private long initCpuCumulativeTime = 0;
-  protected TezEngineTaskContext tezEngineTaskContext;
-  protected TezTaskAttemptID taskAttemptId;
+  protected TezProcessorContext processorContext;
+  protected TaskAttemptID taskAttemptId;
+  protected Progress progress = new Progress();
+  protected SecretKey jobTokenSecret;
+
+  boolean isMap;
 
   /* flag to track whether task is done */
   AtomicBoolean taskDone = new AtomicBoolean(false);
@@ -109,12 +119,8 @@ public abstract class MRTask extends RunningTaskContext {
     NUMBER_FORMAT.setGroupingUsed(false);
   }
 
-  private final static int MAX_RETRIES = 10;
-
-  /** The number of milliseconds between progress reports. */
-  public static final int PROGRESS_INTERVAL = 3000;
-
-  private MRTaskReporter mrReporter;
+  protected MRTaskReporter mrReporter;
+  protected boolean useNewApi;
 
   /**
    * A Map where Key-> URIScheme and value->FileSystemStatisticUpdater
@@ -122,41 +128,202 @@ public abstract class MRTask extends RunningTaskContext {
   private Map<String, FileSystemStatisticUpdater> statisticUpdaters =
      new HashMap<String, FileSystemStatisticUpdater>();
 
-  public MRTask(TezEngineTaskContext context) {
-    tezEngineTaskContext = context;
-    this.taskAttemptId = context.getTaskAttemptId();
-    // TODO TEZAM4 Figure out initialization / run sequence of Input, Process,
-    // Output. Phase is MR specific.
-    status =
-        new MRTaskStatus(
-            taskAttemptId,
-            counters,
-            (taskAttemptId.getTaskID().getVertexID().getId() == 0 ?
-                Phase.MAP : Phase.SHUFFLE)
-        );
-    gcUpdater = new GcTimeUpdater(counters);
+  public MRTask(boolean isMap) {
+    this.isMap = isMap;
   }
 
-  public void initialize(Configuration conf, Master master) throws IOException,
+  // TODO how to update progress
+  public void initialize(TezProcessorContext context) throws IOException,
   InterruptedException {
 
+    DeprecatedKeys.init();
+
+    processorContext = context;
+    counters = context.getCounters();
+    this.taskAttemptId = new TaskAttemptID(
+        new TaskID(
+            Long.toString(context.getApplicationId().getClusterTimestamp()),
+            context.getApplicationId().getId(),
+            (isMap ? TaskType.MAP : TaskType.REDUCE),
+            context.getTaskIndex()),
+          context.getTaskAttemptNumber());
+    // TODO TEZAM4 Figure out initialization / run sequence of Input, Process,
+    // Output. Phase is MR specific.
+    gcUpdater = new GcTimeUpdater(counters);
+
+    byte[] userPayload = context.getUserPayload();
+    Configuration conf = TezUtils.createConfFromUserPayload(userPayload);
     if (conf instanceof JobConf) {
       this.jobConf = (JobConf)conf;
     } else {
       this.jobConf = new JobConf(conf);
     }
-    reporter =
-        new TezTaskReporterImpl(this, (TezTaskUmbilicalProtocol)master);
-    ((TezTaskReporterImpl)reporter).startCommunicationThread();
-
     jobConf.set(Constants.TEZ_ENGINE_TASK_ATTEMPT_ID,
         taskAttemptId.toString());
+    jobConf.setInt(MRJobConfig.APPLICATION_ATTEMPT_ID,
+        context.getDAGAttemptNumber());
 
     initResourceCalculatorPlugin();
 
     LOG.info("MRTask.inited: taskAttemptId = " + taskAttemptId.toString());
+
+    // TODO Post MRR
+    // A single file per vertex will likely be a better solution. Does not
+    // require translation - client can take care of this. Will work independent
+    // of whether the configuration is for intermediate tasks or not. Has the
+    // overhead of localizing multiple files per job - i.e. the client would
+    // need to write these files to hdfs, add them as local resources per
+    // vertex. A solution like this may be more practical once it's possible to
+    // submit configuration parameters to the AM and effectively tasks via RPC.
+
+    jobConf.set(MRJobConfig.VERTEX_NAME, processorContext.getTaskVertexName());
+
+    if (LOG.isDebugEnabled() && userPayload != null) {
+      Iterator<Entry<String, String>> iter = jobConf.iterator();
+      String taskIdStr = taskAttemptId.getTaskID().toString();
+      while (iter.hasNext()) {
+        Entry<String, String> confEntry = iter.next();
+        LOG.debug("TaskConf Entry"
+            + ", taskId=" + taskIdStr
+            + ", key=" + confEntry.getKey()
+            + ", value=" + confEntry.getValue());
+      }
+    }
+
+    configureMRTask();
   }
 
+  private void configureMRTask()
+      throws IOException, InterruptedException {
+
+    Credentials credentials = UserGroupInformation.getCurrentUser()
+        .getCredentials();
+    jobConf.setCredentials(credentials);
+    // TODO Can this be avoided all together. Have the MRTezOutputCommitter use
+    // the Tez parameter.
+    // TODO This could be fetched from the env if YARN is setting it for all
+    // Containers.
+    // Set it in conf, so as to be able to be used the the OutputCommitter.
+
+    jobConf.setClass(MRConfig.TASK_LOCAL_OUTPUT_CLASS, YarnOutputFiles.class,
+        MapOutputFile.class); // MR
+
+    // Not needed. This is probably being set via the source/consumer meta
+    Token<JobTokenIdentifier> jobToken = TokenCache.getJobToken(credentials);
+    if (jobToken != null) {
+      // Will MR ever run without a job token.
+      SecretKey sk = JobTokenSecretManager.createSecretKey(jobToken
+          .getPassword());
+      this.jobTokenSecret = sk;
+    } else {
+      LOG.warn("No job token set");
+    }
+
+    configureLocalDirs();
+
+    if (jobConf.get(TezJobConfig.DAG_CREDENTIALS_BINARY) != null) {
+      jobConf.set(MRJobConfig.MAPREDUCE_JOB_CREDENTIALS_BINARY,
+          jobConf.get(TezJobConfig.DAG_CREDENTIALS_BINARY));
+    }
+
+    // Set up the DistributedCache related configs
+    setupDistributedCacheConfig(jobConf);
+  }
+
+  private void configureLocalDirs() throws IOException {
+    // TODO NEWTEZ Is most of this functionality required ?
+    jobConf.setStrings(TezJobConfig.LOCAL_DIRS, processorContext.getWorkDirs());
+    jobConf.set(TezJobConfig.TASK_LOCAL_RESOURCE_DIR, System.getenv(Environment.PWD.name()));
+
+    jobConf.setStrings(MRConfig.LOCAL_DIR, processorContext.getWorkDirs());
+
+    LocalDirAllocator lDirAlloc = new LocalDirAllocator(TezJobConfig.LOCAL_DIRS);
+    Path workDir = null;
+    // First, try to find the JOB_LOCAL_DIR on this host.
+    try {
+      workDir = lDirAlloc.getLocalPathToRead("work", jobConf);
+    } catch (DiskErrorException e) {
+      // DiskErrorException means dir not found. If not found, it will
+      // be created below.
+    }
+    if (workDir == null) {
+      // JOB_LOCAL_DIR doesn't exist on this host -- Create it.
+      workDir = lDirAlloc.getLocalPathForWrite("work", jobConf);
+      FileSystem lfs = FileSystem.getLocal(jobConf).getRaw();
+      boolean madeDir = false;
+      try {
+        madeDir = lfs.mkdirs(workDir);
+      } catch (FileAlreadyExistsException e) {
+        // Since all tasks will be running in their own JVM, the race condition
+        // exists where multiple tasks could be trying to create this directory
+        // at the same time. If this task loses the race, it's okay because
+        // the directory already exists.
+        madeDir = true;
+        workDir = lDirAlloc.getLocalPathToRead("work", jobConf);
+      }
+      if (!madeDir) {
+          throw new IOException("Mkdirs failed to create "
+              + workDir.toString());
+      }
+    }
+    // TODO NEWTEZ Is this required ?
+    jobConf.set(TezJobConfig.JOB_LOCAL_DIR, workDir.toString());
+    jobConf.set(MRJobConfig.JOB_LOCAL_DIR, workDir.toString());
+  }
+
+  /**
+   * Set up the DistributedCache related configs to make
+   * {@link DistributedCache#getLocalCacheFiles(Configuration)} and
+   * {@link DistributedCache#getLocalCacheArchives(Configuration)} working.
+   *
+   * @param job
+   * @throws IOException
+   */
+  private static void setupDistributedCacheConfig(final JobConf job)
+      throws IOException {
+
+    String localWorkDir = (job.get(TezJobConfig.TASK_LOCAL_RESOURCE_DIR));
+    // ^ ^ all symlinks are created in the current work-dir
+
+    // Update the configuration object with localized archives.
+    URI[] cacheArchives = DistributedCache.getCacheArchives(job);
+    if (cacheArchives != null) {
+      List<String> localArchives = new ArrayList<String>();
+      for (int i = 0; i < cacheArchives.length; ++i) {
+        URI u = cacheArchives[i];
+        Path p = new Path(u);
+        Path name = new Path((null == u.getFragment()) ? p.getName()
+            : u.getFragment());
+        String linkName = name.toUri().getPath();
+        localArchives.add(new Path(localWorkDir, linkName).toUri().getPath());
+      }
+      if (!localArchives.isEmpty()) {
+        job.set(MRJobConfig.CACHE_LOCALARCHIVES, StringUtils
+            .arrayToString(localArchives.toArray(new String[localArchives
+                .size()])));
+      }
+    }
+
+    // Update the configuration object with localized files.
+    URI[] cacheFiles = DistributedCache.getCacheFiles(job);
+    if (cacheFiles != null) {
+      List<String> localFiles = new ArrayList<String>();
+      for (int i = 0; i < cacheFiles.length; ++i) {
+        URI u = cacheFiles[i];
+        Path p = new Path(u);
+        Path name = new Path((null == u.getFragment()) ? p.getName()
+            : u.getFragment());
+        String linkName = name.toUri().getPath();
+        localFiles.add(new Path(localWorkDir, linkName).toUri().getPath());
+      }
+      if (!localFiles.isEmpty()) {
+        job.set(MRJobConfig.CACHE_LOCALFILES, StringUtils
+            .arrayToString(localFiles.toArray(new String[localFiles.size()])));
+      }
+    }
+  }
+
+
   private void initResourceCalculatorPlugin() {
     Class<? extends ResourceCalculatorProcessTree> clazz =
         this.jobConf.getClass(MRConfig.RESOURCE_CALCULATOR_PROCESS_TREE,
@@ -170,85 +337,28 @@ public abstract class MRTask extends RunningTaskContext {
     }
   }
 
-  public TezTaskUmbilicalProtocol getUmbilical() {
-    return ((TezTaskReporterImpl)reporter).getUmbilical();
+  public TezProcessorContext getUmbilical() {
+    return this.processorContext;
   }
 
-  public void initTask(JobConf job, TezDAGID dagId,
-      MRTaskReporter mrReporter,
-      boolean useNewApi) throws IOException,
+  public void initTask() throws IOException,
                                 InterruptedException {
-    this.jobConf = job;
-    this.jobContext = new JobContextImpl(job, dagId, mrReporter);
+    this.mrReporter = new MRTaskReporter(processorContext);
+    this.useNewApi = jobConf.getUseNewMapper();
+    TezDAGID dagId = IDConverter.fromMRTaskAttemptId(taskAttemptId).getTaskID()
+        .getVertexID().getDAGId();
+
+    this.jobContext = new JobContextImpl(jobConf, dagId, mrReporter);
     this.taskAttemptContext =
-        new TaskAttemptContextImpl(job, taskAttemptId, mrReporter);
-    this.mrReporter = mrReporter;
+        new TaskAttemptContextImpl(jobConf, taskAttemptId, mrReporter);
 
     if (getState() == State.UNASSIGNED) {
       setState(State.RUNNING);
     }
 
-    boolean useCombiner = false;
-    combineProcessor = null;
-    if (useNewApi) {
-      try {
-        useCombiner = (taskAttemptContext.getCombinerClass() != null);
-      } catch (ClassNotFoundException e) {
-        throw new IOException("Could not find combiner class", e);
-      }
-    } else {
-      useCombiner = (job.getCombinerClass() != null);
-    }
-    if (useCombiner) {
-      combineProcessor = new MRCombiner(this);
-      combineProcessor.initialize(job, getTaskReporter());
-    } else {
-    }
-
     localizeConfiguration(jobConf);
   }
 
-  public void initPartitioner(JobConf job) throws IOException,
-      InterruptedException {
-    partitioner = new MRPartitioner(this);
-    ((MRPartitioner) partitioner).initialize(job, getTaskReporter());
-  }
-
-  public void initCommitter(JobConf job, boolean useNewApi,
-      boolean useNullCommitter) throws IOException, InterruptedException {
-    if (useNullCommitter) {
-      setCommitter(new TezNullOutputCommitter());
-      return;
-    }
-    if (useNewApi) {
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("using new api for output committer");
-      }
-      OutputFormat<?, ?> outputFormat = null;
-      try {
-        outputFormat = ReflectionUtils.newInstance(
-            taskAttemptContext.getOutputFormatClass(), job);
-      } catch (ClassNotFoundException cnfe) {
-        throw new IOException("Unknown OutputFormat", cnfe);
-      }
-      setCommitter(outputFormat.getOutputCommitter(taskAttemptContext));
-    } else {
-      setCommitter(job.getOutputCommitter());
-    }
-
-    Path outputPath = FileOutputFormat.getOutputPath(job);
-    if (outputPath != null) {
-      if ((getCommitter() instanceof FileOutputCommitter)) {
-        FileOutputFormat.setWorkOutputPath(job,
-            ((FileOutputCommitter) getCommitter())
-                .getTaskAttemptPath(taskAttemptContext));
-      } else {
-        FileOutputFormat.setWorkOutputPath(job, outputPath);
-      }
-    }
-    getCommitter().setupTask(taskAttemptContext);
-  }
-
   public MRTaskReporter getMRReporter() {
     return mrReporter;
   }
@@ -273,23 +383,6 @@ public abstract class MRTask extends RunningTaskContext {
 
   public TezCounters getCounters() { return counters; }
 
-  /**
-   * Return current phase of the task.
-   * needs to be synchronized as communication thread sends the phase every second
-   * @return the curent phase of the task
-   */
-  public synchronized TezTaskStatus.Phase getPhase(){
-    return status.getPhase();
-  }
-
-  /**
-   * Set current phase of the task.
-   * @param phase task phase
-   */
-  protected synchronized void setPhase(TezTaskStatus.Phase phase){
-    status.setPhase(phase);
-  }
-
   public void setConf(JobConf jobConf) {
     this.jobConf = jobConf;
   }
@@ -327,33 +420,6 @@ public abstract class MRTask extends RunningTaskContext {
 
   public void waitBeforeCompletion(MRTaskReporter reporter) throws IOException,
       InterruptedException {
-    TezTaskUmbilicalProtocol umbilical = getUmbilical();
-    int retries = MAX_RETRIES;
-    boolean readyToProceed = false;
-    while (!readyToProceed) {
-      try {
-        ProceedToCompletionResponse response =
-            umbilical.proceedToCompletion(taskAttemptId);
-        if (LOG.isDebugEnabled()) {
-          LOG.debug("Got readyToProceed: " + response);
-        }
-        if (response.shouldDie()) {
-          throw new IOException("Task was asked to die by the AM");
-          // TODO EVENTUALLY Figure out a good way for a graceful exit, instead
-          // of an exit via an Exception. This isn' necessarily an error.
-        }
-        readyToProceed = response.readyToProceed();
-      } catch (IOException ie) {
-        LOG.warn("Failure waiting for exit signal: " +
-            StringUtils.stringifyException(ie));
-        if (--retries == 0) {
-          throw ie;
-        }
-      }
-      synchronized(this) {
-        wait(1000l); // Check if ready to exit every second.
-      }
-    }
   }
 
   public void outputReady(MRTaskReporter reporter, OutputContext outputContext)
@@ -362,61 +428,31 @@ public abstract class MRTask extends RunningTaskContext {
     LOG.info("Task: " + taskAttemptId + " reporting outputReady");
     updateCounters();
     statusUpdate();
-
-    TezTaskUmbilicalProtocol umbilical = getUmbilical();
-    int retries = MAX_RETRIES;
-    while (true) {
-      try {
-        umbilical.outputReady(taskAttemptId, outputContext);
-        LOG.info("Task '" + taskAttemptId + "' reported outputReady.");
-        return;
-      } catch (IOException ie) {
-        LOG.warn("Failure signalling outputReady: " +
-            StringUtils.stringifyException(ie));
-        if (--retries == 0) {
-          throw ie;
-        }
-      }
-    }
   }
 
-  public void done(
-      OutputContext outputContext,
-      MRTaskReporter reporter
-      ) throws IOException, InterruptedException {
+  public void done(LogicalOutput output) throws IOException, InterruptedException {
     updateCounters();
-    if (outputContext != null) {
-      LOG.info("Task: "
-          + taskAttemptId
-          + " is done."
-          + " And is in the process of sending output-context with shuffle port: "
-          + outputContext.getShufflePort());
-      outputReady(reporter, outputContext);
-      waitBeforeCompletion(reporter);
-    }
 
     LOG.info("Task:" + taskAttemptId + " is done."
         + " And is in the process of committing");
-    TezTaskUmbilicalProtocol umbilical = getUmbilical();
+    // TODO change this to use the new context
     // TODO TEZ Interaciton between Commit and OutputReady. Merge ?
-    if (isCommitRequired()) {
-      //wait for commit approval and commit
-      // TODO EVENTUALLY - Commit is not required for map tasks. skip a couple of RPCs before exiting.
-      commit(umbilical, reporter, committer);
+    if (output instanceof SimpleOutput) {
+      SimpleOutput sOut = (SimpleOutput)output;
+      if (sOut.isCommitRequired()) {
+        //wait for commit approval and commit
+        // TODO EVENTUALLY - Commit is not required for map tasks.
+        // skip a couple of RPCs before exiting.
+        commit(sOut);
+      }
     }
     taskDone.set(true);
-    reporter.stopCommunicationThread();
     // Make sure we send at least one set of counter increments. It's
     // ok to call updateCounters() in this thread after comm thread stopped.
     updateCounters();
     sendLastUpdate();
     //signal the tasktracker that we are done
-    sendDone(umbilical);
-  }
-
-
-  private boolean isCommitRequired() throws IOException {
-    return committer.needsTaskCommit(taskAttemptContext);
+    //sendDone(umbilical);
   }
 
   /**
@@ -425,27 +461,7 @@ public abstract class MRTask extends RunningTaskContext {
    * @throws IOException
    */
   public void statusUpdate() throws IOException, InterruptedException {
-    int retries = MAX_RETRIES;
-    /* broken code due to engine re-factor
-    while (true) {
-      try {
-        if (!getUmbilical().statusUpdate(taskAttemptId, status)) {
-          LOG.warn("Parent died.  Exiting " + taskAttemptId);
-          System.exit(66);
-        }
-        status.clearStatus();
-        return;
-      } catch (InterruptedException ie) {
-        Thread.currentThread().interrupt(); // interrupt ourself
-      } catch (IOException ie) {
-        LOG.warn("Failure sending status update: " +
-            StringUtils.stringifyException(ie));
-        if (--retries == 0) {
-          throw ie;
-        }
-      }
-    }
-     */
+    // TODO call progress update here if not being called within Map/Reduce
   }
 
   /**
@@ -453,39 +469,27 @@ public abstract class MRTask extends RunningTaskContext {
    */
   private void sendLastUpdate()
       throws IOException, InterruptedException {
-    status.setOutputSize(-1l);
-    // send a final status report
-    status.statusUpdate(
-        getProgress().get(), getProgress().toString(), counters);
     statusUpdate();
   }
 
-  private void commit(TezTaskUmbilicalProtocol umbilical,
-      MRTaskReporter reporter,
-      org.apache.hadoop.mapreduce.OutputCommitter committer
-      ) throws IOException {
-    int retries = MAX_RETRIES;
+  private void commit(SimpleOutput output) throws IOException {
+    int retries = 3;
     while (true) {
+      // This will loop till the AM asks for the task to be killed. As
+      // against, the AM sending a signal to the task to kill itself
+      // gracefully.
       try {
-        while (!umbilical.canCommit(taskAttemptId)) {
-          // This will loop till the AM asks for the task to be killed. As
-          // against, the AM sending a signal to the task to kill itself
-          // gracefully.
-          try {
-            Thread.sleep(1000);
-          } catch(InterruptedException ie) {
-            //ignore
-          }
-          reporter.setProgressFlag();
+        if (processorContext.canCommit()) {
+          break;
         }
-        break;
+        Thread.sleep(1000);
+      } catch(InterruptedException ie) {
+        //ignore
       } catch (IOException ie) {
-        LOG.warn("Failure asking whether task can commit: " +
-            StringUtils.stringifyException(ie));
+        LOG.warn("Failure sending canCommit: "
+            + StringUtils.stringifyException(ie));
         if (--retries == 0) {
-          //if it couldn't query successfully then delete the output
-          discardOutput(taskAttemptContext);
-          System.exit(68);
+          throw ie;
         }
       }
     }
@@ -493,21 +497,21 @@ public abstract class MRTask extends RunningTaskContext {
     // task can Commit now
     try {
       LOG.info("Task " + taskAttemptId + " is allowed to commit now");
-      committer.commitTask(taskAttemptContext);
+      output.commit();
       return;
     } catch (IOException iee) {
       LOG.warn("Failure committing: " +
           StringUtils.stringifyException(iee));
       //if it couldn't commit a successfully then delete the output
-      discardOutput(taskAttemptContext);
+      discardOutput(output);
       throw iee;
     }
   }
 
   private
-  void discardOutput(TaskAttemptContext taskContext) {
+  void discardOutput(SimpleOutput output) {
     try {
-      committer.abortTask(taskContext);
+      output.abort();
     } catch (IOException ioe)  {
       LOG.warn("Failure cleaning up: " +
                StringUtils.stringifyException(ioe));
@@ -515,25 +519,6 @@ public abstract class MRTask extends RunningTaskContext {
   }
 
 
-  private void sendDone(TezTaskUmbilicalProtocol umbilical) throws IOException {
-    int retries = MAX_RETRIES;
-    /* broken code due to engine re-factor
-    while (true) {
-      try {
-        umbilical.done(taskAttemptId);
-        LOG.info("Task '" + taskAttemptId + "' done.");
-        return;
-      } catch (IOException ie) {
-        LOG.warn("Failure signalling completion: " +
-                 StringUtils.stringifyException(ie));
-        if (--retries == 0) {
-          throw ie;
-        }
-      }
-    }
-    */
-  }
-
   public void updateCounters() {
     // TODO Auto-generated method stub
     // TODO TEZAM Implement.
@@ -617,7 +602,7 @@ public abstract class MRTask extends RunningTaskContext {
   createReduceContext(org.apache.hadoop.mapreduce.Reducer
                         <INKEY,INVALUE,OUTKEY,OUTVALUE> reducer,
                       Configuration job,
-                      TezTaskAttemptID taskId,
+                      TaskAttemptID taskId,
                       final TezRawKeyValueIterator rIter,
                       org.apache.hadoop.mapreduce.Counter inputKeyCounter,
                       org.apache.hadoop.mapreduce.Counter inputValueCounter,
@@ -659,7 +644,7 @@ public abstract class MRTask extends RunningTaskContext {
     reduceContext =
       new ReduceContextImpl<INKEY, INVALUE, OUTKEY, OUTVALUE>(
           job,
-          IDConverter.toMRTaskAttemptId(taskId),
+          taskId,
           r,
           inputKeyCounter,
           inputValueCounter,
@@ -682,11 +667,9 @@ public abstract class MRTask extends RunningTaskContext {
     return reducerContext;
   }
 
-  public void taskCleanup(TezTaskUmbilicalProtocol umbilical)
+  public void taskCleanup()
       throws IOException, InterruptedException {
     // set phase for this task
-    setPhase(TezTaskStatus.Phase.CLEANUP);
-    getProgress().setStatus("cleanup");
     statusUpdate();
     LOG.info("Runnning cleanup for the task");
     // do the cleanup
@@ -695,106 +678,30 @@ public abstract class MRTask extends RunningTaskContext {
 
   public void localizeConfiguration(JobConf jobConf)
       throws IOException, InterruptedException {
-    jobConf.set(JobContext.TASK_ID, IDConverter
-        .toMRTaskAttemptId(taskAttemptId).toString());
-    jobConf.set(JobContext.TASK_ATTEMPT_ID,
-        IDConverter.toMRTaskAttemptId(taskAttemptId).toString());
+    jobConf.set(JobContext.TASK_ID, taskAttemptId.getTaskID().toString());
+    jobConf.set(JobContext.TASK_ATTEMPT_ID, taskAttemptId.toString());
     jobConf.setInt(JobContext.TASK_PARTITION,
         taskAttemptId.getTaskID().getId());
-    jobConf.set(JobContext.ID, taskAttemptId.getTaskID().getVertexID().getDAGId().toString());
+    jobConf.set(JobContext.ID, taskAttemptId.getJobID().toString());
   }
 
   public abstract TezCounter getOutputRecordsCounter();
 
   public abstract TezCounter getInputRecordsCounter();
 
-  public TezCounter getFileOutputBytesCounter() {
-    return reporter.getCounter(FileOutputFormatCounter.BYTES_WRITTEN);
-  }
-
   public org.apache.hadoop.mapreduce.TaskAttemptContext getTaskAttemptContext() {
     return taskAttemptContext;
   }
 
-  public TezCounter getFileInputBytesCounter() {
-    return reporter.getCounter(FileInputFormatCounter.BYTES_READ);
-  }
-
   public JobContext getJobContext() {
     return jobContext;
   }
 
-  public TezTaskAttemptID getTaskAttemptId() {
+  public TaskAttemptID getTaskAttemptId() {
     return taskAttemptId;
   }
 
-  public TezEngineTaskContext getTezEngineTaskContext() {
-    return tezEngineTaskContext;
-  }
-
-  protected FutureTask<Void> initInputAsync(Input input) {
-    FutureTask<Void> initInputFuture = new FutureTask<Void>(
-        new InitInputCallable(input));
-    new Thread(initInputFuture, "InitInputThread").start();
-    return initInputFuture;
-  }
-
-  protected FutureTask<Void> initOutputAsync(Output output) {
-    FutureTask<Void> initOutputFuture = new FutureTask<Void>(
-        new InitOutputCallable(output));
-    new Thread(initOutputFuture, "InitOutputThread").start();
-    return initOutputFuture;
-  }
-
-  protected class InitInputCallable implements Callable<Void> {
-    Input input;
-    InitInputCallable(Input input) {
-      this.input = input;
-    }
-    @Override
-    public Void call() throws IOException, InterruptedException {
-      input.initialize(jobConf, getTaskReporter());
-      LOG.info("Input initialized");
-      return null;
-    }
-  }
-
-  protected class InitOutputCallable implements Callable<Void> {
-    Output output;
-    InitOutputCallable(Output output) {
-      this.output = output;
-    }
-    @Override
-    public Void call() throws IOException, InterruptedException {
-      output.initialize(jobConf, getTaskReporter());
-      LOG.info("Output initialized");
-      return null;
-    }
-  }
-
-  private void waitForIOInitialization(FutureTask<Void> future)
-      throws InterruptedException, IOException {
-    try {
-      future.get();
-    } catch (ExecutionException e) {
-      if (e.getCause() instanceof InterruptedException) {
-        throw (InterruptedException) e.getCause();
-      } else if (e.getCause() instanceof IOException) {
-        throw (IOException) e.getCause();
-      } else {
-        throw new RuntimeException("UnknownException from I/O initialization",
-            e.getCause());
-      }
-    }
-  }
-
-  protected void waitForInputInitialization(FutureTask<Void> future)
-      throws InterruptedException, IOException {
-    waitForIOInitialization(future);
-  }
-
-  protected void waitForOutputInitialization(FutureTask<Void> future)
-      throws InterruptedException, IOException {
-    waitForIOInitialization(future);
+  public TezProcessorContext getTezEngineTaskContext() {
+    return processorContext;
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/3d609458/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/MRTaskReporter.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/MRTaskReporter.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/MRTaskReporter.java
index 1639bf2..d32b4c0 100644
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/MRTaskReporter.java
+++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/MRTaskReporter.java
@@ -23,89 +23,100 @@ import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.mapred.Counters;
 import org.apache.hadoop.mapred.InputSplit;
 import org.apache.hadoop.mapred.Reporter;
-import org.apache.tez.common.TezTaskReporter;
 import org.apache.tez.common.counters.TezCounter;
+import org.apache.tez.engine.newapi.TezInputContext;
+import org.apache.tez.engine.newapi.TezOutputContext;
+import org.apache.tez.engine.newapi.TezProcessorContext;
+import org.apache.tez.engine.newapi.TezTaskContext;
 import org.apache.tez.mapreduce.hadoop.mapred.MRCounters;
+import org.apache.tez.mapreduce.hadoop.mapred.MRReporter;
 
 @InterfaceAudience.Private
-@InterfaceStability.Unstable 
-public class MRTaskReporter 
+@InterfaceStability.Unstable
+public class MRTaskReporter
     extends org.apache.hadoop.mapreduce.StatusReporter
     implements Reporter {
 
-  private final TezTaskReporterImpl reporter;
-  
+  private final TezTaskContext context;
+  private final boolean isProcessorContext;
+  private final Reporter reporter;
+
   private InputSplit split = null;
 
-  public MRTaskReporter(TezTaskReporter reporter) {
-    this.reporter =  (TezTaskReporterImpl)reporter;
+  public MRTaskReporter(TezProcessorContext context) {
+    this.context = context;
+    this.reporter = new MRReporter(context);
+    this.isProcessorContext = true;
   }
 
-  // getters and setters for flag
-  void setProgressFlag() {
-    reporter.setProgressFlag();
+  public MRTaskReporter(TezOutputContext context) {
+    this.context = context;
+    this.reporter = new MRReporter(context);
+    this.isProcessorContext = false;
+  }
+  
+  public MRTaskReporter(TezInputContext context) {
+    this.context= context;
+    this.reporter = new MRReporter(context);
+    this.isProcessorContext = false;
   }
-  boolean resetProgressFlag() {
-    return reporter.resetProgressFlag();
+
+  public void setProgress(float progress) {
+    if (isProcessorContext) {
+      ((TezProcessorContext)context).setProgress(progress);
+    } else {
+      // TODO FIXME NEWTEZ - will simpleoutput's reporter use this api?
+    }
   }
+
   public void setStatus(String status) {
     reporter.setStatus(status);
   }
-  public void setProgress(float progress) {
-    reporter.setProgress(progress);
-  }
-  
+
   public float getProgress() {
     return reporter.getProgress();
   };
-  
+
   public void progress() {
     reporter.progress();
   }
-  
+
   public Counters.Counter getCounter(String group, String name) {
-    TezCounter counter = reporter.getCounter(group, name);
+    TezCounter counter = context.getCounters().findCounter(group, name);
     MRCounters.MRCounter mrCounter = null;
     if (counter != null) {
       mrCounter = new MRCounters.MRCounter(counter);
     }
     return mrCounter;
   }
-  
+
   public Counters.Counter getCounter(Enum<?> name) {
-    TezCounter counter = reporter.getCounter(name);
+    TezCounter counter = context.getCounters().findCounter(name);
     MRCounters.MRCounter mrCounter = null;
     if (counter != null) {
       mrCounter = new MRCounters.MRCounter(counter);
     }
     return mrCounter;
   }
-  
+
   public void incrCounter(Enum<?> key, long amount) {
     reporter.incrCounter(key, amount);
   }
-  
+
   public void incrCounter(String group, String counter, long amount) {
     reporter.incrCounter(group, counter, amount);
   }
-  
+
   public void setInputSplit(InputSplit split) {
     this.split = split;
   }
-  
+
   public InputSplit getInputSplit() throws UnsupportedOperationException {
     if (split == null) {
       throw new UnsupportedOperationException("Input only available on map");
     } else {
       return split;
     }
-  }  
-  
-  public void startCommunicationThread() {
-    reporter.startCommunicationThread();
-  }
-  
-  public void stopCommunicationThread() throws InterruptedException {
-    reporter.stopCommunicationThread();
   }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/3d609458/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/TezTaskReporterImpl.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/TezTaskReporterImpl.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/TezTaskReporterImpl.java
deleted file mode 100644
index 6323fc9..0000000
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/TezTaskReporterImpl.java
+++ /dev/null
@@ -1,279 +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.processor;
-
-import java.io.IOException;
-import java.util.concurrent.atomic.AtomicBoolean;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.ipc.ProtocolSignature;
-import org.apache.hadoop.util.Progress;
-import org.apache.hadoop.util.ReflectionUtils;
-import org.apache.hadoop.util.StringUtils;
-import org.apache.tez.common.TezTaskUmbilicalProtocol;
-import org.apache.tez.common.counters.TezCounter;
-import org.apache.tez.dag.records.TezTaskAttemptID;
-import org.apache.tez.engine.records.TezTaskDependencyCompletionEventsUpdate;
-
-@InterfaceAudience.Private
-@InterfaceStability.Unstable 
-class TezTaskReporterImpl 
-    implements org.apache.tez.common.TezTaskReporter, Runnable {
-
-  private static final Log LOG = LogFactory.getLog(TezTaskReporterImpl.class);
-  
-  private final MRTask mrTask;
-  private final TezTaskUmbilicalProtocol umbilical;
-  private final Progress taskProgress;
-  
-  private Thread pingThread = null;
-  private boolean done = true;
-  private Object lock = new Object();
-
-  /**
-   * flag that indicates whether progress update needs to be sent to parent.
-   * If true, it has been set. If false, it has been reset. 
-   * Using AtomicBoolean since we need an atomic read & reset method. 
-   */  
-  private AtomicBoolean progressFlag = new AtomicBoolean(false);
-  
-  TezTaskReporterImpl(MRTask mrTask, TezTaskUmbilicalProtocol umbilical) {
-    this.mrTask = mrTask;
-    this.umbilical = umbilical;
-    this.taskProgress = mrTask.getProgress();
-  }
-
-  // getters and setters for flag
-  void setProgressFlag() {
-    progressFlag.set(true);
-  }
-  
-  boolean resetProgressFlag() {
-    return progressFlag.getAndSet(false);
-  }
-  
-  public void setStatus(String status) {
-    // FIXME - BADLY
-    if (true) {
-      return;
-    }
-    taskProgress.setStatus(
-        MRTask.normalizeStatus(status, this.mrTask.jobConf));
-    // indicate that progress update needs to be sent
-    setProgressFlag();
-  }
-  
-  public void setProgress(float progress) {
-    // set current phase progress.
-    // This method assumes that task has phases.
-    taskProgress.phase().set(progress);
-    // indicate that progress update needs to be sent
-    setProgressFlag();
-  }
-  
-  public float getProgress() {
-    return taskProgress.getProgress();
-  };
-  
-  public void progress() {
-    // indicate that progress update needs to be sent
-    setProgressFlag();
-  }
-  
-  public TezCounter getCounter(String group, String name) {
-    return this.mrTask.counters == null ? 
-        null : 
-        this.mrTask.counters.findCounter(group, name);
-  }
-  
-  public TezCounter getCounter(Enum<?> name) {
-    return this.mrTask.counters == null ? 
-        null : 
-        this.mrTask.counters.findCounter(name);
-  }
-  
-  public void incrCounter(Enum<?> key, long amount) {
-    if (this.mrTask.counters != null) {
-      this.mrTask.counters.findCounter(key).increment(amount);
-    }
-    setProgressFlag();
-  }
-  
-  public void incrCounter(String group, String counter, long amount) {
-    if (this.mrTask.counters != null) {
-      this.mrTask.counters.findCounter(group, counter).increment(amount);
-    }
-    setProgressFlag();
-  }
-  
-  /** 
-   * The communication thread handles communication with the parent (Task Tracker). 
-   * It sends progress updates if progress has been made or if the task needs to 
-   * let the parent know that it's alive. It also pings the parent to see if it's alive. 
-   */
-  public void run() {
-    final int MAX_RETRIES = 3;
-    int remainingRetries = MAX_RETRIES;
-    // get current flag value and reset it as well
-    boolean sendProgress = resetProgressFlag();
-    while (!this.mrTask.taskDone.get()) {
-      synchronized (lock) {
-        done = false;
-      }
-      try {
-        boolean taskFound = true; // whether TT knows about this task
-        // sleep for a bit
-        synchronized(lock) {
-          if (this.mrTask.taskDone.get()) {
-            break;
-          }
-          lock.wait(MRTask.PROGRESS_INTERVAL);
-        }
-        if (this.mrTask.taskDone.get()) {
-          break;
-        }
-
-        if (sendProgress) {
-          // we need to send progress update
-          this.mrTask.updateCounters();
-          this.mrTask.getStatus().statusUpdate(
-              taskProgress.get(),
-              taskProgress.toString(), 
-              this.mrTask.counters);
-
-          // broken code now due to tez engine changes
-          taskFound = false;
-          /*
-          taskFound = 
-              umbilical.statusUpdate(
-                  this.mrTask.getTaskAttemptId(), this.mrTask.getStatus());
-           */
-          this.mrTask.getStatus().clearStatus();
-        }
-        else {
-          // send ping 
-          taskFound = false;
-          // broken code now due to tez engine changes
-          //umbilical.ping(this.mrTask.getTaskAttemptId());
-        }
-
-        // if Task Tracker is not aware of our task ID (probably because it died and 
-        // came back up), kill ourselves
-        if (!taskFound) {
-          MRTask.LOG.warn("Parent died.  Exiting " + this.mrTask.getTaskAttemptId());
-          resetDoneFlag();
-          System.exit(66);
-        }
-
-        sendProgress = resetProgressFlag(); 
-        remainingRetries = MAX_RETRIES;
-      } 
-      catch (Throwable t) {
-        MRTask.LOG.info("Communication exception: " + StringUtils.stringifyException(t));
-        remainingRetries -=1;
-        if (remainingRetries == 0) {
-          ReflectionUtils.logThreadInfo(MRTask.LOG, "Communication exception", 0);
-          MRTask.LOG.warn("Last retry, killing " + this.mrTask.getTaskAttemptId());
-          resetDoneFlag();
-          System.exit(65);
-        }
-      }
-    }
-    //Notify that we are done with the work
-    resetDoneFlag();
-  }
-  void resetDoneFlag() {
-    synchronized (lock) {
-      done = true;
-      lock.notify();
-    }
-  }
-  public void startCommunicationThread() {
-    if (pingThread == null) {
-      pingThread = new Thread(this, "communication thread");
-      pingThread.setDaemon(true);
-      pingThread.start();
-    }
-  }
-  public void stopCommunicationThread() throws InterruptedException {
-    if (pingThread != null) {
-      // Intent of the lock is to not send an interupt in the middle of an
-      // umbilical.ping or umbilical.statusUpdate
-      synchronized(lock) {
-      //Interrupt if sleeping. Otherwise wait for the RPC call to return.
-        lock.notify(); 
-      }
-
-      synchronized (lock) { 
-        while (!done) {
-          lock.wait();
-        }
-      }
-      pingThread.interrupt();
-      pingThread.join();
-    }
-  }
-
-  @Override
-  public TezTaskDependencyCompletionEventsUpdate getDependentTasksCompletionEvents(
-      int fromEventIdx, int maxEventsToFetch,
-      TezTaskAttemptID reduce) {
-    return umbilical.getDependentTasksCompletionEvents(
-        fromEventIdx, maxEventsToFetch, reduce);
-  }
-
-  @Override
-  public void reportFatalError(TezTaskAttemptID taskAttemptId,
-      Throwable throwable, String logMsg) {
-    LOG.fatal(logMsg);
-    Throwable tCause = throwable.getCause();
-    String cause = tCause == null 
-                   ? StringUtils.stringifyException(throwable)
-                   : StringUtils.stringifyException(tCause);
-/*
-                   try {
-      umbilical.fatalError(mrTask.getTaskAttemptId(), cause);
-    } catch (IOException ioe) {
-      LOG.fatal("Failed to contact the tasktracker", ioe);
-      System.exit(-1);
-    }
-    */
-  }
-
-  public TezTaskUmbilicalProtocol getUmbilical() {
-    return umbilical;
-  }
-
-  @Override
-  public long getProtocolVersion(String protocol, long clientVersion)
-      throws IOException {
-    // TODO TEZAM3
-    return 1;
-  }
-
-  @Override
-  public ProtocolSignature getProtocolSignature(String protocol,
-      long clientVersion, int clientMethodsHash) throws IOException {
-    return ProtocolSignature.getProtocolSignature(this, protocol,
-        clientVersion, clientMethodsHash);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/3d609458/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/map/MapProcessor.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/map/MapProcessor.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/map/MapProcessor.java
index 67794e7..80a2337 100644
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/map/MapProcessor.java
+++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/map/MapProcessor.java
@@ -18,11 +18,11 @@
 package org.apache.tez.mapreduce.processor.map;
 
 import java.io.IOException;
-import java.util.concurrent.FutureTask;
+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.mapred.JobConf;
 import org.apache.hadoop.mapred.JobContext;
 import org.apache.hadoop.mapred.MapRunnable;
@@ -31,162 +31,134 @@ import org.apache.hadoop.mapred.RecordReader;
 import org.apache.hadoop.mapred.Reporter;
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
 import org.apache.hadoop.mapreduce.lib.map.WrappedMapper;
-import org.apache.hadoop.util.Progress;
 import org.apache.hadoop.util.ReflectionUtils;
-import org.apache.tez.common.TezEngineTaskContext;
-import org.apache.tez.common.TezTaskStatus;
 import org.apache.tez.common.counters.TaskCounter;
 import org.apache.tez.common.counters.TezCounter;
-import org.apache.tez.engine.api.Input;
-import org.apache.tez.engine.api.Master;
-import org.apache.tez.engine.api.Output;
-import org.apache.tez.engine.api.Processor;
-import org.apache.tez.engine.common.sort.SortingOutput;
-import org.apache.tez.mapreduce.hadoop.IDConverter;
-import org.apache.tez.mapreduce.hadoop.mapred.TaskAttemptContextImpl;
+import org.apache.tez.dag.api.TezException;
+import org.apache.tez.engine.lib.output.OnFileSortedOutput;
+import org.apache.tez.engine.newapi.Event;
+import org.apache.tez.engine.newapi.KVReader;
+import org.apache.tez.engine.newapi.KVWriter;
+import org.apache.tez.engine.newapi.LogicalIOProcessor;
+import org.apache.tez.engine.newapi.LogicalInput;
+import org.apache.tez.engine.newapi.LogicalOutput;
+import org.apache.tez.engine.newapi.TezProcessorContext;
+import org.apache.tez.mapreduce.hadoop.mapreduce.MapContextImpl;
 import org.apache.tez.mapreduce.input.SimpleInput;
+import org.apache.tez.mapreduce.input.SimpleInputLegacy;
 import org.apache.tez.mapreduce.output.SimpleOutput;
 import org.apache.tez.mapreduce.processor.MRTask;
 import org.apache.tez.mapreduce.processor.MRTaskReporter;
 
 @SuppressWarnings({ "unchecked", "rawtypes" })
-public class MapProcessor extends MRTask implements Processor {
+public class MapProcessor extends MRTask implements LogicalIOProcessor {
 
   private static final Log LOG = LogFactory.getLog(MapProcessor.class);
 
-  private Progress mapPhase;
+  public MapProcessor(){
+    super(true);
+  }
 
-  public MapProcessor(TezEngineTaskContext context) throws IOException {
-    super(context);
+  @Override
+  public void initialize(TezProcessorContext processorContext)
+      throws IOException {
+    try {
+      super.initialize(processorContext);
+    } catch (InterruptedException e) {
+      throw new IOException(e);
+    }
   }
-  
 
 
   @Override
-  public void initialize(Configuration conf, Master master) throws IOException,
-  InterruptedException {
-    super.initialize(conf, master);
+  public void handleEvents(List<Event> processorEvents) {
+    // TODO Auto-generated method stub
+
+  }
+
+  public void close() throws IOException {
+    // TODO Auto-generated method stub
+
   }
 
   @Override
-  public void process(
-      final Input[] ins,
-      final Output[] outs)
-          throws IOException, InterruptedException {
-
-    MRTaskReporter reporter = new MRTaskReporter(getTaskReporter());
-    boolean useNewApi = jobConf.getUseNewMapper();
-    initTask(jobConf, taskAttemptId.getTaskID().getVertexID().getDAGId(),
-        reporter, useNewApi);
-
-    if (ins.length != 1
-        || outs.length != 1) {
+  public void run(Map<String, LogicalInput> inputs,
+      Map<String, LogicalOutput> outputs) throws Exception {
+
+    LOG.info("Running map: " + processorContext.getUniqueIdentifier());
+
+    initTask();
+
+    if (inputs.size() != 1
+        || outputs.size() != 1) {
       throw new IOException("Cannot handle multiple inputs or outputs"
-          + ", inputCount=" + ins.length
-          + ", outputCount=" + outs.length);
+          + ", inputCount=" + inputs.size()
+          + ", outputCount=" + outputs.size());
     }
-    Input in = ins[0];
-    Output out = outs[0];
-    
+    LogicalInput in = inputs.values().iterator().next();
+    LogicalOutput out = outputs.values().iterator().next();
+
     // Sanity check
-    if (!(in instanceof SimpleInput)) {
-      throw new IOException("Unknown input! - " + in.getClass());
-    }
-    SimpleInput input = (SimpleInput)in;
-    input.setTask(this);
-
-    if (out instanceof SimpleOutput) {
-      initCommitter(jobConf, useNewApi, false);
-      ((SimpleOutput)out).setTask(this);
-    } else if (out instanceof SortingOutput) {
-      initCommitter(jobConf, useNewApi, true);
-      initPartitioner(jobConf);
-      ((SortingOutput)out).setTask(this);
+    if (!(in instanceof SimpleInputLegacy)) {
+      throw new IOException(new TezException(
+          "Only Simple Input supported. Input: " + in.getClass()));
     }
+    SimpleInputLegacy input = (SimpleInputLegacy)in;
 
-    // If there are no reducers then there won't be any sort. Hence the map 
-    // phase will govern the entire attempt's progress.
-    if (tezEngineTaskContext.getOutputSpecList().get(0).getNumOutputs() == 0) {
-      mapPhase = getProgress().addPhase("map");
+    KVWriter kvWriter = null;
+    if (!(out instanceof OnFileSortedOutput)) {
+      kvWriter = ((SimpleOutput)out).getWriter();
     } else {
-      // If there are reducers then the entire attempt's progress will be 
-      // split between the map phase (67%) and the sort phase (33%).
-      mapPhase = getProgress().addPhase("map");
+      kvWriter = ((OnFileSortedOutput)out).getWriter();
     }
 
-
     if (useNewApi) {
-      runNewMapper(jobConf, reporter, input, out, getTaskReporter());
+      runNewMapper(jobConf, mrReporter, input, kvWriter);
     } else {
-      runOldMapper(jobConf, reporter, input, out, getTaskReporter());
+      runOldMapper(jobConf, mrReporter, input, kvWriter);
     }
 
-    done(out.getOutputContext(), reporter);
+    done(out);
   }
 
-  public void close() throws IOException, InterruptedException {
-    // TODO Auto-generated method stub
-
-  }
-  
   void runOldMapper(
       final JobConf job,
       final MRTaskReporter reporter,
-      final SimpleInput input,
-      final Output output,
-      final Master master
+      final SimpleInputLegacy input,
+      final KVWriter output
       ) throws IOException, InterruptedException {
-    
+
     // Initialize input in-line since it sets parameters which may be used by the processor.
     // Done only for SimpleInput.
-    input.initialize(job, master);
-    FutureTask<Void> initOutputFuture = initOutputAsync(output);
-    
+    // TODO use new method in SimpleInput to get required info
+    //input.initialize(job, master);
+
     RecordReader in = new OldRecordReader(input);
-        
-    int numReduceTasks = tezEngineTaskContext.getOutputSpecList().get(0)
-        .getNumOutputs();
-    LOG.info("numReduceTasks: " + numReduceTasks);
 
     OutputCollector collector = new OldOutputCollector(output);
 
     MapRunnable runner =
         (MapRunnable)ReflectionUtils.newInstance(job.getMapRunnerClass(), job);
 
-    // Wait for output to be initialized before starting processing.
-    LOG.info("Waiting on output initialization");
-    waitForOutputInitialization(initOutputFuture);
-
-    try {
-      runner.run(in, collector, (Reporter)reporter);
-      mapPhase.complete();
-      // start the sort phase only if there are reducers
-      if (numReduceTasks > 0) {
-        setPhase(TezTaskStatus.Phase.SORT);
-      }
-      this.statusUpdate();
-    } finally {
-      //close
-      in.close();                               // close input
-      output.close();
-    }
+    runner.run(in, collector, (Reporter)reporter);
+    // start the sort phase only if there are reducers
+    this.statusUpdate();
   }
 
   private void runNewMapper(final JobConf job,
       MRTaskReporter reporter,
-      final SimpleInput in,
-      Output out,
-      final Master master
+      final SimpleInputLegacy in,
+      KVWriter out
       ) throws IOException, InterruptedException {
 
     // Initialize input in-line since it sets parameters which may be used by the processor.
     // Done only for SimpleInput.
-    in.initialize(job, master);
-    FutureTask<Void> initOutputFuture = initOutputAsync(out);
-    
+    // TODO use new method in SimpleInput to get required info
+    //in.initialize(job, master);
+
     // make a task context so we can get the classes
     org.apache.hadoop.mapreduce.TaskAttemptContext taskContext =
-        new TaskAttemptContextImpl(job, taskAttemptId, reporter);
+        getTaskAttemptContext();
 
     // make a mapper
     org.apache.hadoop.mapreduce.Mapper mapper;
@@ -197,37 +169,27 @@ public class MapProcessor extends MRTask implements Processor {
       throw new IOException(cnfe);
     }
 
-    if (!(in instanceof SimpleInput)) {
-      throw new IOException("Unknown input! - " + in.getClass());
-    }
-
     org.apache.hadoop.mapreduce.RecordReader input =
         new NewRecordReader(in);
 
-    org.apache.hadoop.mapreduce.RecordWriter output = 
+    org.apache.hadoop.mapreduce.RecordWriter output =
         new NewOutputCollector(out);
 
-    // Wait for output to be initialized before starting processing.
-    LOG.info("Waiting on output initialization");
-    waitForOutputInitialization(initOutputFuture);
-
     org.apache.hadoop.mapreduce.InputSplit split = in.getNewInputSplit();
-    
-    org.apache.hadoop.mapreduce.MapContext 
-    mapContext = 
-    new org.apache.tez.mapreduce.hadoop.mapreduce.MapContextImpl(
-        job, IDConverter.toMRTaskAttemptId(taskAttemptId), 
-        input, output, 
-        getCommitter(), 
-        reporter, split);
-
-    org.apache.hadoop.mapreduce.Mapper.Context mapperContext = 
+
+    org.apache.hadoop.mapreduce.MapContext
+    mapContext =
+    new MapContextImpl(
+        job, taskAttemptId,
+        input, output,
+        getCommitter(),
+        processorContext, split);
+
+    org.apache.hadoop.mapreduce.Mapper.Context mapperContext =
         new WrappedMapper().getMapContext(mapContext);
 
     input.initialize(split, mapperContext);
     mapper.run(mapperContext);
-    mapPhase.complete();
-    setPhase(TezTaskStatus.Phase.SORT);
     this.statusUpdate();
     input.close();
     output.close(mapperContext);
@@ -236,34 +198,36 @@ public class MapProcessor extends MRTask implements Processor {
   private static class NewRecordReader extends
       org.apache.hadoop.mapreduce.RecordReader {
     private final SimpleInput in;
+    private KVReader reader;
 
-    private NewRecordReader(SimpleInput in) {
+    private NewRecordReader(SimpleInput in) throws IOException {
       this.in = in;
+      this.reader = in.getReader();
     }
 
     @Override
     public void initialize(org.apache.hadoop.mapreduce.InputSplit split,
         TaskAttemptContext context) throws IOException,
         InterruptedException {
-      in.initializeNewRecordReader(split, context);
+      //in.initializeNewRecordReader(split, context);
     }
 
     @Override
     public boolean nextKeyValue() throws IOException,
         InterruptedException {
-      return in.hasNext();
+      return reader.next();
     }
 
     @Override
     public Object getCurrentKey() throws IOException,
         InterruptedException {
-      return in.getNextKey();
+      return reader.getCurrentKV().getKey();
     }
 
     @Override
     public Object getCurrentValue() throws IOException,
         InterruptedException {
-      return in.getNextValues().iterator().next();
+      return reader.getCurrentKV().getValues().iterator().next();
     }
 
     @Override
@@ -273,26 +237,27 @@ public class MapProcessor extends MRTask implements Processor {
 
     @Override
     public void close() throws IOException {
-      in.close();
     }
   }
 
   private static class OldRecordReader implements RecordReader {
-    private final SimpleInput simpleInput;
+    private final SimpleInputLegacy simpleInput;
 
-    private OldRecordReader(SimpleInput simpleInput) {
+    private OldRecordReader(SimpleInputLegacy simpleInput) {
       this.simpleInput = simpleInput;
     }
 
     @Override
     public boolean next(Object key, Object value) throws IOException {
-      simpleInput.setKey(key);
-      simpleInput.setValue(value);
-      try {
-        return simpleInput.hasNext();
-      } catch (InterruptedException ie) {
-        throw new IOException(ie);
-      }
+      // TODO broken
+//      simpleInput.setKey(key);
+//      simpleInput.setValue(value);
+//      try {
+//        return simpleInput.hasNext();
+//      } catch (InterruptedException ie) {
+//        throw new IOException(ie);
+//      }
+      return simpleInput.getOldRecordReader().next(key, value);
     }
 
     @Override
@@ -312,7 +277,6 @@ public class MapProcessor extends MRTask implements Processor {
 
     @Override
     public void close() throws IOException {
-      simpleInput.close();
     }
 
     @Override
@@ -325,29 +289,24 @@ public class MapProcessor extends MRTask implements Processor {
     }
   }
 
-  private static class OldOutputCollector 
+  private static class OldOutputCollector
   implements OutputCollector {
-    private final Output output;
-    
-    OldOutputCollector(Output output) {
+    private final KVWriter output;
+
+    OldOutputCollector(KVWriter output) {
       this.output = output;
     }
 
     public void collect(Object key, Object value) throws IOException {
-      try {
         output.write(key, value);
-      } catch (InterruptedException ie) {
-        Thread.currentThread().interrupt();
-        throw new IOException("interrupt exception", ie);
-      }
     }
   }
 
   private class NewOutputCollector
     extends org.apache.hadoop.mapreduce.RecordWriter {
-    private final Output out;
+    private final KVWriter out;
 
-    NewOutputCollector(Output out) throws IOException {
+    NewOutputCollector(KVWriter out) throws IOException {
       this.out = out;
     }
 
@@ -359,25 +318,24 @@ public class MapProcessor extends MRTask implements Processor {
     @Override
     public void close(TaskAttemptContext context
                       ) throws IOException, InterruptedException {
-      out.close();
     }
   }
 
   @Override
-  public void localizeConfiguration(JobConf jobConf) 
+  public void localizeConfiguration(JobConf jobConf)
       throws IOException, InterruptedException {
     super.localizeConfiguration(jobConf);
     jobConf.setBoolean(JobContext.TASK_ISMAP, true);
   }
-  
+
   @Override
   public TezCounter getOutputRecordsCounter() {
-    return reporter.getCounter(TaskCounter.MAP_OUTPUT_RECORDS);
+    return processorContext.getCounters().findCounter(TaskCounter.MAP_OUTPUT_RECORDS);
   }
 
   @Override
   public TezCounter getInputRecordsCounter() {
-    return reporter.getCounter(TaskCounter.MAP_INPUT_RECORDS);
-
+    return processorContext.getCounters().findCounter(TaskCounter.MAP_INPUT_RECORDS);
   }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/3d609458/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/reduce/ReduceProcessor.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/reduce/ReduceProcessor.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/reduce/ReduceProcessor.java
index 4fb1876..88ab754 100644
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/reduce/ReduceProcessor.java
+++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/reduce/ReduceProcessor.java
@@ -18,12 +18,12 @@
 package org.apache.tez.mapreduce.processor.reduce;
 
 import java.io.IOException;
+import java.util.Iterator;
 import java.util.List;
-import java.util.concurrent.FutureTask;
+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.io.DataInputBuffer;
 import org.apache.hadoop.io.RawComparator;
 import org.apache.hadoop.mapred.Counters.Counter;
@@ -35,295 +35,251 @@ import org.apache.hadoop.mapreduce.TaskAttemptContext;
 import org.apache.hadoop.util.Progress;
 import org.apache.hadoop.util.Progressable;
 import org.apache.hadoop.util.ReflectionUtils;
-import org.apache.tez.common.InputSpec;
-import org.apache.tez.common.TezEngineTaskContext;
-import org.apache.tez.common.TezTaskStatus;
-import org.apache.tez.common.TezTaskUmbilicalProtocol;
 import org.apache.tez.common.counters.TaskCounter;
 import org.apache.tez.common.counters.TezCounter;
-import org.apache.tez.engine.api.Input;
-import org.apache.tez.engine.api.Master;
-import org.apache.tez.engine.api.Output;
-import org.apache.tez.engine.api.Processor;
 import org.apache.tez.engine.common.ConfigUtils;
-import org.apache.tez.engine.common.sort.SortingOutput;
 import org.apache.tez.engine.common.sort.impl.TezRawKeyValueIterator;
-import org.apache.tez.engine.lib.oldinput.OldShuffledMergedInput;
-import org.apache.tez.mapreduce.hadoop.mapred.TaskAttemptContextImpl;
-import org.apache.tez.mapreduce.input.SimpleInput;
+import org.apache.tez.engine.lib.output.OnFileSortedOutput;
+import org.apache.tez.engine.newapi.Event;
+import org.apache.tez.engine.newapi.KVReader;
+import org.apache.tez.engine.newapi.KVWriter;
+import org.apache.tez.engine.newapi.LogicalIOProcessor;
+import org.apache.tez.engine.newapi.LogicalInput;
+import org.apache.tez.engine.newapi.LogicalOutput;
+import org.apache.tez.engine.newapi.TezProcessorContext;
+import org.apache.tez.mapreduce.input.ShuffledMergedInputLegacy;
 import org.apache.tez.mapreduce.output.SimpleOutput;
 import org.apache.tez.mapreduce.processor.MRTask;
 import org.apache.tez.mapreduce.processor.MRTaskReporter;
 
-import com.google.common.base.Preconditions;
 
 @SuppressWarnings({ "unchecked", "rawtypes" })
 public class ReduceProcessor
 extends MRTask
-implements Processor {
+implements LogicalIOProcessor {
 
   private static final Log LOG = LogFactory.getLog(ReduceProcessor.class);
-  
-  private Progress sortPhase;
-  private Progress reducePhase;
 
   private Counter reduceInputKeyCounter;
   private Counter reduceInputValueCounter;
 
-  public ReduceProcessor(TezEngineTaskContext context) {
-    super(context);
-    TezEngineTaskContext tezEngineContext = (TezEngineTaskContext) context;
-    Preconditions.checkNotNull(tezEngineContext.getInputSpecList(),
-        "InputSpecList should not be null");
+  public ReduceProcessor() {
+    super(false);
   }
-  
+
   @Override
-  public void initialize(Configuration conf, Master master) throws IOException,
-      InterruptedException {
-    super.initialize(conf, master);
+  public void initialize(TezProcessorContext processorContext)
+      throws IOException {
+    try {
+      super.initialize(processorContext);
+    } catch (InterruptedException e) {
+      throw new IOException(e);
+    }
   }
 
+
   @Override
-  public void process(Input[] ins, Output[] outs)
-      throws IOException, InterruptedException {
-    MRTaskReporter reporter = new MRTaskReporter(getTaskReporter());
-    boolean useNewApi = jobConf.getUseNewReducer();
-    initTask(jobConf, taskAttemptId.getTaskID().getVertexID().getDAGId(),
-        reporter, useNewApi);
+  public void handleEvents(List<Event> processorEvents) {
+    // TODO Auto-generated method stub
 
-    if (outs.length <= 0 || outs.length > 1) {
-      throw new IOException("Invalid number of outputs"
-          + ", outputCount=" + outs.length);
-    }
-    
-    if (ins.length <= 0) {
-      throw new IOException("Invalid number of inputs"
-          + ", inputCount=" + ins.length);
-    }
+  }
 
-    Input in = ins[0];
-    Output out = outs[0];
+  public void close() throws IOException {
+    // TODO Auto-generated method stub
 
-    List<InputSpec> inputs = getTezEngineTaskContext().getInputSpecList();
+  }
 
-    if (in instanceof SimpleInput) {
-      ((SimpleInput)in).setTask(this);
-    } else if (in instanceof OldShuffledMergedInput) {
-      ((OldShuffledMergedInput)in).setTask(this);
-    }
-    
-    if(ins.length > 1) {
-      if (!(in instanceof OldShuffledMergedInput)) {
-        throw new IOException(
-            "Only ShuffledMergedInput can support multiple inputs"
-                + ". inputCount=" + ins.length);
-      }      
-      if(ins.length != inputs.size()) {
-        throw new IOException(
-            "Mismatch in input size passed and context inputspec size. Passed: "
-                + ins.length + " From contex:" + inputs.size());
-      }
-      // initialize and merge the remaining
-      OldShuffledMergedInput s0 = ((OldShuffledMergedInput)in);
-      for(int i=1; i<ins.length; ++i) {
-        Input inputi = ins[i];
-        if (!(inputi instanceof OldShuffledMergedInput)) {
-          throw new IOException(
-              "Only ShuffledMergedInput can support multiple inputs"
-                  + ". inputCount=" + ins.length);
-        }      
-        OldShuffledMergedInput si = ((OldShuffledMergedInput)inputi);
-        s0.mergeWith(si);
-      }
+  @Override
+  public void run(Map<String, LogicalInput> inputs,
+      Map<String, LogicalOutput> outputs) throws Exception {
+
+    LOG.info("Running reduce: " + processorContext.getUniqueIdentifier());
+
+    initTask();
+
+    if (outputs.size() <= 0 || outputs.size() > 1) {
+      throw new IOException("Invalid number of outputs"
+          + ", outputCount=" + outputs.size());
     }
-    
-    if (out instanceof SimpleOutput) {
-      initCommitter(jobConf, useNewApi, false);
-      ((SimpleOutput)out).setTask(this);
-    } else if (out instanceof SortingOutput) {
-      initCommitter(jobConf, useNewApi, true);
-      initPartitioner(jobConf);
-      ((SortingOutput)out).setTask(this);
+
+    if (inputs.size() <= 0 || inputs.size() > 1) {
+      throw new IOException("Invalid number of inputs"
+          + ", inputCount=" + inputs.size());
     }
 
+    LogicalInput in = inputs.values().iterator().next();
+    LogicalOutput out = outputs.values().iterator().next();
+
     this.statusUpdate();
-    
+
     Class keyClass = ConfigUtils.getIntermediateInputKeyClass(jobConf);
     Class valueClass = ConfigUtils.getIntermediateInputValueClass(jobConf);
     LOG.info("Using keyClass: " + keyClass);
     LOG.info("Using valueClass: " + valueClass);
-    RawComparator comparator = 
+    RawComparator comparator =
         ConfigUtils.getInputKeySecondaryGroupingComparator(jobConf);
     LOG.info("Using comparator: " + comparator);
 
-    reduceInputKeyCounter = 
-        reporter.getCounter(TaskCounter.REDUCE_INPUT_GROUPS);
-    reduceInputValueCounter = 
-        reporter.getCounter(TaskCounter.REDUCE_INPUT_RECORDS);
-        
+    reduceInputKeyCounter =
+        mrReporter.getCounter(TaskCounter.REDUCE_INPUT_GROUPS);
+    reduceInputValueCounter =
+        mrReporter.getCounter(TaskCounter.REDUCE_INPUT_RECORDS);
+
     // Sanity check
-    if (!(in instanceof OldShuffledMergedInput)) {
+    if (!(in instanceof ShuffledMergedInputLegacy)) {
+      throw new IOException("Illegal input to reduce: " + in.getClass());
+    }
+    ShuffledMergedInputLegacy shuffleInput = (ShuffledMergedInputLegacy)in;
+    KVReader kvReader = shuffleInput.getReader();
+
+    KVWriter kvWriter = null;
+    if((out instanceof SimpleOutput)) {
+      kvWriter = ((SimpleOutput) out).getWriter();
+    } else if ((out instanceof OnFileSortedOutput)) {
+      kvWriter = ((OnFileSortedOutput) out).getWriter();
+    } else {
       throw new IOException("Illegal input to reduce: " + in.getClass());
     }
-    OldShuffledMergedInput shuffleInput = (OldShuffledMergedInput)in;
 
     if (useNewApi) {
       try {
         runNewReducer(
-            jobConf, 
-            (TezTaskUmbilicalProtocol)getUmbilical(), reporter, 
-            shuffleInput, comparator,  keyClass, valueClass, 
-            out);
+            jobConf,
+            mrReporter,
+            shuffleInput, comparator,  keyClass, valueClass,
+            kvWriter);
       } catch (ClassNotFoundException cnfe) {
         throw new IOException(cnfe);
       }
     } else {
       runOldReducer(
-          jobConf, (TezTaskUmbilicalProtocol)getUmbilical(), reporter, 
-          shuffleInput, comparator, keyClass, valueClass, out);
+          jobConf, mrReporter,
+          kvReader, comparator, keyClass, valueClass, kvWriter);
     }
-    
-    done(out.getOutputContext(), reporter);
-  }
 
-  public void close() throws IOException, InterruptedException {
-    // TODO Auto-generated method stub
-    
+    done(out);
   }
 
   void runOldReducer(JobConf job,
-      TezTaskUmbilicalProtocol umbilical,
       final MRTaskReporter reporter,
-      OldShuffledMergedInput input,
+      KVReader input,
       RawComparator comparator,
       Class keyClass,
       Class valueClass,
-      final Output output) throws IOException, InterruptedException {
-    
-    FutureTask<Void> initInputFuture = initInputAsync(input);
-    FutureTask<Void> initOutputFuture = initOutputAsync(output);
+      final KVWriter output) throws IOException, InterruptedException {
 
-    Reducer reducer = 
+    Reducer reducer =
         ReflectionUtils.newInstance(job.getReducerClass(), job);
 
     // make output collector
 
-    OutputCollector collector = 
+    OutputCollector collector =
         new OutputCollector() {
       public void collect(Object key, Object value)
           throws IOException {
-        try {
-          output.write(key, value);
-        } catch (InterruptedException ie) {
-          throw new IOException(ie);
-        }
+        output.write(key, value);
       }
     };
 
-    // Wait for input/output to be initialized before starting processing.
-    LOG.info("Waiting on input initialization");
-    waitForInputInitialization(initInputFuture);
-
-    LOG.info("Waiting on output initialization");
-    waitForOutputInitialization(initOutputFuture);
-
     // apply reduce function
     try {
-      ReduceValuesIterator values = 
+      ReduceValuesIterator values =
           new ReduceValuesIterator(
-              input, 
-              job.getOutputValueGroupingComparator(), keyClass, valueClass, 
-              job, reporter, reduceInputValueCounter, reducePhase);
-      
+              input, reporter, reduceInputValueCounter);
+
       values.informReduceProgress();
       while (values.more()) {
         reduceInputKeyCounter.increment(1);
         reducer.reduce(values.getKey(), values, collector, reporter);
-        values.nextKey();
         values.informReduceProgress();
       }
 
       //Clean up: repeated in catch block below
       reducer.close();
-      output.close();
       //End of clean up.
     } catch (IOException ioe) {
       try {
         reducer.close();
-      } catch (IOException ignored) {}
-
-      try {
-        output.close();
-      } catch (IOException ignored) {}
+      } catch (IOException ignored) {
+      }
 
       throw ioe;
     }
   }
-  
-  private static class ReduceValuesIterator<KEY,VALUE> 
-  extends org.apache.tez.engine.common.task.impl.ValuesIterator<KEY,VALUE> {
+
+  private static class ReduceValuesIterator<KEY,VALUE>
+  implements Iterator<VALUE> {
     private Counter reduceInputValueCounter;
-    private Progress reducePhase;
-
-    public ReduceValuesIterator (OldShuffledMergedInput in,
-        RawComparator<KEY> comparator, 
-        Class<KEY> keyClass,
-        Class<VALUE> valClass,
-        Configuration conf, Progressable reporter,
-        Counter reduceInputValueCounter,
-        Progress reducePhase)
+    private KVReader in;
+    private Progressable reporter;
+    private Object currentKey;
+    private Iterator<Object> currentValues;
+
+    public ReduceValuesIterator (KVReader in,
+        Progressable reporter,
+        Counter reduceInputValueCounter)
             throws IOException {
-      super(in.getIterator(), comparator, keyClass, valClass, conf, reporter);
       this.reduceInputValueCounter = reduceInputValueCounter;
-      this.reducePhase = reducePhase;
+      this.in = in;
+      this.reporter = reporter;
     }
 
-    @Override
-    public VALUE next() {
-      reduceInputValueCounter.increment(1);
-      return moveToNext();
+    public boolean more() throws IOException {
+      boolean more = in.next();
+      if(more) {
+        currentKey = in.getCurrentKV().getKey();
+        currentValues = in.getCurrentKV().getValues().iterator();
+      } else {
+        currentKey = null;
+        currentValues = null;
+      }
+      return more;
     }
 
-    protected VALUE moveToNext() {
-      return super.next();
+    public KEY getKey() throws IOException {
+      return (KEY) currentKey;
     }
 
     public void informReduceProgress() {
-      reducePhase.set(super.in.getProgress().getProgress()); // update progress
       reporter.progress();
     }
+
+    @Override
+    public boolean hasNext() {
+      return currentValues.hasNext();
+    }
+
+    @Override
+    public VALUE next() {
+      reduceInputValueCounter.increment(1);
+      return (VALUE) currentValues.next();
+    }
+
+    @Override
+    public void remove() {
+      throw new UnsupportedOperationException();
+    }
+
   }
 
   void runNewReducer(JobConf job,
-      final TezTaskUmbilicalProtocol umbilical,
       final MRTaskReporter reporter,
-      OldShuffledMergedInput input,
+      ShuffledMergedInputLegacy input,
       RawComparator comparator,
       Class keyClass,
       Class valueClass,
-      final Output out
-      ) throws IOException,InterruptedException, 
+      final KVWriter out
+      ) throws IOException,InterruptedException,
       ClassNotFoundException {
-    
-    FutureTask<Void> initInputFuture = initInputAsync(input);
-    FutureTask<Void> initOutputFuture = initOutputAsync(out);
-    
+
     // make a task context so we can get the classes
-    org.apache.hadoop.mapreduce.TaskAttemptContext taskContext =
-        new TaskAttemptContextImpl(job, taskAttemptId, reporter);
-    
+    org.apache.hadoop.mapreduce.TaskAttemptContext taskContext = getTaskAttemptContext();
+
     // make a reducer
     org.apache.hadoop.mapreduce.Reducer reducer =
         (org.apache.hadoop.mapreduce.Reducer)
         ReflectionUtils.newInstance(taskContext.getReducerClass(), job);
 
-    // Wait for input/output to be initialized before starting processing.
-    LOG.info("Waiting on input initialization");
-    waitForInputInitialization(initInputFuture);
-
-    LOG.info("Waiting on output initialization");
-    waitForOutputInitialization(initOutputFuture);
-
     // wrap value iterator to report progress.
     final TezRawKeyValueIterator rawIter = input.getIterator();
     TezRawKeyValueIterator rIter = new TezRawKeyValueIterator() {
@@ -346,7 +302,7 @@ implements Processor {
       }
     };
 
-    org.apache.hadoop.mapreduce.RecordWriter trackedRW = 
+    org.apache.hadoop.mapreduce.RecordWriter trackedRW =
         new org.apache.hadoop.mapreduce.RecordWriter() {
 
       @Override
@@ -358,28 +314,27 @@ implements Processor {
       @Override
       public void close(TaskAttemptContext context) throws IOException,
       InterruptedException {
-        out.close();
       }
     };
 
-    org.apache.hadoop.mapreduce.Reducer.Context reducerContext = 
+    org.apache.hadoop.mapreduce.Reducer.Context reducerContext =
         createReduceContext(
             reducer, job, taskAttemptId,
-            rIter, reduceInputKeyCounter, 
-            reduceInputValueCounter, 
+            rIter, reduceInputKeyCounter,
+            reduceInputValueCounter,
             trackedRW,
             committer,
             reporter, comparator, keyClass,
             valueClass);
-    
-    
-    
+
+
+
     reducer.run(reducerContext);
     trackedRW.close(reducerContext);
   }
 
   @Override
-  public void localizeConfiguration(JobConf jobConf) 
+  public void localizeConfiguration(JobConf jobConf)
       throws IOException, InterruptedException {
     super.localizeConfiguration(jobConf);
     jobConf.setBoolean(JobContext.TASK_ISMAP, false);
@@ -387,21 +342,12 @@ implements Processor {
 
   @Override
   public TezCounter getOutputRecordsCounter() {
-    return reporter.getCounter(TaskCounter.REDUCE_OUTPUT_RECORDS);
+    return processorContext.getCounters().findCounter(TaskCounter.REDUCE_OUTPUT_RECORDS);
   }
 
   @Override
   public TezCounter getInputRecordsCounter() {
-    return reporter.getCounter(TaskCounter.REDUCE_INPUT_GROUPS);
+    return processorContext.getCounters().findCounter(TaskCounter.REDUCE_INPUT_GROUPS);
   }
 
-  @Override
-  protected void waitForInputInitialization(FutureTask<Void> future)
-      throws InterruptedException, IOException {
-    super.waitForInputInitialization(future);
-    sortPhase = getProgress().addPhase("sort");
-    reducePhase = getProgress().addPhase("reduce");
-    sortPhase.complete(); // sort is complete
-    setPhase(TezTaskStatus.Phase.REDUCE);
-  }
 }


[10/50] [abbrv] git commit: TEZ-444. Rename *.new* packages back to what they should be, remove dead code from the old packages - mapreduce module (part of TEZ-398). (sseth)

Posted by ss...@apache.org.
TEZ-444.  Rename *.new* packages back to what they should be, remove
dead code from the old packages - mapreduce module (part of TEZ-398).
(sseth)


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

Branch: refs/heads/master
Commit: 3d6094588cc43f8140b2360bd9032b2ed7249e1a
Parents: 5eb0c86
Author: Siddharth Seth <ss...@apache.org>
Authored: Mon Sep 23 10:43:39 2013 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Mon Sep 23 10:43:39 2013 -0700

----------------------------------------------------------------------
 .../apache/hadoop/mapred/YarnTezDagChild.java   |   4 +-
 .../tez/mapreduce/examples/MRRSleepJob.java     |   6 +-
 .../mapreduce/examples/OrderedWordCount.java    |   6 +-
 .../apache/tez/mapreduce/TestMRRJobsDAGApi.java |   6 +-
 .../apache/hadoop/mapred/LocalJobRunnerTez.java |   8 +-
 .../tez/mapreduce/combine/MRCombiner.java       | 440 +++++------
 .../apache/tez/mapreduce/hadoop/MRHelpers.java  |   4 +-
 .../hadoop/MultiStageMRConfToTezTranslator.java |   4 +-
 .../tez/mapreduce/hadoop/mapred/MRReporter.java |  76 ++
 .../hadoop/mapred/TaskAttemptContextImpl.java   |  21 +-
 .../hadoop/mapreduce/MapContextImpl.java        |   6 +-
 .../hadoop/mapreduce/ReduceContextImpl.java     | 359 ---------
 .../mapreduce/TaskAttemptContextImpl.java       | 102 +--
 .../mapreduce/TaskInputOutputContextImpl.java   |   6 +-
 .../mapreduce/hadoop/newmapred/MRReporter.java  |  76 --
 .../newmapred/TaskAttemptContextImpl.java       |  93 ---
 .../hadoop/newmapreduce/MapContextImpl.java     |  84 ---
 .../newmapreduce/TaskAttemptContextImpl.java    |  90 ---
 .../TaskInputOutputContextImpl.java             |  95 ---
 .../input/ShuffledMergedInputLegacy.java        |  29 +
 .../apache/tez/mapreduce/input/SimpleInput.java | 359 +++++----
 .../tez/mapreduce/input/SimpleInputLegacy.java  |  36 +
 .../tez/mapreduce/newcombine/MRCombiner.java    | 242 ------
 .../newinput/ShuffledMergedInputLegacy.java     |  29 -
 .../tez/mapreduce/newinput/SimpleInput.java     | 438 -----------
 .../mapreduce/newinput/SimpleInputLegacy.java   |  36 -
 .../tez/mapreduce/newoutput/SimpleOutput.java   | 326 ---------
 .../mapreduce/newpartition/MRPartitioner.java   |  88 ---
 .../FileSystemStatisticsUpdater.java            |  84 ---
 .../mapreduce/newprocessor/GcTimeUpdater.java   |  71 --
 .../tez/mapreduce/newprocessor/MRTask.java      | 731 -------------------
 .../mapreduce/newprocessor/MRTaskReporter.java  | 122 ----
 .../newprocessor/map/MapProcessor.java          | 341 ---------
 .../newprocessor/reduce/ReduceProcessor.java    | 353 ---------
 .../tez/mapreduce/output/SimpleOutput.java      | 355 ++++++---
 .../tez/mapreduce/partition/MRPartitioner.java  | 105 ++-
 .../apache/tez/mapreduce/processor/MRTask.java  | 625 +++++++---------
 .../tez/mapreduce/processor/MRTaskReporter.java |  77 +-
 .../processor/TezTaskReporterImpl.java          | 279 -------
 .../mapreduce/processor/map/MapProcessor.java   | 268 +++----
 .../processor/reduce/ReduceProcessor.java       | 336 ++++-----
 .../tez/mapreduce/task/MRRuntimeTask.java       | 272 -------
 .../tez/mapreduce/processor/MapUtils.java       |   2 +-
 .../processor/map/TestMapProcessor.java         |   2 +-
 .../processor/reduce/TestReduceProcessor.java   |   8 +-
 .../org/apache/tez/mapreduce/YARNRunner.java    |   6 +-
 46 files changed, 1439 insertions(+), 5667 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/3d609458/tez-dag/src/main/java/org/apache/hadoop/mapred/YarnTezDagChild.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/hadoop/mapred/YarnTezDagChild.java b/tez-dag/src/main/java/org/apache/hadoop/mapred/YarnTezDagChild.java
index 2d3e4d1..31898a3 100644
--- a/tez-dag/src/main/java/org/apache/hadoop/mapred/YarnTezDagChild.java
+++ b/tez-dag/src/main/java/org/apache/hadoop/mapred/YarnTezDagChild.java
@@ -85,8 +85,8 @@ import org.apache.tez.engine.newapi.impl.TezHeartbeatRequest;
 import org.apache.tez.engine.newapi.impl.TezHeartbeatResponse;
 import org.apache.tez.engine.newapi.impl.TezUmbilical;
 import org.apache.tez.engine.newruntime.LogicalIOProcessorRuntimeTask;
-import org.apache.tez.mapreduce.newinput.SimpleInputLegacy;
-import org.apache.tez.mapreduce.newoutput.SimpleOutput;
+import org.apache.tez.mapreduce.input.SimpleInputLegacy;
+import org.apache.tez.mapreduce.output.SimpleOutput;
 
 import com.google.inject.Guice;
 import com.google.inject.Injector;

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/3d609458/tez-mapreduce-examples/src/main/java/org/apache/tez/mapreduce/examples/MRRSleepJob.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce-examples/src/main/java/org/apache/tez/mapreduce/examples/MRRSleepJob.java b/tez-mapreduce-examples/src/main/java/org/apache/tez/mapreduce/examples/MRRSleepJob.java
index 3e79949..429d458 100644
--- a/tez-mapreduce-examples/src/main/java/org/apache/tez/mapreduce/examples/MRRSleepJob.java
+++ b/tez-mapreduce-examples/src/main/java/org/apache/tez/mapreduce/examples/MRRSleepJob.java
@@ -85,9 +85,9 @@ import org.apache.tez.mapreduce.hadoop.InputSplitInfo;
 import org.apache.tez.mapreduce.hadoop.MRHelpers;
 import org.apache.tez.mapreduce.hadoop.MultiStageMRConfToTezTranslator;
 import org.apache.tez.mapreduce.hadoop.MultiStageMRConfigUtil;
-import org.apache.tez.mapreduce.newinput.ShuffledMergedInputLegacy;
-import org.apache.tez.mapreduce.newprocessor.map.MapProcessor;
-import org.apache.tez.mapreduce.newprocessor.reduce.ReduceProcessor;
+import org.apache.tez.mapreduce.input.ShuffledMergedInputLegacy;
+import org.apache.tez.mapreduce.processor.map.MapProcessor;
+import org.apache.tez.mapreduce.processor.reduce.ReduceProcessor;
 
 import com.google.common.annotations.VisibleForTesting;
 

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/3d609458/tez-mapreduce-examples/src/main/java/org/apache/tez/mapreduce/examples/OrderedWordCount.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce-examples/src/main/java/org/apache/tez/mapreduce/examples/OrderedWordCount.java b/tez-mapreduce-examples/src/main/java/org/apache/tez/mapreduce/examples/OrderedWordCount.java
index adfc8cd..07fe58a 100644
--- a/tez-mapreduce-examples/src/main/java/org/apache/tez/mapreduce/examples/OrderedWordCount.java
+++ b/tez-mapreduce-examples/src/main/java/org/apache/tez/mapreduce/examples/OrderedWordCount.java
@@ -75,9 +75,9 @@ import org.apache.tez.mapreduce.hadoop.InputSplitInfo;
 import org.apache.tez.mapreduce.hadoop.MRHelpers;
 import org.apache.tez.mapreduce.hadoop.MRJobConfig;
 import org.apache.tez.mapreduce.hadoop.MultiStageMRConfToTezTranslator;
-import org.apache.tez.mapreduce.newinput.ShuffledMergedInputLegacy;
-import org.apache.tez.mapreduce.newprocessor.map.MapProcessor;
-import org.apache.tez.mapreduce.newprocessor.reduce.ReduceProcessor;
+import org.apache.tez.mapreduce.input.ShuffledMergedInputLegacy;
+import org.apache.tez.mapreduce.processor.map.MapProcessor;
+import org.apache.tez.mapreduce.processor.reduce.ReduceProcessor;
 
 /**
  * An MRR job built on top of word count to return words sorted by

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/3d609458/tez-mapreduce-tests/src/test/java/org/apache/tez/mapreduce/TestMRRJobsDAGApi.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce-tests/src/test/java/org/apache/tez/mapreduce/TestMRRJobsDAGApi.java b/tez-mapreduce-tests/src/test/java/org/apache/tez/mapreduce/TestMRRJobsDAGApi.java
index eb20876..aca5b8e 100644
--- a/tez-mapreduce-tests/src/test/java/org/apache/tez/mapreduce/TestMRRJobsDAGApi.java
+++ b/tez-mapreduce-tests/src/test/java/org/apache/tez/mapreduce/TestMRRJobsDAGApi.java
@@ -79,9 +79,9 @@ import org.apache.tez.mapreduce.hadoop.InputSplitInfo;
 import org.apache.tez.mapreduce.hadoop.MRHelpers;
 import org.apache.tez.mapreduce.hadoop.MRJobConfig;
 import org.apache.tez.mapreduce.hadoop.MultiStageMRConfToTezTranslator;
-import org.apache.tez.mapreduce.newinput.ShuffledMergedInputLegacy;
-import org.apache.tez.mapreduce.newprocessor.map.MapProcessor;
-import org.apache.tez.mapreduce.newprocessor.reduce.ReduceProcessor;
+import org.apache.tez.mapreduce.input.ShuffledMergedInputLegacy;
+import org.apache.tez.mapreduce.processor.map.MapProcessor;
+import org.apache.tez.mapreduce.processor.reduce.ReduceProcessor;
 import org.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.BeforeClass;

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/3d609458/tez-mapreduce/src/main/java/org/apache/hadoop/mapred/LocalJobRunnerTez.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/hadoop/mapred/LocalJobRunnerTez.java b/tez-mapreduce/src/main/java/org/apache/hadoop/mapred/LocalJobRunnerTez.java
index 1362396..ac6d5dd 100644
--- a/tez-mapreduce/src/main/java/org/apache/hadoop/mapred/LocalJobRunnerTez.java
+++ b/tez-mapreduce/src/main/java/org/apache/hadoop/mapred/LocalJobRunnerTez.java
@@ -84,10 +84,10 @@ import org.apache.tez.engine.records.TezTaskDependencyCompletionEventsUpdate;
 import org.apache.tez.engine.runtime.RuntimeUtils;
 import org.apache.tez.mapreduce.hadoop.IDConverter;
 import org.apache.tez.mapreduce.hadoop.mapred.MRCounters;
-import org.apache.tez.mapreduce.newinput.SimpleInput;
-import org.apache.tez.mapreduce.newoutput.SimpleOutput;
-import org.apache.tez.mapreduce.newprocessor.map.MapProcessor;
-import org.apache.tez.mapreduce.newprocessor.reduce.ReduceProcessor;
+import org.apache.tez.mapreduce.input.SimpleInput;
+import org.apache.tez.mapreduce.output.SimpleOutput;
+import org.apache.tez.mapreduce.processor.map.MapProcessor;
+import org.apache.tez.mapreduce.processor.reduce.ReduceProcessor;
 
 import com.google.common.util.concurrent.ThreadFactoryBuilder;
 

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/3d609458/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/combine/MRCombiner.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/combine/MRCombiner.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/combine/MRCombiner.java
index dc7c53d..56e88c7 100644
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/combine/MRCombiner.java
+++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/combine/MRCombiner.java
@@ -25,324 +25,218 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.DataInputBuffer;
 import org.apache.hadoop.io.RawComparator;
-import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.OutputCollector;
 import org.apache.hadoop.mapred.RawKeyValueIterator;
 import org.apache.hadoop.mapred.Reducer;
-import org.apache.hadoop.mapred.Counters.Counter;
+import org.apache.hadoop.mapreduce.Counter;
+import org.apache.hadoop.mapreduce.RecordWriter;
+import org.apache.hadoop.mapreduce.ReduceContext;
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.TaskAttemptID;
+import org.apache.hadoop.mapreduce.TaskCounter;
+import org.apache.hadoop.mapreduce.TaskID;
+import org.apache.hadoop.mapreduce.TaskType;
 import org.apache.hadoop.mapreduce.lib.reduce.WrappedReducer;
 import org.apache.hadoop.mapreduce.task.ReduceContextImpl;
 import org.apache.hadoop.util.Progress;
-import org.apache.hadoop.util.Progressable;
 import org.apache.hadoop.util.ReflectionUtils;
-import org.apache.tez.common.TezTaskUmbilicalProtocol;
-import org.apache.tez.common.counters.TaskCounter;
-import org.apache.tez.dag.records.TezTaskAttemptID;
-import org.apache.tez.engine.api.Input;
-import org.apache.tez.engine.api.Master;
-import org.apache.tez.engine.api.Output;
-import org.apache.tez.engine.api.Processor;
+import org.apache.tez.common.TezUtils;
+import org.apache.tez.common.counters.TezCounter;
 import org.apache.tez.engine.common.ConfigUtils;
-import org.apache.tez.engine.common.combine.CombineInput;
-import org.apache.tez.engine.common.combine.CombineOutput;
+import org.apache.tez.engine.common.ValuesIterator;
+import org.apache.tez.engine.common.combine.Combiner;
+import org.apache.tez.engine.common.sort.impl.IFile.Writer;
 import org.apache.tez.engine.common.sort.impl.TezRawKeyValueIterator;
-import org.apache.tez.mapreduce.hadoop.IDConverter;
-import org.apache.tez.mapreduce.hadoop.mapred.TaskAttemptContextImpl;
-import org.apache.tez.mapreduce.processor.MRTask;
+import org.apache.tez.engine.newapi.TezInputContext;
+import org.apache.tez.engine.newapi.TezOutputContext;
+import org.apache.tez.engine.newapi.TezTaskContext;
+import org.apache.tez.mapreduce.hadoop.MRConfig;
+import org.apache.tez.mapreduce.hadoop.MRJobConfig;
+import org.apache.tez.mapreduce.hadoop.mapred.MRCounters;
 import org.apache.tez.mapreduce.processor.MRTaskReporter;
 
-public class MRCombiner implements Processor {
+@SuppressWarnings({"rawtypes", "unchecked"})
+public class MRCombiner implements Combiner {
 
   private static Log LOG = LogFactory.getLog(MRCombiner.class);
-
-  JobConf jobConf;
-  boolean useNewApi;
-
-  private final MRTask task;
-
-  private Counter combinerInputKeyCounter;
-  private Counter combinerInputValueCounter;
-  private Progress combinePhase;
-
-  public MRCombiner(MRTask task) {
-    this.task = task;
-  }
-
-  @Override
-  public void initialize(Configuration conf, Master master) throws IOException,
-      InterruptedException {
-    if (conf instanceof JobConf) {
-      jobConf = (JobConf)conf;
+  
+  private final Configuration conf;
+  private final Class<?> keyClass;
+  private final Class<?> valClass;
+  private final RawComparator<?> comparator;
+  private final boolean useNewApi;
+  
+  private final TezCounter combineInputKeyCounter;
+  private final TezCounter combineInputValueCounter;
+  
+  private final MRTaskReporter reporter;
+  private final TaskAttemptID mrTaskAttemptID;
+
+  public MRCombiner(TezTaskContext taskContext) throws IOException {
+    this.conf = TezUtils.createConfFromUserPayload(taskContext.getUserPayload());
+
+    assert(taskContext instanceof TezInputContext || taskContext instanceof TezOutputContext);
+    if (taskContext instanceof TezOutputContext) {
+      this.keyClass = ConfigUtils.getIntermediateOutputKeyClass(conf);
+      this.valClass = ConfigUtils.getIntermediateOutputValueClass(conf);
+      this.comparator = ConfigUtils.getIntermediateOutputKeyComparator(conf);
+      this.reporter = new MRTaskReporter((TezOutputContext)taskContext);
     } else {
-      jobConf = new JobConf(conf);
+      this.keyClass = ConfigUtils.getIntermediateInputKeyClass(conf);
+      this.valClass = ConfigUtils.getIntermediateInputValueClass(conf);
+      this.comparator = ConfigUtils.getIntermediateInputKeyComparator(conf);
+      this.reporter = new MRTaskReporter((TezInputContext)taskContext);
     }
-    useNewApi = jobConf.getUseNewMapper();
+
+    this.useNewApi = ConfigUtils.useNewApi(conf);
+    
+    combineInputKeyCounter = taskContext.getCounters().findCounter(TaskCounter.COMBINE_INPUT_RECORDS);
+    combineInputValueCounter = taskContext.getCounters().findCounter(TaskCounter.COMBINE_OUTPUT_RECORDS);
+    
+    boolean isMap = conf.getBoolean(MRConfig.IS_MAP_PROCESSOR,false);
+    this.mrTaskAttemptID = new TaskAttemptID(
+        new TaskID(String.valueOf(taskContext.getApplicationId()
+            .getClusterTimestamp()), taskContext.getApplicationId().getId(),
+            isMap ? TaskType.MAP : TaskType.REDUCE,
+            taskContext.getTaskIndex()), taskContext.getTaskAttemptNumber());
+    
+    LOG.info("Using combineKeyClass: " + keyClass + ", combineValueClass: " + valClass + ", combineComparator: " +comparator + ", useNewApi: " + useNewApi);
   }
 
   @Override
-  public void process(Input[] in, Output[] out) throws IOException,
-      InterruptedException {
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Running MRCombiner, usingNewAPI=" + useNewApi);
-    }
-
-    CombineInput input = (CombineInput)in[0];
-    CombineOutput output = (CombineOutput)out[0];
-
-    combinePhase  = task.getProgress().addPhase("combine");
-
-    Class<?> keyClass = ConfigUtils.getIntermediateOutputKeyClass(jobConf);
-    Class<?> valueClass = ConfigUtils.getIntermediateOutputValueClass(jobConf);
-    LOG.info("Using combineKeyClass: " + keyClass);
-    LOG.info("Using combineValueClass: " + valueClass);
-    RawComparator<?> comparator =
-        ConfigUtils.getIntermediateOutputKeyComparator(jobConf);
-    LOG.info("Using combineComparator: " + comparator);
-
-    combinerInputKeyCounter =
-        task.getMRReporter().getCounter(TaskCounter.COMBINE_INPUT_RECORDS);
-    combinerInputValueCounter =
-        task.getMRReporter().getCounter(TaskCounter.COMBINE_OUTPUT_RECORDS);
-
+  public void combine(TezRawKeyValueIterator rawIter, Writer writer)
+      throws InterruptedException, IOException {
     if (useNewApi) {
-      try {
-        runNewCombiner(this.jobConf,
-            task.getUmbilical(),
-            task.getMRReporter(),
-            input, comparator, keyClass, valueClass, output);
-      } catch (ClassNotFoundException cnfe) {
-        throw new IOException(cnfe);
-      }
+      runNewCombiner(rawIter, writer);
     } else {
-      runOldCombiner(this.jobConf,
-          task.getUmbilical(),
-          task.getMRReporter(),
-          input,
-          comparator, keyClass, valueClass,
-          output);
+      runOldCombiner(rawIter, writer);
     }
+    
   }
 
-  @SuppressWarnings({ "rawtypes", "unchecked" })
-  private void runOldCombiner(JobConf job,
-        TezTaskUmbilicalProtocol umbilical,
-        final MRTaskReporter reporter,
-        CombineInput input,
-        RawComparator comparator,
-        Class keyClass,
-        Class valueClass,
-        final Output output) throws IOException, InterruptedException {
-
-    Reducer combiner =
-        ReflectionUtils.newInstance(job.getCombinerClass(), job);
-
-    // make output collector
-
-    OutputCollector collector =
-        new OutputCollector() {
-      public void collect(Object key, Object value)
-          throws IOException {
-        try {
-          output.write(key, value);
-        } catch (InterruptedException ie) {
-          throw new IOException(ie);
-        }
+  ///////////////// Methods for old API //////////////////////
+  
+  private void runOldCombiner(final TezRawKeyValueIterator rawIter, final Writer writer) throws IOException {
+    Class<? extends Reducer> reducerClazz = (Class<? extends Reducer>) conf.getClass("mapred.combiner.class", null, Reducer.class);
+    
+    Reducer combiner = ReflectionUtils.newInstance(reducerClazz, conf);
+    
+    OutputCollector collector = new OutputCollector() {
+      @Override
+      public void collect(Object key, Object value) throws IOException {
+        writer.append(key, value);
       }
     };
-
-    // apply combiner function
-    CombinerValuesIterator values =
-        new CombinerValuesIterator(input,
-            comparator, keyClass, valueClass, job, reporter,
-            combinerInputValueCounter, combinePhase);
-
-    values.informReduceProgress();
-    while (values.more()) {
-      combinerInputKeyCounter.increment(1);
-      combiner.reduce(values.getKey(), values, collector, reporter);
-      values.nextKey();
-      values.informReduceProgress();
+    
+    CombinerValuesIterator values = new CombinerValuesIterator(rawIter, keyClass, valClass, comparator);
+    
+    while (values.moveToNext()) {
+      combiner.reduce(values.getKey(), values.getValues().iterator(), collector, reporter);
     }
   }
-
-  private static final class CombinerValuesIterator<KEY,VALUE>
-  extends org.apache.tez.engine.common.task.impl.ValuesIterator<KEY,VALUE> {
-    private Counter combineInputValueCounter;
-    private Progress combinePhase;
-
-    public CombinerValuesIterator (CombineInput in,
-        RawComparator<KEY> comparator,
-        Class<KEY> keyClass,
-        Class<VALUE> valClass,
-        Configuration conf, Progressable reporter,
-        Counter combineInputValueCounter,
-        Progress combinePhase)
-            throws IOException {
-      super(in.getIterator(), comparator, keyClass, valClass, conf, reporter);
-      this.combineInputValueCounter = combineInputValueCounter;
-      this.combinePhase = combinePhase;
-    }
-
-    @Override
-    public VALUE next() {
-      combineInputValueCounter.increment(1);
-      return moveToNext();
-    }
-
-    protected VALUE moveToNext() {
-      return super.next();
-    }
-
-    public void informReduceProgress() {
-      // FIXME implement correct progress updates for combiner TEZ-184
-      // combinePhase.set(super.in.getProgress().getProgress()); // update progress
-      reporter.progress();
+  
+  private final class CombinerValuesIterator<KEY,VALUE> extends ValuesIterator<KEY, VALUE> {
+    public CombinerValuesIterator(TezRawKeyValueIterator rawIter,
+        Class<KEY> keyClass, Class<VALUE> valClass,
+        RawComparator<KEY> comparator) throws IOException {
+      super(rawIter, comparator, keyClass, valClass, conf,
+          combineInputKeyCounter, combineInputValueCounter);
     }
   }
-
-
-  @SuppressWarnings({ "rawtypes", "unchecked" })
-  private void runNewCombiner(JobConf job,
-      final TezTaskUmbilicalProtocol umbilical,
-      final MRTaskReporter reporter,
-      CombineInput input,
-      RawComparator comparator,
-      Class keyClass,
-      Class valueClass,
-      final Output out
-      ) throws IOException,InterruptedException,
-      ClassNotFoundException {
-    // wrap value iterator to report progress.
-    final TezRawKeyValueIterator rawIter = input.getIterator();
-    TezRawKeyValueIterator rIter = new TezRawKeyValueIterator() {
-      public void close() throws IOException {
-        rawIter.close();
-      }
-      public DataInputBuffer getKey() throws IOException {
-        return rawIter.getKey();
+  
+  ///////////////// End of methods for old API //////////////////////
+  
+  ///////////////// Methods for new API //////////////////////
+  
+  private void runNewCombiner(final TezRawKeyValueIterator rawIter, final Writer writer) throws InterruptedException, IOException {
+    
+    RecordWriter recordWriter = new RecordWriter() {
+
+      @Override
+      public void write(Object key, Object value) throws IOException,
+          InterruptedException {
+        writer.append(key, value);
       }
-      public Progress getProgress() {
-        return rawIter.getProgress();
-      }
-      public DataInputBuffer getValue() throws IOException {
-        return rawIter.getValue();
-      }
-      public boolean next() throws IOException {
-        boolean ret = rawIter.next();
-        // FIXME progress updates for combiner
-        // reporter.setProgress(rawIter.getProgress().getProgress());
-        return ret;
+
+      @Override
+      public void close(TaskAttemptContext context) throws IOException,
+          InterruptedException {
+        // Will be closed by whoever invokes the combiner.
       }
     };
-
-    // make a task context so we can get the classes
-    org.apache.hadoop.mapreduce.TaskAttemptContext taskContext =
-        new TaskAttemptContextImpl(job, task.getTaskAttemptId(), reporter);
-
-    // make a reducer
-    org.apache.hadoop.mapreduce.Reducer reducer =
-        (org.apache.hadoop.mapreduce.Reducer)
-        ReflectionUtils.newInstance(taskContext.getCombinerClass(), job);
-
-    org.apache.hadoop.mapreduce.RecordWriter trackedRW =
-        new org.apache.hadoop.mapreduce.RecordWriter() {
-
-          @Override
-          public void write(Object key, Object value) throws IOException,
-              InterruptedException {
-            out.write(key, value);
-          }
-
-          @Override
-          public void close(TaskAttemptContext context) throws IOException,
-              InterruptedException {
-            // Should not close this here as the sorter will close the
-            // combine output
-          }
-        };
-
+    
+    Class<? extends org.apache.hadoop.mapreduce.Reducer> reducerClazz = (Class<? extends org.apache.hadoop.mapreduce.Reducer>) conf
+        .getClass(MRJobConfig.COMBINE_CLASS_ATTR, null,
+            org.apache.hadoop.mapreduce.Reducer.class);
+    org.apache.hadoop.mapreduce.Reducer reducer = ReflectionUtils.newInstance(reducerClazz, conf);
+    
     org.apache.hadoop.mapreduce.Reducer.Context reducerContext =
         createReduceContext(
-            reducer, job, task.getTaskAttemptId(),
-            rIter, combinerInputKeyCounter,
-            combinerInputValueCounter,
-            trackedRW,
-            null,
-            reporter, comparator, keyClass,
-            valueClass);
+            conf,
+            mrTaskAttemptID,
+            rawIter,
+            new MRCounters.MRCounter(combineInputKeyCounter),
+            new MRCounters.MRCounter(combineInputValueCounter),
+            recordWriter,
+            reporter,
+            (RawComparator)comparator,
+            keyClass,
+            valClass);
+    
     reducer.run(reducerContext);
-    trackedRW.close(reducerContext);
-  }
-
-  @Override
-  public void close() throws IOException, InterruptedException {
+    recordWriter.close(reducerContext);
   }
 
-  protected static <INKEY,INVALUE,OUTKEY,OUTVALUE>
-  org.apache.hadoop.mapreduce.Reducer<INKEY,INVALUE,OUTKEY,OUTVALUE>.Context
-  createReduceContext(org.apache.hadoop.mapreduce.Reducer
-                        <INKEY,INVALUE,OUTKEY,OUTVALUE> reducer,
-                      Configuration job,
-                      TezTaskAttemptID taskId,
-                      final TezRawKeyValueIterator rIter,
-                      org.apache.hadoop.mapreduce.Counter inputKeyCounter,
-                      org.apache.hadoop.mapreduce.Counter inputValueCounter,
-                      org.apache.hadoop.mapreduce.RecordWriter<OUTKEY,OUTVALUE> output,
-                      org.apache.hadoop.mapreduce.OutputCommitter committer,
-                      org.apache.hadoop.mapreduce.StatusReporter reporter,
-                      RawComparator<INKEY> comparator,
-                      Class<INKEY> keyClass, Class<INVALUE> valueClass
-  ) throws IOException, InterruptedException {
-    RawKeyValueIterator r =
-        new RawKeyValueIterator() {
-
-          @Override
-          public boolean next() throws IOException {
-            return rIter.next();
-          }
+  private static <KEYIN, VALUEIN, KEYOUT, VALUEOUT> org.apache.hadoop.mapreduce.Reducer<KEYIN, VALUEIN, KEYOUT, VALUEOUT>.Context createReduceContext(
+      Configuration conf,
+      TaskAttemptID mrTaskAttemptID,
+      final TezRawKeyValueIterator rawIter,
+      Counter combineInputKeyCounter,
+      Counter combineInputValueCounter,
+      RecordWriter<KEYOUT, VALUEOUT> recordWriter,
+      MRTaskReporter reporter,
+      RawComparator<KEYIN> comparator,
+      Class<KEYIN> keyClass,
+      Class<VALUEIN> valClass) throws InterruptedException, IOException {
+
+    RawKeyValueIterator r = new RawKeyValueIterator() {
+
+      @Override
+      public boolean next() throws IOException {
+        return rawIter.next();
+      }
 
-          @Override
-          public DataInputBuffer getValue() throws IOException {
-            return rIter.getValue();
-          }
+      @Override
+      public DataInputBuffer getValue() throws IOException {
+        return rawIter.getValue();
+      }
 
-          @Override
-          public Progress getProgress() {
-            return rIter.getProgress();
-          }
+      @Override
+      public Progress getProgress() {
+        return rawIter.getProgress();
+      }
 
-          @Override
-          public DataInputBuffer getKey() throws IOException {
-            return rIter.getKey();
-          }
+      @Override
+      public DataInputBuffer getKey() throws IOException {
+        return rawIter.getKey();
+      }
 
-          @Override
-          public void close() throws IOException {
-            rIter.close();
-          }
-        };
-    org.apache.hadoop.mapreduce.ReduceContext<INKEY, INVALUE, OUTKEY, OUTVALUE>
-    reduceContext =
-      new ReduceContextImpl<INKEY, INVALUE, OUTKEY, OUTVALUE>(
-          job,
-          IDConverter.toMRTaskAttemptId(taskId),
-          r,
-          inputKeyCounter,
-          inputValueCounter,
-          output,
-          committer,
-          reporter,
-          comparator,
-          keyClass,
-          valueClass);
+      @Override
+      public void close() throws IOException {
+        rawIter.close();
+      }
+    };
 
-    org.apache.hadoop.mapreduce.Reducer<INKEY,INVALUE,OUTKEY,OUTVALUE>.Context
-        reducerContext = new
-          WrappedReducer<INKEY, INVALUE, OUTKEY, OUTVALUE>().getReducerContext(
-              reduceContext);
+    ReduceContext<KEYIN, VALUEIN, KEYOUT, VALUEOUT> rContext = new ReduceContextImpl<KEYIN, VALUEIN, KEYOUT, VALUEOUT>(
+        conf, mrTaskAttemptID, r, combineInputKeyCounter,
+        combineInputValueCounter, recordWriter, null, reporter, comparator,
+        keyClass, valClass);
 
+    org.apache.hadoop.mapreduce.Reducer<KEYIN, VALUEIN, KEYOUT, VALUEOUT>.Context reducerContext = new WrappedReducer<KEYIN, VALUEIN, KEYOUT, VALUEOUT>()
+        .getReducerContext(rContext);
     return reducerContext;
   }
 
+  
+ 
 }

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/3d609458/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/MRHelpers.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/MRHelpers.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/MRHelpers.java
index 7a9b7e0..b0ed6ab 100644
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/MRHelpers.java
+++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/MRHelpers.java
@@ -59,8 +59,8 @@ import org.apache.tez.common.TezJobConfig;
 import org.apache.tez.common.TezUtils;
 import org.apache.tez.dag.api.TezConfiguration;
 import org.apache.tez.dag.api.VertexLocationHint.TaskLocationHint;
-import org.apache.tez.mapreduce.newcombine.MRCombiner;
-import org.apache.tez.mapreduce.newpartition.MRPartitioner;
+import org.apache.tez.mapreduce.combine.MRCombiner;
+import org.apache.tez.mapreduce.partition.MRPartitioner;
 
 
 public class MRHelpers {

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/3d609458/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/MultiStageMRConfToTezTranslator.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/MultiStageMRConfToTezTranslator.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/MultiStageMRConfToTezTranslator.java
index d768312..d888c42 100644
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/MultiStageMRConfToTezTranslator.java
+++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/MultiStageMRConfToTezTranslator.java
@@ -29,9 +29,9 @@ import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.tez.common.TezJobConfig;
+import org.apache.tez.mapreduce.combine.MRCombiner;
 import org.apache.tez.mapreduce.hadoop.DeprecatedKeys.MultiStageKeys;
-import org.apache.tez.mapreduce.newcombine.MRCombiner;
-import org.apache.tez.mapreduce.newpartition.MRPartitioner;
+import org.apache.tez.mapreduce.partition.MRPartitioner;
 
 import com.google.common.base.Preconditions;
 

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/3d609458/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/mapred/MRReporter.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/mapred/MRReporter.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/mapred/MRReporter.java
new file mode 100644
index 0000000..f5e08dc
--- /dev/null
+++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/mapred/MRReporter.java
@@ -0,0 +1,76 @@
+package org.apache.tez.mapreduce.hadoop.mapred;
+
+import org.apache.hadoop.mapred.Counters.Counter;
+import org.apache.hadoop.mapred.InputSplit;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.tez.engine.newapi.TezProcessorContext;
+import org.apache.tez.engine.newapi.TezTaskContext;
+import org.apache.tez.mapreduce.common.Utils;
+
+public class MRReporter implements Reporter {
+
+  private TezTaskContext tezTaskContext;
+  private InputSplit split;
+  private boolean isProcessorContext = false;
+  
+  public MRReporter(TezProcessorContext tezProcContext) {
+    this(tezProcContext, null);
+    isProcessorContext = true;
+  }
+  public MRReporter(TezTaskContext tezTaskContext) {
+    this(tezTaskContext, null);
+  }
+
+  public MRReporter(TezTaskContext tezTaskContext, InputSplit split) {
+    this.tezTaskContext = tezTaskContext;
+    this.split = split;
+  }
+  
+  @Override
+  public void progress() {
+    //TODO NEWTEZ
+  }
+
+  @Override
+  public void setStatus(String status) {
+    // Not setting status string in Tez.
+
+  }
+
+  @Override
+  public Counter getCounter(Enum<?> name) {
+    return Utils.getMRCounter(tezTaskContext.getCounters().findCounter(name));
+  }
+
+  @Override
+  public Counter getCounter(String group, String name) {
+    return Utils.getMRCounter(tezTaskContext.getCounters().findCounter(group,
+        name));
+  }
+
+  @Override
+  public void incrCounter(Enum<?> key, long amount) {
+    getCounter(key).increment(amount);
+  }
+
+  @Override
+  public void incrCounter(String group, String counter, long amount) {
+    getCounter(group, counter).increment(amount);
+  }
+
+  @Override
+  public InputSplit getInputSplit() throws UnsupportedOperationException {
+    if (split == null) {
+      throw new UnsupportedOperationException("Input only available on map");
+    } else {
+      return split;
+    }
+  }
+
+  @Override
+  public float getProgress() {
+    // TOOD NEWTEZ Does this make a difference to anything ?
+    return 0.0f;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/3d609458/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/mapred/TaskAttemptContextImpl.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/mapred/TaskAttemptContextImpl.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/mapred/TaskAttemptContextImpl.java
index d953891..fc32825 100644
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/mapred/TaskAttemptContextImpl.java
+++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/mapred/TaskAttemptContextImpl.java
@@ -22,11 +22,8 @@ import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.TaskAttemptContext;
 import org.apache.hadoop.mapred.TaskAttemptID;
-import org.apache.hadoop.mapred.TaskID;
 import org.apache.hadoop.mapreduce.Counter;
-import org.apache.hadoop.mapreduce.TaskType;
 import org.apache.hadoop.util.Progressable;
-import org.apache.tez.dag.records.TezTaskAttemptID;
 import org.apache.tez.mapreduce.processor.MRTaskReporter;
 
 @InterfaceAudience.Private
@@ -36,24 +33,10 @@ public class TaskAttemptContextImpl
        implements TaskAttemptContext {
   private MRTaskReporter reporter;
 
-  public TaskAttemptContextImpl(JobConf conf, TezTaskAttemptID taskid) {
-    this(conf, taskid, null);
-  }
-  
   // FIXME we need to use DAG Id but we are using App Id
-  public TaskAttemptContextImpl(JobConf conf, TezTaskAttemptID taskAttemptId,
+  public TaskAttemptContextImpl(JobConf conf, TaskAttemptID taskAttemptId,
                          MRTaskReporter reporter) {
-    super(conf, 
-        new TaskAttemptID(
-            new TaskID(
-                Long.toString(taskAttemptId.getTaskID().getVertexID().
-                    getDAGId().getApplicationId().getClusterTimestamp()),
-                taskAttemptId.getTaskID().getVertexID().getDAGId().
-                    getApplicationId().getId(),
-                (taskAttemptId.getTaskID().getVertexID().getId() == 0 ?
-                    TaskType.MAP : TaskType.REDUCE),
-                taskAttemptId.getTaskID().getId()),
-              taskAttemptId.getId()));
+    super(conf, taskAttemptId);
     this.reporter = reporter;
   }
   

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/3d609458/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/mapreduce/MapContextImpl.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/mapreduce/MapContextImpl.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/mapreduce/MapContextImpl.java
index 4552397..b0348c9 100644
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/mapreduce/MapContextImpl.java
+++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/mapreduce/MapContextImpl.java
@@ -30,7 +30,7 @@ import org.apache.hadoop.mapreduce.OutputCommitter;
 import org.apache.hadoop.mapreduce.RecordReader;
 import org.apache.hadoop.mapreduce.RecordWriter;
 import org.apache.hadoop.mapreduce.TaskAttemptID;
-import org.apache.tez.mapreduce.processor.MRTaskReporter;
+import org.apache.tez.engine.newapi.TezTaskContext;
 
 /**
  * The context that is given to the {@link Mapper}.
@@ -51,9 +51,9 @@ public class MapContextImpl<KEYIN,VALUEIN,KEYOUT,VALUEOUT>
                         RecordReader<KEYIN,VALUEIN> reader,
                         RecordWriter<KEYOUT,VALUEOUT> writer,
                         OutputCommitter committer,
-                        MRTaskReporter reporter,
+                        TezTaskContext context,
                         InputSplit split) {
-    super(conf, taskid, writer, committer, reporter);
+    super(conf, taskid, writer, committer, context);
     this.reader = reader;
     this.split = split;
   }

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/3d609458/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/mapreduce/ReduceContextImpl.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/mapreduce/ReduceContextImpl.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/mapreduce/ReduceContextImpl.java
deleted file mode 100644
index e775b7e..0000000
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/mapreduce/ReduceContextImpl.java
+++ /dev/null
@@ -1,359 +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.hadoop.mapreduce;
-
-import java.io.DataOutputStream;
-import java.io.IOException;
-import java.util.Iterator;
-import java.util.NoSuchElementException;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.io.BytesWritable;
-import org.apache.hadoop.io.DataInputBuffer;
-import org.apache.hadoop.io.RawComparator;
-import org.apache.hadoop.io.WritableUtils;
-import org.apache.hadoop.io.serializer.Deserializer;
-import org.apache.hadoop.io.serializer.SerializationFactory;
-import org.apache.hadoop.io.serializer.Serializer;
-import org.apache.hadoop.mapred.BackupStore;
-import org.apache.hadoop.mapred.RawKeyValueIterator;
-import org.apache.hadoop.mapreduce.Counter;
-import org.apache.hadoop.mapreduce.OutputCommitter;
-import org.apache.hadoop.mapreduce.RecordWriter;
-import org.apache.hadoop.mapreduce.ReduceContext;
-import org.apache.hadoop.mapreduce.Reducer;
-import org.apache.hadoop.mapreduce.TaskAttemptID;
-import org.apache.hadoop.util.Progressable;
-import org.apache.tez.mapreduce.processor.MRTaskReporter;
-
-/**
- * The context passed to the {@link Reducer}.
- * @param <KEYIN> the class of the input keys
- * @param <VALUEIN> the class of the input values
- * @param <KEYOUT> the class of the output keys
- * @param <VALUEOUT> the class of the output values
- */
-@InterfaceAudience.Private
-@InterfaceStability.Unstable
-public class ReduceContextImpl<KEYIN,VALUEIN,KEYOUT,VALUEOUT>
-    extends TaskInputOutputContextImpl<KEYIN,VALUEIN,KEYOUT,VALUEOUT> 
-    implements ReduceContext<KEYIN, VALUEIN, KEYOUT, VALUEOUT> {
-  private RawKeyValueIterator input;
-  private Counter inputValueCounter;
-  private Counter inputKeyCounter;
-  private RawComparator<KEYIN> comparator;
-  private KEYIN key;                                  // current key
-  private VALUEIN value;                              // current value
-  private boolean firstValue = false;                 // first value in key
-  private boolean nextKeyIsSame = false;              // more w/ this key
-  private boolean hasMore;                            // more in file
-  protected Progressable reporter;
-  private Deserializer<KEYIN> keyDeserializer;
-  private Deserializer<VALUEIN> valueDeserializer;
-  private DataInputBuffer buffer = new DataInputBuffer();
-  private BytesWritable currentRawKey = new BytesWritable();
-  private ValueIterable iterable = new ValueIterable();
-  private boolean isMarked = false;
-  private BackupStore<KEYIN,VALUEIN> backupStore;
-  private final SerializationFactory serializationFactory;
-  private final Class<KEYIN> keyClass;
-  private final Class<VALUEIN> valueClass;
-  private final Configuration conf;
-  private final TaskAttemptID taskid;
-  private int currentKeyLength = -1;
-  private int currentValueLength = -1;
-  
-  public ReduceContextImpl(Configuration conf, TaskAttemptID taskid,
-                           RawKeyValueIterator input, 
-                           Counter inputKeyCounter,
-                           Counter inputValueCounter,
-                           RecordWriter<KEYOUT,VALUEOUT> output,
-                           OutputCommitter committer,
-                           MRTaskReporter reporter,
-                           RawComparator<KEYIN> comparator,
-                           Class<KEYIN> keyClass,
-                           Class<VALUEIN> valueClass
-                          ) throws InterruptedException, IOException{
-    super(conf, taskid, output, committer, reporter);
-    this.input = input;
-    this.inputKeyCounter = inputKeyCounter;
-    this.inputValueCounter = inputValueCounter;
-    this.comparator = comparator;
-    this.serializationFactory = new SerializationFactory(conf);
-    this.keyDeserializer = serializationFactory.getDeserializer(keyClass);
-    this.keyDeserializer.open(buffer);
-    this.valueDeserializer = serializationFactory.getDeserializer(valueClass);
-    this.valueDeserializer.open(buffer);
-    hasMore = input.next();
-    this.keyClass = keyClass;
-    this.valueClass = valueClass;
-    this.conf = conf;
-    this.taskid = taskid;
-  }
-
-  /** Start processing next unique key. */
-  public boolean nextKey() throws IOException,InterruptedException {
-    while (hasMore && nextKeyIsSame) {
-      nextKeyValue();
-    }
-    if (hasMore) {
-      if (inputKeyCounter != null) {
-        inputKeyCounter.increment(1);
-      }
-      return nextKeyValue();
-    } else {
-      return false;
-    }
-  }
-
-  /**
-   * Advance to the next key/value pair.
-   */
-  @Override
-  public boolean nextKeyValue() throws IOException, InterruptedException {
-    if (!hasMore) {
-      key = null;
-      value = null;
-      return false;
-    }
-    firstValue = !nextKeyIsSame;
-    DataInputBuffer nextKey = input.getKey();
-    currentRawKey.set(nextKey.getData(), nextKey.getPosition(), 
-                      nextKey.getLength() - nextKey.getPosition());
-    buffer.reset(currentRawKey.getBytes(), 0, currentRawKey.getLength());
-    key = keyDeserializer.deserialize(key);
-    DataInputBuffer nextVal = input.getValue();
-    buffer.reset(nextVal.getData(), nextVal.getPosition(), nextVal.getLength());
-    value = valueDeserializer.deserialize(value);
-
-    currentKeyLength = nextKey.getLength() - nextKey.getPosition();
-    currentValueLength = nextVal.getLength() - nextVal.getPosition();
-
-    if (isMarked) {
-      backupStore.write(nextKey, nextVal);
-    }
-
-    hasMore = input.next();
-    if (hasMore) {
-      nextKey = input.getKey();
-      nextKeyIsSame = comparator.compare(currentRawKey.getBytes(), 0, 
-                                     currentRawKey.getLength(),
-                                     nextKey.getData(),
-                                     nextKey.getPosition(),
-                                     nextKey.getLength() - nextKey.getPosition()
-                                         ) == 0;
-    } else {
-      nextKeyIsSame = false;
-    }
-    inputValueCounter.increment(1);
-    return true;
-  }
-
-  public KEYIN getCurrentKey() {
-    return key;
-  }
-
-  @Override
-  public VALUEIN getCurrentValue() {
-    return value;
-  }
-  
-  BackupStore<KEYIN,VALUEIN> getBackupStore() {
-    return backupStore;
-  }
-  
-  protected class ValueIterator implements ReduceContext.ValueIterator<VALUEIN> {
-
-    private boolean inReset = false;
-    private boolean clearMarkFlag = false;
-
-    public boolean hasNext() {
-      try {
-        if (inReset && backupStore.hasNext()) {
-          return true;
-        } 
-      } catch (Exception e) {
-        e.printStackTrace();
-        throw new RuntimeException("hasNext failed", e);
-      }
-      return firstValue || nextKeyIsSame;
-    }
-
-    public VALUEIN next() {
-      if (inReset) {
-        try {
-          if (backupStore.hasNext()) {
-            backupStore.next();
-            DataInputBuffer next = backupStore.nextValue();
-            buffer.reset(next.getData(), next.getPosition(), next.getLength());
-            value = valueDeserializer.deserialize(value);
-            return value;
-          } else {
-            inReset = false;
-            backupStore.exitResetMode();
-            if (clearMarkFlag) {
-              clearMarkFlag = false;
-              isMarked = false;
-            }
-          }
-        } catch (IOException e) {
-          e.printStackTrace();
-          throw new RuntimeException("next value iterator failed", e);
-        }
-      } 
-
-      // if this is the first record, we don't need to advance
-      if (firstValue) {
-        firstValue = false;
-        return value;
-      }
-      // if this isn't the first record and the next key is different, they
-      // can't advance it here.
-      if (!nextKeyIsSame) {
-        throw new NoSuchElementException("iterate past last value");
-      }
-      // otherwise, go to the next key/value pair
-      try {
-        nextKeyValue();
-        return value;
-      } catch (IOException ie) {
-        throw new RuntimeException("next value iterator failed", ie);
-      } catch (InterruptedException ie) {
-        // this is bad, but we can't modify the exception list of java.util
-        throw new RuntimeException("next value iterator interrupted", ie);        
-      }
-    }
-
-    public void remove() {
-      throw new UnsupportedOperationException("remove not implemented");
-    }
-
-    public void mark() throws IOException {
-      if (getBackupStore() == null) {
-        backupStore = new BackupStore<KEYIN,VALUEIN>(conf, taskid);
-      }
-      isMarked = true;
-      if (!inReset) {
-        backupStore.reinitialize();
-        if (currentKeyLength == -1) {
-          // The user has not called next() for this iterator yet, so
-          // there is no current record to mark and copy to backup store.
-          return;
-        }
-        assert (currentValueLength != -1);
-        int requestedSize = currentKeyLength + currentValueLength + 
-          WritableUtils.getVIntSize(currentKeyLength) +
-          WritableUtils.getVIntSize(currentValueLength);
-        DataOutputStream out = backupStore.getOutputStream(requestedSize);
-        writeFirstKeyValueBytes(out);
-        backupStore.updateCounters(requestedSize);
-      } else {
-        backupStore.mark();
-      }
-    }
-
-    public void reset() throws IOException {
-      // We reached the end of an iteration and user calls a 
-      // reset, but a clearMark was called before, just throw
-      // an exception
-      if (clearMarkFlag) {
-        clearMarkFlag = false;
-        backupStore.clearMark();
-        throw new IOException("Reset called without a previous mark");
-      }
-      
-      if (!isMarked) {
-        throw new IOException("Reset called without a previous mark");
-      }
-      inReset = true;
-      backupStore.reset();
-    }
-
-    public void clearMark() throws IOException {
-      if (getBackupStore() == null) {
-        return;
-      }
-      if (inReset) {
-        clearMarkFlag = true;
-        backupStore.clearMark();
-      } else {
-        inReset = isMarked = false;
-        backupStore.reinitialize();
-      }
-    }
-    
-    /**
-     * This method is called when the reducer moves from one key to 
-     * another.
-     * @throws IOException
-     */
-    public void resetBackupStore() throws IOException {
-      if (getBackupStore() == null) {
-        return;
-      }
-      inReset = isMarked = false;
-      backupStore.reinitialize();
-      currentKeyLength = -1;
-    }
-
-    /**
-     * This method is called to write the record that was most recently
-     * served (before a call to the mark). Since the framework reads one
-     * record in advance, to get this record, we serialize the current key
-     * and value
-     * @param out
-     * @throws IOException
-     */
-    private void writeFirstKeyValueBytes(DataOutputStream out) 
-    throws IOException {
-      assert (getCurrentKey() != null && getCurrentValue() != null);
-      WritableUtils.writeVInt(out, currentKeyLength);
-      WritableUtils.writeVInt(out, currentValueLength);
-      Serializer<KEYIN> keySerializer = 
-        serializationFactory.getSerializer(keyClass);
-      keySerializer.open(out);
-      keySerializer.serialize(getCurrentKey());
-
-      Serializer<VALUEIN> valueSerializer = 
-        serializationFactory.getSerializer(valueClass);
-      valueSerializer.open(out);
-      valueSerializer.serialize(getCurrentValue());
-    }
-  }
-
-  protected class ValueIterable implements Iterable<VALUEIN> {
-    private ValueIterator iterator = new ValueIterator();
-    public Iterator<VALUEIN> iterator() {
-      return iterator;
-    } 
-  }
-  
-  /**
-   * Iterate through the values for the current key, reusing the same value 
-   * object, which is stored in the context.
-   * @return the series of values associated with the current key. All of the 
-   * objects returned directly and indirectly from this method are reused.
-   */
-  public 
-  Iterable<VALUEIN> getValues() throws IOException, InterruptedException {
-    return iterable;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/3d609458/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/mapreduce/TaskAttemptContextImpl.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/mapreduce/TaskAttemptContextImpl.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/mapreduce/TaskAttemptContextImpl.java
index 7b69872..d8548a4 100644
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/mapreduce/TaskAttemptContextImpl.java
+++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/mapreduce/TaskAttemptContextImpl.java
@@ -1,5 +1,4 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
+/* 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
@@ -21,98 +20,71 @@ package org.apache.tez.mapreduce.hadoop.mapreduce;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.mapred.Task;
 import org.apache.hadoop.mapreduce.Counter;
-import org.apache.hadoop.mapreduce.Counters;
-import org.apache.hadoop.mapreduce.StatusReporter;
-import org.apache.hadoop.mapreduce.TaskAttemptContext;
 import org.apache.hadoop.mapreduce.TaskAttemptID;
-import org.apache.tez.mapreduce.hadoop.IDConverter;
-import org.apache.tez.mapreduce.processor.MRTaskReporter;
+import org.apache.hadoop.mapreduce.TaskID;
+import org.apache.hadoop.mapreduce.TaskType;
+import org.apache.tez.engine.newapi.TezTaskContext;
+import org.apache.tez.mapreduce.common.Utils;
+
+// NOTE: NEWTEZ: This is a copy of org.apache.tez.mapreduce.hadoop.mapred (not mapreduce). mapred likely does not need it's own copy of this class.
+// Meant for use by the "mapreduce" API
 
-/**
- * The context for task attempts.
- */
 @InterfaceAudience.Private
 @InterfaceStability.Unstable
-public class TaskAttemptContextImpl extends JobContextImpl 
-    implements TaskAttemptContext {
-  private final TaskAttemptID taskId;
-  private String status = "";
-  private MRTaskReporter reporter;
+public class TaskAttemptContextImpl
+       extends org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl {
 
-  public TaskAttemptContextImpl(Configuration conf, 
-                                TaskAttemptID taskId) {
-    this(conf, taskId, null);
-  }
+  private TezTaskContext taskContext;
 
+  // FIXME we need to use DAG Id but we are using App Id
   public TaskAttemptContextImpl(Configuration conf,
-      TaskAttemptID taskId, MRTaskReporter reporter) {
-    super(conf, IDConverter.fromMRJobId(taskId.getJobID()));
-    this.taskId = taskId;
-    this.reporter = reporter;
+      TezTaskContext taskContext, boolean isMap) {
+    // TODO NEWTEZ Can the jt Identifier string be taskContext.getUniqueId ?
+    this(conf, new TaskAttemptID(
+        new TaskID(String.valueOf(taskContext.getApplicationId()
+            .getClusterTimestamp()), taskContext.getApplicationId().getId(),
+            isMap ? TaskType.MAP : TaskType.REDUCE,
+            taskContext.getTaskIndex()),
+            taskContext.getTaskAttemptNumber()), taskContext);
   }
 
-  /**
-   * Get the unique name for this task attempt.
-   */
-  public TaskAttemptID getTaskAttemptID() {
-    return taskId;
+  public TaskAttemptContextImpl(Configuration conf, TaskAttemptID taId, TezTaskContext context) {
+    super(conf, taId);
+    this.taskContext = context;
   }
 
-  /**
-   * Get the last set status message.
-   * @return the current status message
-   */
-  public String getStatus() {
-    return status;
+  @Override
+  public float getProgress() {
+    // TODO NEWTEZ Will this break anything ?
+    return 0.0f;
   }
 
+  @Override
   public Counter getCounter(Enum<?> counterName) {
-    return (Counter) reporter.getCounter(counterName);
+    return Utils.getMRCounter(taskContext.getCounters().findCounter(counterName));
   }
 
+  @Override
   public Counter getCounter(String groupName, String counterName) {
-    return (Counter) reporter.getCounter(groupName, counterName);
+    return Utils.getMRCounter(taskContext.getCounters().findCounter(groupName, counterName));
   }
 
   /**
    * Report progress.
    */
+  @Override
   public void progress() {
-    reporter.progress();
-  }
-
-  protected void setStatusString(String status) {
-    this.status = status;
+    // Nothing to do.
   }
 
   /**
    * Set the current status of the task to the given string.
    */
+  @Override
   public void setStatus(String status) {
-    String normalizedStatus = Task.normalizeStatus(status, conf);
-    setStatusString(normalizedStatus);
-    reporter.setStatus(normalizedStatus);
-  }
-
-  public static class DummyReporter extends StatusReporter {
-    public void setStatus(String s) {
-    }
-    public void progress() {
-    }
-    public Counter getCounter(Enum<?> name) {
-      return new Counters().findCounter(name);
-    }
-    public Counter getCounter(String group, String name) {
-      return new Counters().findCounter(group, name);
-    }
-    public float getProgress() {
-      return 0f;
-    }
-  }
-  
-  public float getProgress() {
-    return reporter.getProgress();
+    setStatusString(status);
+    // Nothing to do until InputContext supports some kind of custom string
+    // diagnostics.
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/3d609458/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/mapreduce/TaskInputOutputContextImpl.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/mapreduce/TaskInputOutputContextImpl.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/mapreduce/TaskInputOutputContextImpl.java
index ff4e18f..c2920dc 100644
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/mapreduce/TaskInputOutputContextImpl.java
+++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/mapreduce/TaskInputOutputContextImpl.java
@@ -29,7 +29,7 @@ import org.apache.hadoop.mapreduce.RecordWriter;
 import org.apache.hadoop.mapreduce.Reducer;
 import org.apache.hadoop.mapreduce.TaskAttemptID;
 import org.apache.hadoop.mapreduce.TaskInputOutputContext;
-import org.apache.tez.mapreduce.processor.MRTaskReporter;
+import org.apache.tez.engine.newapi.TezTaskContext;
 
 /**
  * A context object that allows input and output from the task. It is only
@@ -50,8 +50,8 @@ public abstract class TaskInputOutputContextImpl<KEYIN,VALUEIN,KEYOUT,VALUEOUT>
   public TaskInputOutputContextImpl(Configuration conf, TaskAttemptID taskid,
                                     RecordWriter<KEYOUT,VALUEOUT> output,
                                     OutputCommitter committer,
-                                    MRTaskReporter reporter) {
-    super(conf, taskid, reporter);
+                                    TezTaskContext context) {
+    super(conf, taskid, context);
     this.output = output;
     this.committer = committer;
   }

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/3d609458/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/newmapred/MRReporter.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/newmapred/MRReporter.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/newmapred/MRReporter.java
deleted file mode 100644
index df23b9f..0000000
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/newmapred/MRReporter.java
+++ /dev/null
@@ -1,76 +0,0 @@
-package org.apache.tez.mapreduce.hadoop.newmapred;
-
-import org.apache.hadoop.mapred.Counters.Counter;
-import org.apache.hadoop.mapred.InputSplit;
-import org.apache.hadoop.mapred.Reporter;
-import org.apache.tez.engine.newapi.TezProcessorContext;
-import org.apache.tez.engine.newapi.TezTaskContext;
-import org.apache.tez.mapreduce.common.Utils;
-
-public class MRReporter implements Reporter {
-
-  private TezTaskContext tezTaskContext;
-  private InputSplit split;
-  private boolean isProcessorContext = false;
-  
-  public MRReporter(TezProcessorContext tezProcContext) {
-    this(tezProcContext, null);
-    isProcessorContext = true;
-  }
-  public MRReporter(TezTaskContext tezTaskContext) {
-    this(tezTaskContext, null);
-  }
-
-  public MRReporter(TezTaskContext tezTaskContext, InputSplit split) {
-    this.tezTaskContext = tezTaskContext;
-    this.split = split;
-  }
-  
-  @Override
-  public void progress() {
-    //TODO NEWTEZ
-  }
-
-  @Override
-  public void setStatus(String status) {
-    // Not setting status string in Tez.
-
-  }
-
-  @Override
-  public Counter getCounter(Enum<?> name) {
-    return Utils.getMRCounter(tezTaskContext.getCounters().findCounter(name));
-  }
-
-  @Override
-  public Counter getCounter(String group, String name) {
-    return Utils.getMRCounter(tezTaskContext.getCounters().findCounter(group,
-        name));
-  }
-
-  @Override
-  public void incrCounter(Enum<?> key, long amount) {
-    getCounter(key).increment(amount);
-  }
-
-  @Override
-  public void incrCounter(String group, String counter, long amount) {
-    getCounter(group, counter).increment(amount);
-  }
-
-  @Override
-  public InputSplit getInputSplit() throws UnsupportedOperationException {
-    if (split == null) {
-      throw new UnsupportedOperationException("Input only available on map");
-    } else {
-      return split;
-    }
-  }
-
-  @Override
-  public float getProgress() {
-    // TOOD NEWTEZ Does this make a difference to anything ?
-    return 0.0f;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/3d609458/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/newmapred/TaskAttemptContextImpl.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/newmapred/TaskAttemptContextImpl.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/newmapred/TaskAttemptContextImpl.java
deleted file mode 100644
index 956fcc2..0000000
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/newmapred/TaskAttemptContextImpl.java
+++ /dev/null
@@ -1,93 +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.hadoop.newmapred;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.TaskAttemptContext;
-import org.apache.hadoop.mapred.TaskAttemptID;
-import org.apache.hadoop.mapreduce.Counter;
-import org.apache.hadoop.util.Progressable;
-import org.apache.tez.mapreduce.newprocessor.MRTaskReporter;
-
-@InterfaceAudience.Private
-@InterfaceStability.Unstable
-public class TaskAttemptContextImpl
-       extends org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl 
-       implements TaskAttemptContext {
-  private MRTaskReporter reporter;
-
-  // FIXME we need to use DAG Id but we are using App Id
-  public TaskAttemptContextImpl(JobConf conf, TaskAttemptID taskAttemptId,
-                         MRTaskReporter reporter) {
-    super(conf, taskAttemptId);
-    this.reporter = reporter;
-  }
-  
-  /**
-   * Get the taskAttemptID.
-   *  
-   * @return TaskAttemptID
-   */
-  public TaskAttemptID getTaskAttemptID() {
-    return (TaskAttemptID) super.getTaskAttemptID();
-  }
-  
-  public Progressable getProgressible() {
-    return reporter;
-  }
-  
-  public JobConf getJobConf() {
-    return (JobConf) getConfiguration();
-  }
-  
-  @Override
-  public float getProgress() {
-    return reporter.getProgress();
-  }
-
-  @Override
-  public Counter getCounter(Enum<?> counterName) {
-    return (Counter) reporter.getCounter(counterName);
-  }
-
-  @Override
-  public Counter getCounter(String groupName, String counterName) {
-    return (Counter) reporter.getCounter(groupName, counterName);
-  }
-
-  /**
-   * Report progress.
-   */
-  @Override
-  public void progress() {
-    reporter.progress();
-  }
-
-  /**
-   * Set the current status of the task to the given string.
-   */
-  @Override
-  public void setStatus(String status) {
-    setStatusString(status);
-    reporter.setStatus(status);
-  }
-
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/3d609458/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/newmapreduce/MapContextImpl.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/newmapreduce/MapContextImpl.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/newmapreduce/MapContextImpl.java
deleted file mode 100644
index 39c7194..0000000
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/newmapreduce/MapContextImpl.java
+++ /dev/null
@@ -1,84 +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.hadoop.newmapreduce;
-
-import java.io.IOException;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.mapreduce.InputSplit;
-import org.apache.hadoop.mapreduce.MapContext;
-import org.apache.hadoop.mapreduce.Mapper;
-import org.apache.hadoop.mapreduce.OutputCommitter;
-import org.apache.hadoop.mapreduce.RecordReader;
-import org.apache.hadoop.mapreduce.RecordWriter;
-import org.apache.hadoop.mapreduce.TaskAttemptID;
-import org.apache.tez.engine.newapi.TezTaskContext;
-
-/**
- * The context that is given to the {@link Mapper}.
- * @param <KEYIN> the key input type to the Mapper
- * @param <VALUEIN> the value input type to the Mapper
- * @param <KEYOUT> the key output type from the Mapper
- * @param <VALUEOUT> the value output type from the Mapper
- */
-@InterfaceAudience.Private
-@InterfaceStability.Unstable
-public class MapContextImpl<KEYIN,VALUEIN,KEYOUT,VALUEOUT> 
-    extends TaskInputOutputContextImpl<KEYIN,VALUEIN,KEYOUT,VALUEOUT> 
-    implements MapContext<KEYIN, VALUEIN, KEYOUT, VALUEOUT> {
-  private RecordReader<KEYIN,VALUEIN> reader;
-  private InputSplit split;
-
-  public MapContextImpl(Configuration conf, TaskAttemptID taskid,
-                        RecordReader<KEYIN,VALUEIN> reader,
-                        RecordWriter<KEYOUT,VALUEOUT> writer,
-                        OutputCommitter committer,
-                        TezTaskContext context,
-                        InputSplit split) {
-    super(conf, taskid, writer, committer, context);
-    this.reader = reader;
-    this.split = split;
-  }
-
-  /**
-   * Get the input split for this map.
-   */
-  public InputSplit getInputSplit() {
-    return split;
-  }
-
-  @Override
-  public KEYIN getCurrentKey() throws IOException, InterruptedException {
-    return reader.getCurrentKey();
-  }
-
-  @Override
-  public VALUEIN getCurrentValue() throws IOException, InterruptedException {
-    return reader.getCurrentValue();
-  }
-
-  @Override
-  public boolean nextKeyValue() throws IOException, InterruptedException {
-    return reader.nextKeyValue();
-  }
-
-}
-     

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/3d609458/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/newmapreduce/TaskAttemptContextImpl.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/newmapreduce/TaskAttemptContextImpl.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/newmapreduce/TaskAttemptContextImpl.java
deleted file mode 100644
index f2057e9..0000000
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/newmapreduce/TaskAttemptContextImpl.java
+++ /dev/null
@@ -1,90 +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.hadoop.newmapreduce;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.mapreduce.Counter;
-import org.apache.hadoop.mapreduce.TaskAttemptID;
-import org.apache.hadoop.mapreduce.TaskID;
-import org.apache.hadoop.mapreduce.TaskType;
-import org.apache.tez.engine.newapi.TezTaskContext;
-import org.apache.tez.mapreduce.common.Utils;
-
-// NOTE: NEWTEZ: This is a copy of org.apache.tez.mapreduce.hadoop.mapred (not mapreduce). mapred likely does not need it's own copy of this class.
-// Meant for use by the "mapreduce" API
-
-@InterfaceAudience.Private
-@InterfaceStability.Unstable
-public class TaskAttemptContextImpl
-       extends org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl {
-
-  private TezTaskContext taskContext;
-
-  // FIXME we need to use DAG Id but we are using App Id
-  public TaskAttemptContextImpl(Configuration conf,
-      TezTaskContext taskContext, boolean isMap) {
-    // TODO NEWTEZ Can the jt Identifier string be taskContext.getUniqueId ?
-    this(conf, new TaskAttemptID(
-        new TaskID(String.valueOf(taskContext.getApplicationId()
-            .getClusterTimestamp()), taskContext.getApplicationId().getId(),
-            isMap ? TaskType.MAP : TaskType.REDUCE,
-            taskContext.getTaskIndex()),
-            taskContext.getTaskAttemptNumber()), taskContext);
-  }
-
-  public TaskAttemptContextImpl(Configuration conf, TaskAttemptID taId, TezTaskContext context) {
-    super(conf, taId);
-    this.taskContext = context;
-  }
-
-  @Override
-  public float getProgress() {
-    // TODO NEWTEZ Will this break anything ?
-    return 0.0f;
-  }
-
-  @Override
-  public Counter getCounter(Enum<?> counterName) {
-    return Utils.getMRCounter(taskContext.getCounters().findCounter(counterName));
-  }
-
-  @Override
-  public Counter getCounter(String groupName, String counterName) {
-    return Utils.getMRCounter(taskContext.getCounters().findCounter(groupName, counterName));
-  }
-
-  /**
-   * Report progress.
-   */
-  @Override
-  public void progress() {
-    // Nothing to do.
-  }
-
-  /**
-   * Set the current status of the task to the given string.
-   */
-  @Override
-  public void setStatus(String status) {
-    setStatusString(status);
-    // Nothing to do until InputContext supports some kind of custom string
-    // diagnostics.
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/3d609458/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/newmapreduce/TaskInputOutputContextImpl.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/newmapreduce/TaskInputOutputContextImpl.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/newmapreduce/TaskInputOutputContextImpl.java
deleted file mode 100644
index a34cf8d..0000000
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/newmapreduce/TaskInputOutputContextImpl.java
+++ /dev/null
@@ -1,95 +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.hadoop.newmapreduce;
-
-import java.io.IOException;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.mapreduce.Mapper;
-import org.apache.hadoop.mapreduce.OutputCommitter;
-import org.apache.hadoop.mapreduce.RecordWriter;
-import org.apache.hadoop.mapreduce.Reducer;
-import org.apache.hadoop.mapreduce.TaskAttemptID;
-import org.apache.hadoop.mapreduce.TaskInputOutputContext;
-import org.apache.tez.engine.newapi.TezTaskContext;
-
-/**
- * A context object that allows input and output from the task. It is only
- * supplied to the {@link Mapper} or {@link Reducer}.
- * @param <KEYIN> the input key type for the task
- * @param <VALUEIN> the input value type for the task
- * @param <KEYOUT> the output key type for the task
- * @param <VALUEOUT> the output value type for the task
- */
-@InterfaceAudience.Private
-@InterfaceStability.Unstable
-public abstract class TaskInputOutputContextImpl<KEYIN,VALUEIN,KEYOUT,VALUEOUT> 
-       extends TaskAttemptContextImpl 
-       implements TaskInputOutputContext<KEYIN, VALUEIN, KEYOUT, VALUEOUT> {
-  private RecordWriter<KEYOUT,VALUEOUT> output;
-  private OutputCommitter committer;
-
-  public TaskInputOutputContextImpl(Configuration conf, TaskAttemptID taskid,
-                                    RecordWriter<KEYOUT,VALUEOUT> output,
-                                    OutputCommitter committer,
-                                    TezTaskContext context) {
-    super(conf, taskid, context);
-    this.output = output;
-    this.committer = committer;
-  }
-
-  /**
-   * Advance to the next key, value pair, returning null if at end.
-   * @return the key object that was read into, or null if no more
-   */
-  public abstract 
-  boolean nextKeyValue() throws IOException, InterruptedException;
- 
-  /**
-   * Get the current key.
-   * @return the current key object or null if there isn't one
-   * @throws IOException
-   * @throws InterruptedException
-   */
-  public abstract 
-  KEYIN getCurrentKey() throws IOException, InterruptedException;
-
-  /**
-   * Get the current value.
-   * @return the value object that was read into
-   * @throws IOException
-   * @throws InterruptedException
-   */
-  public abstract VALUEIN getCurrentValue() throws IOException, 
-                                                   InterruptedException;
-
-  /**
-   * Generate an output key/value pair.
-   */
-  public void write(KEYOUT key, VALUEOUT value
-                    ) throws IOException, InterruptedException {
-    output.write(key, value);
-  }
-
-  public OutputCommitter getOutputCommitter() {
-    return committer;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/3d609458/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/ShuffledMergedInputLegacy.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/ShuffledMergedInputLegacy.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/ShuffledMergedInputLegacy.java
new file mode 100644
index 0000000..2d230d6
--- /dev/null
+++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/ShuffledMergedInputLegacy.java
@@ -0,0 +1,29 @@
+/**
+ * <code>ShuffleMergedInput</code> in a {@link LogicalInput} which shuffles
+ * intermediate sorted data, merges them and provides key/<values> to the
+ * consumer.
+ * 
+ * The Copy and Merge will be triggered by the initialization - which is handled
+ * by the Tez framework. Input is not consumable until the Copy and Merge are
+ * complete. Methods are provided to check for this, as well as to wait for
+ * completion. Attempting to get a reader on a non-complete input will block.
+ * 
+ */
+
+package org.apache.tez.mapreduce.input;
+
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.tez.engine.common.sort.impl.TezRawKeyValueIterator;
+import org.apache.tez.engine.lib.input.ShuffledMergedInput;
+
+public class ShuffledMergedInputLegacy extends ShuffledMergedInput {
+
+  @Private
+  public TezRawKeyValueIterator getIterator() throws IOException, InterruptedException {
+    // wait for input so that iterator is available
+    waitForInputReady();
+    return rawIter;
+  }
+}


[05/50] [abbrv] git commit: TEZ-477. Fix NPEs etc caused by TEZ-433 (part of TEZ-398). (sseth)

Posted by ss...@apache.org.
TEZ-477. Fix NPEs etc caused by TEZ-433 (part of TEZ-398). (sseth)


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

Branch: refs/heads/master
Commit: 5eb0c86f204aad1354e55192bc82e17320949acf
Parents: 9a7e7bc
Author: Siddharth Seth <ss...@apache.org>
Authored: Sat Sep 21 12:11:54 2013 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Sat Sep 21 12:11:54 2013 -0700

----------------------------------------------------------------------
 .../main/java/org/apache/hadoop/mapred/YarnTezDagChild.java    | 2 +-
 .../tez/mapreduce/hadoop/MultiStageMRConfToTezTranslator.java  | 2 +-
 .../java/org/apache/tez/mapreduce/newcombine/MRCombiner.java   | 6 +++---
 .../java/org/apache/tez/mapreduce/newoutput/SimpleOutput.java  | 2 +-
 4 files changed, 6 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/5eb0c86f/tez-dag/src/main/java/org/apache/hadoop/mapred/YarnTezDagChild.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/hadoop/mapred/YarnTezDagChild.java b/tez-dag/src/main/java/org/apache/hadoop/mapred/YarnTezDagChild.java
index c749850..2d3e4d1 100644
--- a/tez-dag/src/main/java/org/apache/hadoop/mapred/YarnTezDagChild.java
+++ b/tez-dag/src/main/java/org/apache/hadoop/mapred/YarnTezDagChild.java
@@ -514,7 +514,7 @@ public class YarnTezDagChild {
     }
     String [] localDirs = StringUtils.getTrimmedStrings(System.getenv(Environment.LOCAL_DIRS.name()));
     conf.setStrings(TezJobConfig.LOCAL_DIRS, localDirs);
-    LOG.info("LocalDirs for child: " + localDirs);
+    LOG.info("LocalDirs for child: " + Arrays.toString(localDirs));
     return new LogicalIOProcessorRuntimeTask(taskSpec, attemptNum, conf,
         tezUmbilical, jobToken);
   }

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/5eb0c86f/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/MultiStageMRConfToTezTranslator.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/MultiStageMRConfToTezTranslator.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/MultiStageMRConfToTezTranslator.java
index ad231b3..d768312 100644
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/MultiStageMRConfToTezTranslator.java
+++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/MultiStageMRConfToTezTranslator.java
@@ -240,9 +240,9 @@ public class MultiStageMRConfToTezTranslator {
     Configuration confs[] = new Configuration[numStages];
     Configuration nonItermediateConf = MultiStageMRConfigUtil.extractStageConf(
         conf, "");
-    confs[0].setBoolean(MRConfig.IS_MAP_PROCESSOR, true);
     if (numStages == 1) {
       confs[0] = nonItermediateConf;
+      confs[0].setBoolean(MRConfig.IS_MAP_PROCESSOR, true);
     } else {
       confs[0] = nonItermediateConf;
       confs[numStages - 1] = new Configuration(nonItermediateConf);

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/5eb0c86f/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/newcombine/MRCombiner.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/newcombine/MRCombiner.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/newcombine/MRCombiner.java
index 788019a..5566fd8 100644
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/newcombine/MRCombiner.java
+++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/newcombine/MRCombiner.java
@@ -49,7 +49,7 @@ import org.apache.tez.engine.common.sort.impl.IFile.Writer;
 import org.apache.tez.engine.common.sort.impl.TezRawKeyValueIterator;
 import org.apache.tez.engine.newapi.TezInputContext;
 import org.apache.tez.engine.newapi.TezOutputContext;
-import org.apache.tez.engine.newapi.impl.TezTaskContextImpl;
+import org.apache.tez.engine.newapi.TezTaskContext;
 import org.apache.tez.mapreduce.hadoop.MRConfig;
 import org.apache.tez.mapreduce.hadoop.MRJobConfig;
 import org.apache.tez.mapreduce.hadoop.mapred.MRCounters;
@@ -72,13 +72,13 @@ public class MRCombiner implements Combiner {
   private final MRTaskReporter reporter;
   private final TaskAttemptID mrTaskAttemptID;
 
-  public MRCombiner(TezTaskContextImpl taskContext) throws IOException {
+  public MRCombiner(TezTaskContext taskContext) throws IOException {
     this.conf = TezUtils.createConfFromUserPayload(taskContext.getUserPayload());
 
     assert(taskContext instanceof TezInputContext || taskContext instanceof TezOutputContext);
     if (taskContext instanceof TezOutputContext) {
       this.keyClass = ConfigUtils.getIntermediateOutputKeyClass(conf);
-      this.valClass = ConfigUtils.getIntermediateOutputKeyClass(conf);
+      this.valClass = ConfigUtils.getIntermediateOutputValueClass(conf);
       this.comparator = ConfigUtils.getIntermediateOutputKeyComparator(conf);
       this.reporter = new MRTaskReporter((TezOutputContext)taskContext);
     } else {

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/5eb0c86f/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/newoutput/SimpleOutput.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/newoutput/SimpleOutput.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/newoutput/SimpleOutput.java
index 1dd94e1..a8fb900 100644
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/newoutput/SimpleOutput.java
+++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/newoutput/SimpleOutput.java
@@ -166,7 +166,7 @@ public class SimpleOutput implements LogicalOutput {
     initCommitter(jobConf, useNewApi);
 
     LOG.info("Initialized Simple Output"
-        + ", using_new_api" + useNewApi);
+        + ", using_new_api: " + useNewApi);
     return null;
   }
 


[28/50] [abbrv] Rename tez-engine-api to tez-runtime-api and tez-engine is split into 2: - tez-engine-library for user-visible Input/Output/Processor implementations - tez-engine-internals for framework internals

Posted by ss...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/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
new file mode 100644
index 0000000..ab7e5ba
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/Fetcher.java
@@ -0,0 +1,608 @@
+/**
+ * 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.library.shuffle.common;
+
+import java.io.DataInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.net.HttpURLConnection;
+import java.net.MalformedURLException;
+import java.net.URL;
+import java.net.URLConnection;
+import java.security.GeneralSecurityException;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.Callable;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import javax.crypto.SecretKey;
+import javax.net.ssl.HttpsURLConnection;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.io.compress.CompressionCodec;
+import org.apache.hadoop.io.compress.Decompressor;
+import org.apache.hadoop.security.ssl.SSLFactory;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.tez.common.TezJobConfig;
+import org.apache.tez.runtime.library.common.InputAttemptIdentifier;
+import org.apache.tez.runtime.library.common.security.SecureShuffleUtils;
+import org.apache.tez.runtime.library.common.shuffle.impl.ShuffleHeader;
+import org.apache.tez.runtime.library.common.sort.impl.IFileInputStream;
+import org.apache.tez.runtime.library.shuffle.common.FetchedInput.Type;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * Responsible for fetching inputs served by the ShuffleHandler for a single
+ * host. Construct using {@link FetcherBuilder}
+ */
+public class Fetcher implements Callable<FetchResult> {
+
+  private static final Log LOG = LogFactory.getLog(Fetcher.class);
+
+  private static final int UNIT_CONNECT_TIMEOUT = 60 * 1000;
+  private static final AtomicInteger fetcherIdGen = new AtomicInteger(0);
+
+  // Configurable fields.
+  private CompressionCodec codec;
+  private Decompressor decompressor;
+  private int connectionTimeout;
+  private int readTimeout;
+
+  private final SecretKey shuffleSecret;
+  private final Configuration conf;
+
+  private final FetcherCallback fetcherCallback;
+  private final FetchedInputAllocator inputManager;
+  private final ApplicationId appId;
+
+  private static boolean sslShuffle;
+  private static SSLFactory sslFactory;
+  private static boolean sslFactoryInited;
+
+  private final int fetcherIdentifier;
+  
+  // Parameters to track work.
+  private List<InputAttemptIdentifier> srcAttempts;
+  private String host;
+  private int port;
+  private int partition;
+
+  // Maps from the pathComponents (unique per srcTaskId) to the specific taskId
+  private Map<String, InputAttemptIdentifier> pathToAttemptMap;
+  private Set<InputAttemptIdentifier> remaining;
+
+  private URL url;
+  private String encHash;
+  private String msgToEncode;
+
+  private Fetcher(FetcherCallback fetcherCallback,
+      FetchedInputAllocator inputManager, ApplicationId appId, SecretKey shuffleSecret,
+      Configuration conf) {
+    this.fetcherCallback = fetcherCallback;
+    this.inputManager = inputManager;
+    this.shuffleSecret = shuffleSecret;
+    this.appId = appId;
+    this.conf = conf;
+
+    this.fetcherIdentifier = fetcherIdGen.getAndIncrement();
+    
+    // TODO NEWTEZ Ideally, move this out from here into a static initializer block.
+    synchronized (Fetcher.class) {
+      if (!sslFactoryInited) {
+        sslFactoryInited = true;
+        sslShuffle = conf.getBoolean(
+            TezJobConfig.TEZ_RUNTIME_SHUFFLE_ENABLE_SSL,
+            TezJobConfig.DEFAULT_TEZ_RUNTIME_SHUFFLE_ENABLE_SSL);
+        if (sslShuffle) {
+          sslFactory = new SSLFactory(SSLFactory.Mode.CLIENT, conf);
+          try {
+            sslFactory.init();
+          } catch (Exception ex) {
+            sslFactory.destroy();
+            throw new RuntimeException(ex);
+          }
+        }
+      }
+    }
+  }
+
+  @Override
+  public FetchResult call() throws Exception {
+    if (srcAttempts.size() == 0) {
+      return new FetchResult(host, port, partition, srcAttempts);
+    }
+
+    for (InputAttemptIdentifier in : srcAttempts) {
+      pathToAttemptMap.put(in.getPathComponent(), in);
+    }
+
+    remaining = new HashSet<InputAttemptIdentifier>(srcAttempts);
+
+    HttpURLConnection connection;
+    try {
+      connection = connectToShuffleHandler(host, port, partition, srcAttempts);
+    } catch (IOException e) {
+      // ioErrs.increment(1);
+      // If connect did not succeed, just mark all the maps as failed,
+      // indirectly penalizing the host
+      for (Iterator<InputAttemptIdentifier> leftIter = remaining.iterator(); leftIter
+          .hasNext();) {
+        fetcherCallback.fetchFailed(host, leftIter.next(), true);
+        leftIter.remove();
+      }
+      return new FetchResult(host, port, partition, remaining);
+    }
+
+    DataInputStream input;
+
+    try {
+      input = new DataInputStream(connection.getInputStream());
+      validateConnectionResponse(connection, url, msgToEncode, encHash);
+    } catch (IOException e) {
+      // ioErrs.increment(1);
+      // If we got a read error at this stage, it implies there was a problem
+      // with the first map, typically lost map. So, penalize only that map
+      // and add the rest
+      InputAttemptIdentifier firstAttempt = srcAttempts.get(0);
+      fetcherCallback.fetchFailed(host, firstAttempt, false);
+      remaining.remove(firstAttempt);
+      return new FetchResult(host, port, partition, remaining);
+    }
+
+    // By this point, the connection is setup and the response has been
+    // validated.
+
+    // Loop through available map-outputs and fetch them
+    // On any error, faildTasks is not null and we exit
+    // after putting back the remaining maps to the
+    // yet_to_be_fetched list and marking the failed tasks.
+    InputAttemptIdentifier[] failedInputs = null;
+    while (!remaining.isEmpty() && failedInputs == null) {
+      failedInputs = fetchInputs(input);
+    }
+
+    if (failedInputs != null && failedInputs.length > 0) {
+      LOG.warn("copyInputs failed for tasks " + Arrays.toString(failedInputs));
+      for (InputAttemptIdentifier left : failedInputs) {
+        fetcherCallback.fetchFailed(host, left, false);
+        remaining.remove(left);
+      }
+    }
+
+    IOUtils.cleanup(LOG, input);
+
+    // Sanity check
+    if (failedInputs == null && !remaining.isEmpty()) {
+      throw new IOException("server didn't return all expected map outputs: "
+          + remaining.size() + " left.");
+    }
+
+    return new FetchResult(host, port, partition, remaining);
+
+  }
+
+  private InputAttemptIdentifier[] fetchInputs(DataInputStream input) {
+    FetchedInput fetchedInput = null;
+    InputAttemptIdentifier srcAttemptId = null;
+    long decompressedLength = -1;
+    long compressedLength = -1;
+
+    try {
+      long startTime = System.currentTimeMillis();
+      int responsePartition = -1;
+      // Read the shuffle header
+      try {
+        ShuffleHeader header = new ShuffleHeader();
+        header.readFields(input);
+        String pathComponent = header.getMapId();
+
+        srcAttemptId = pathToAttemptMap.get(pathComponent);
+        compressedLength = header.getCompressedLength();
+        decompressedLength = header.getUncompressedLength();
+        responsePartition = header.getPartition();
+      } catch (IllegalArgumentException e) {
+        // badIdErrs.increment(1);
+        LOG.warn("Invalid src id ", e);
+        // Don't know which one was bad, so consider all of them as bad
+        return remaining.toArray(new InputAttemptIdentifier[remaining.size()]);
+      }
+
+      // Do some basic sanity verification
+      if (!verifySanity(compressedLength, decompressedLength,
+          responsePartition, srcAttemptId)) {
+        return new InputAttemptIdentifier[] { srcAttemptId };
+      }
+
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("header: " + srcAttemptId + ", len: " + compressedLength
+            + ", decomp len: " + decompressedLength);
+      }
+
+      // Get the location for the map output - either in-memory or on-disk
+      fetchedInput = inputManager.allocate(decompressedLength, srcAttemptId);
+
+      // TODO NEWTEZ No concept of WAIT at the moment.
+      // // Check if we can shuffle *now* ...
+      // if (fetchedInput.getType() == FetchedInput.WAIT) {
+      // LOG.info("fetcher#" + id +
+      // " - MergerManager returned Status.WAIT ...");
+      // //Not an error but wait to process data.
+      // return EMPTY_ATTEMPT_ID_ARRAY;
+      // }
+
+      // Go!
+      LOG.info("fetcher" + " about to shuffle output of srcAttempt "
+          + fetchedInput.getInputAttemptIdentifier() + " decomp: "
+          + decompressedLength + " len: " + compressedLength + " to "
+          + fetchedInput.getType());
+
+      if (fetchedInput.getType() == Type.MEMORY) {
+        shuffleToMemory((MemoryFetchedInput) fetchedInput, input,
+            (int) decompressedLength, (int) compressedLength);
+      } else {
+        shuffleToDisk((DiskFetchedInput) fetchedInput, input, compressedLength);
+      }
+
+      // Inform the shuffle scheduler
+      long endTime = System.currentTimeMillis();
+      fetcherCallback.fetchSucceeded(host, srcAttemptId, fetchedInput,
+          compressedLength, (endTime - startTime));
+
+      // Note successful shuffle
+      remaining.remove(srcAttemptId);
+      // metrics.successFetch();
+      return null;
+    } catch (IOException ioe) {
+      // ioErrs.increment(1);
+      if (srcAttemptId == null || fetchedInput == null) {
+        LOG.info("fetcher" + " failed to read map header" + srcAttemptId
+            + " decomp: " + decompressedLength + ", " + compressedLength, ioe);
+        if (srcAttemptId == null) {
+          return remaining
+              .toArray(new InputAttemptIdentifier[remaining.size()]);
+        } else {
+          return new InputAttemptIdentifier[] { srcAttemptId };
+        }
+      }
+      LOG.warn("Failed to shuffle output of " + srcAttemptId + " from " + host,
+          ioe);
+
+      // Inform the shuffle-scheduler
+      try {
+        fetchedInput.abort();
+      } catch (IOException e) {
+        LOG.info("Failure to cleanup fetchedInput: " + fetchedInput);
+      }
+      // metrics.failedFetch();
+      return new InputAttemptIdentifier[] { srcAttemptId };
+    }
+  }
+
+  @SuppressWarnings("resource")
+  private void shuffleToMemory(MemoryFetchedInput fetchedInput,
+      InputStream input, int decompressedLength, int compressedLength)
+      throws IOException {
+    IFileInputStream checksumIn = new IFileInputStream(input, compressedLength,
+        conf);
+
+    input = checksumIn;
+
+    // Are map-outputs compressed?
+    if (codec != null) {
+      decompressor.reset();
+      input = codec.createInputStream(input, decompressor);
+    }
+    // Copy map-output into an in-memory buffer
+    byte[] shuffleData = fetchedInput.getBytes();
+
+    try {
+      IOUtils.readFully(input, shuffleData, 0, shuffleData.length);
+      // metrics.inputBytes(shuffleData.length);
+      LOG.info("Read " + shuffleData.length + " bytes from input for "
+          + fetchedInput.getInputAttemptIdentifier());
+    } catch (IOException ioe) {
+      // Close the streams
+      IOUtils.cleanup(LOG, input);
+      // Re-throw
+      throw ioe;
+    }
+  }
+
+  private void shuffleToDisk(DiskFetchedInput fetchedInput, InputStream input,
+      long compressedLength) throws IOException {
+    // Copy data to local-disk
+    OutputStream output = fetchedInput.getOutputStream();
+    long bytesLeft = compressedLength;
+    try {
+      final int BYTES_TO_READ = 64 * 1024;
+      byte[] buf = new byte[BYTES_TO_READ];
+      while (bytesLeft > 0) {
+        int n = input.read(buf, 0, (int) Math.min(bytesLeft, BYTES_TO_READ));
+        if (n < 0) {
+          throw new IOException("read past end of stream reading "
+              + fetchedInput.getInputAttemptIdentifier());
+        }
+        output.write(buf, 0, n);
+        bytesLeft -= n;
+        // metrics.inputBytes(n);
+      }
+
+      LOG.info("Read " + (compressedLength - bytesLeft)
+          + " bytes from input for " + fetchedInput.getInputAttemptIdentifier());
+
+      output.close();
+    } catch (IOException ioe) {
+      // Close the streams
+      IOUtils.cleanup(LOG, input, output);
+
+      // Re-throw
+      throw ioe;
+    }
+
+    // Sanity check
+    if (bytesLeft != 0) {
+      throw new IOException("Incomplete input received for "
+          + fetchedInput.getInputAttemptIdentifier() + " from " + host + " ("
+          + bytesLeft + " bytes missing of " + compressedLength + ")");
+    }
+  }
+
+  /**
+   * Do some basic verification on the input received -- Being defensive
+   * 
+   * @param compressedLength
+   * @param decompressedLength
+   * @param fetchPartition
+   * @param remaining
+   * @param mapId
+   * @return true/false, based on if the verification succeeded or not
+   */
+  private boolean verifySanity(long compressedLength, long decompressedLength,
+      int fetchPartition, InputAttemptIdentifier srcAttemptId) {
+    if (compressedLength < 0 || decompressedLength < 0) {
+      // wrongLengthErrs.increment(1);
+      LOG.warn(" invalid lengths in input header: id: " + srcAttemptId
+          + " len: " + compressedLength + ", decomp len: " + decompressedLength);
+      return false;
+    }
+
+    if (fetchPartition != this.partition) {
+      // wrongReduceErrs.increment(1);
+      LOG.warn(" data for the wrong reduce map: " + srcAttemptId + " len: "
+          + compressedLength + " decomp len: " + decompressedLength
+          + " for reduce " + fetchPartition);
+      return false;
+    }
+
+    // Sanity check
+    if (!remaining.contains(srcAttemptId)) {
+      // wrongMapErrs.increment(1);
+      LOG.warn("Invalid input. Received output for " + srcAttemptId);
+      return false;
+    }
+    return true;
+  }
+
+  private HttpURLConnection connectToShuffleHandler(String host, int port,
+      int partition, List<InputAttemptIdentifier> inputs) throws IOException {
+    try {
+      this.url = constructInputURL(host, port, partition, inputs);
+      HttpURLConnection connection = openConnection(url);
+
+      // generate hash of the url
+      this.msgToEncode = SecureShuffleUtils.buildMsgFrom(url);
+      this.encHash = SecureShuffleUtils.hashFromString(msgToEncode,
+          shuffleSecret);
+
+      // put url hash into http header
+      connection.addRequestProperty(SecureShuffleUtils.HTTP_HEADER_URL_HASH,
+          encHash);
+      // set the read timeout
+      connection.setReadTimeout(readTimeout);
+      // put shuffle version into http header
+      connection.addRequestProperty(ShuffleHeader.HTTP_HEADER_NAME,
+          ShuffleHeader.DEFAULT_HTTP_HEADER_NAME);
+      connection.addRequestProperty(ShuffleHeader.HTTP_HEADER_VERSION,
+          ShuffleHeader.DEFAULT_HTTP_HEADER_VERSION);
+
+      connect(connection, connectionTimeout);
+      return connection;
+    } catch (IOException e) {
+      LOG.warn("Failed to connect to " + host + " with " + srcAttempts.size()
+          + " inputs", e);
+      throw e;
+    }
+  }
+
+  private void validateConnectionResponse(HttpURLConnection connection,
+      URL url, String msgToEncode, String encHash) throws IOException {
+    int rc = connection.getResponseCode();
+    if (rc != HttpURLConnection.HTTP_OK) {
+      throw new IOException("Got invalid response code " + rc + " from " + url
+          + ": " + connection.getResponseMessage());
+    }
+
+    if (!ShuffleHeader.DEFAULT_HTTP_HEADER_NAME.equals(connection
+        .getHeaderField(ShuffleHeader.HTTP_HEADER_NAME))
+        || !ShuffleHeader.DEFAULT_HTTP_HEADER_VERSION.equals(connection
+            .getHeaderField(ShuffleHeader.HTTP_HEADER_VERSION))) {
+      throw new IOException("Incompatible shuffle response version");
+    }
+
+    // get the replyHash which is HMac of the encHash we sent to the server
+    String replyHash = connection
+        .getHeaderField(SecureShuffleUtils.HTTP_HEADER_REPLY_URL_HASH);
+    if (replyHash == null) {
+      throw new IOException("security validation of TT Map output failed");
+    }
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("url=" + msgToEncode + ";encHash=" + encHash + ";replyHash="
+          + replyHash);
+    }
+    // verify that replyHash is HMac of encHash
+    SecureShuffleUtils.verifyReply(replyHash, encHash, shuffleSecret);
+    LOG.info("for url=" + msgToEncode + " sent hash and receievd reply");
+  }
+
+  protected HttpURLConnection openConnection(URL url) throws IOException {
+    HttpURLConnection conn = (HttpURLConnection) url.openConnection();
+    if (sslShuffle) {
+      HttpsURLConnection httpsConn = (HttpsURLConnection) conn;
+      try {
+        httpsConn.setSSLSocketFactory(sslFactory.createSSLSocketFactory());
+      } catch (GeneralSecurityException ex) {
+        throw new IOException(ex);
+      }
+      httpsConn.setHostnameVerifier(sslFactory.getHostnameVerifier());
+    }
+    return conn;
+  }
+
+  /**
+   * The connection establishment is attempted multiple times and is given up
+   * only on the last failure. Instead of connecting with a timeout of X, we try
+   * connecting with a timeout of x < X but multiple times.
+   */
+  private void connect(URLConnection connection, int connectionTimeout)
+      throws IOException {
+    int unit = 0;
+    if (connectionTimeout < 0) {
+      throw new IOException("Invalid timeout " + "[timeout = "
+          + connectionTimeout + " ms]");
+    } else if (connectionTimeout > 0) {
+      unit = Math.min(UNIT_CONNECT_TIMEOUT, connectionTimeout);
+    }
+    // set the connect timeout to the unit-connect-timeout
+    connection.setConnectTimeout(unit);
+    while (true) {
+      try {
+        connection.connect();
+        break;
+      } catch (IOException ioe) {
+        // update the total remaining connect-timeout
+        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) {
+          throw ioe;
+        }
+
+        // reset the connect timeout for the last try
+        if (connectionTimeout < unit) {
+          unit = connectionTimeout;
+          // reset the connect time out for the final connect
+          connection.setConnectTimeout(unit);
+        }
+      }
+    }
+  }
+
+  private URL constructInputURL(String host, int port, int partition,
+      List<InputAttemptIdentifier> inputs) throws MalformedURLException {
+    StringBuilder url = ShuffleUtils.constructBaseURIForShuffleHandler(host,
+        port, partition, appId);
+    boolean first = true;
+    for (InputAttemptIdentifier input : inputs) {
+      if (first) {
+        first = false;
+        url.append(input.getPathComponent());
+      } else {
+        url.append(",").append(input.getPathComponent());
+      }
+    }
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("InputFetch URL for: " + host + " : " + url.toString());
+    }
+    return new URL(url.toString());
+  }
+
+  /**
+   * Builder for the construction of Fetchers
+   */
+  public static class FetcherBuilder {
+    private Fetcher fetcher;
+    private boolean workAssigned = false;
+
+    public FetcherBuilder(FetcherCallback fetcherCallback,
+        FetchedInputAllocator inputManager, ApplicationId appId,
+        SecretKey shuffleSecret, Configuration conf) {
+      this.fetcher = new Fetcher(fetcherCallback, inputManager, appId,
+          shuffleSecret, conf);
+    }
+
+    public FetcherBuilder setCompressionParameters(CompressionCodec codec,
+        Decompressor decompressor) {
+      fetcher.codec = codec;
+      fetcher.decompressor = decompressor;
+      return this;
+    }
+
+    public FetcherBuilder setConnectionParameters(int connectionTimeout,
+        int readTimeout) {
+      fetcher.connectionTimeout = connectionTimeout;
+      fetcher.readTimeout = readTimeout;
+      return this;
+    }
+
+    public FetcherBuilder assignWork(String host, int port, int partition,
+        List<InputAttemptIdentifier> inputs) {
+      fetcher.host = host;
+      fetcher.port = port;
+      fetcher.partition = partition;
+      fetcher.srcAttempts = inputs;
+      workAssigned = true;
+      return this;
+    }
+
+    public Fetcher build() {
+      Preconditions.checkState(workAssigned == true,
+          "Cannot build a fetcher withot assigning work to it");
+      return fetcher;
+    }
+  }
+
+  @Override
+  public int hashCode() {
+    return fetcherIdentifier;
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (this == obj)
+      return true;
+    if (obj == null)
+      return false;
+    if (getClass() != obj.getClass())
+      return false;
+    Fetcher other = (Fetcher) obj;
+    if (fetcherIdentifier != other.fetcherIdentifier)
+      return false;
+    return true;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/FetcherCallback.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/FetcherCallback.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/FetcherCallback.java
new file mode 100644
index 0000000..18504b1
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/FetcherCallback.java
@@ -0,0 +1,31 @@
+/**
+ * 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.library.shuffle.common;
+
+import java.io.IOException;
+
+import org.apache.tez.runtime.library.common.InputAttemptIdentifier;
+
+public interface FetcherCallback {
+
+  public void fetchSucceeded(String host, InputAttemptIdentifier srcAttemptIdentifier, FetchedInput fetchedInput, long fetchedBytes, long copyDuration) throws IOException;
+  
+  public void fetchFailed(String host, InputAttemptIdentifier srcAttemptIdentifier, boolean connectFailed);
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/InputHost.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/InputHost.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/InputHost.java
new file mode 100644
index 0000000..66605dd
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/InputHost.java
@@ -0,0 +1,90 @@
+/**
+ * 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.library.shuffle.common;
+
+import java.util.ArrayList;
+import java.util.LinkedList;
+import java.util.List;
+
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.tez.runtime.library.common.InputAttemptIdentifier;
+
+public class InputHost {
+
+  private final String host;
+  private final int port;
+
+  private final List<InputAttemptIdentifier> inputs = new LinkedList<InputAttemptIdentifier>();
+
+  public InputHost(String hostName, int port, ApplicationId appId) {
+    this.host = hostName;
+    this.port = port;
+  }
+
+  public String getHost() {
+    return this.host;
+  }
+
+  public int getPort() {
+    return this.port;
+  }
+
+  public synchronized int getNumPendingInputs() {
+    return inputs.size();
+  }
+  
+  public synchronized void addKnownInput(InputAttemptIdentifier srcAttempt) {
+    inputs.add(srcAttempt);
+  }
+
+  public synchronized List<InputAttemptIdentifier> clearAndGetPendingInputs() {
+    List<InputAttemptIdentifier> inputsCopy = new ArrayList<InputAttemptIdentifier>(
+        inputs);
+    inputs.clear();
+    return inputsCopy;
+  }
+
+  @Override
+  public int hashCode() {
+    final int prime = 31;
+    int result = 1;
+    result = prime * result + ((host == null) ? 0 : host.hashCode());
+    result = prime * result + port;
+    return result;
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (this == obj)
+      return true;
+    if (obj == null)
+      return false;
+    if (getClass() != obj.getClass())
+      return false;
+    InputHost other = (InputHost) obj;
+    if (host == null) {
+      if (other.host != null)
+        return false;
+    } else if (!host.equals(other.host))
+      return false;
+    if (port != other.port)
+      return false;
+    return true;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/MemoryFetchedInput.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/MemoryFetchedInput.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/MemoryFetchedInput.java
new file mode 100644
index 0000000..f56877f
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/MemoryFetchedInput.java
@@ -0,0 +1,89 @@
+/**
+ * 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.library.shuffle.common;
+
+import java.io.ByteArrayInputStream;
+import java.io.InputStream;
+import java.io.OutputStream;
+
+import org.apache.hadoop.io.BoundedByteArrayOutputStream;
+import org.apache.tez.runtime.library.common.InputAttemptIdentifier;
+
+import com.google.common.base.Preconditions;
+
+public class MemoryFetchedInput extends FetchedInput {
+
+  private BoundedByteArrayOutputStream byteStream;
+
+  public MemoryFetchedInput(long size,
+      InputAttemptIdentifier inputAttemptIdentifier,
+      FetchedInputCallback callbackHandler) {
+    super(Type.MEMORY, size, inputAttemptIdentifier, callbackHandler);
+    this.byteStream = new BoundedByteArrayOutputStream((int) size);
+  }
+
+  @Override
+  public OutputStream getOutputStream() {
+    return byteStream;
+  }
+
+  @Override
+  public InputStream getInputStream() {
+    return new ByteArrayInputStream(byteStream.getBuffer());
+  }
+
+  public byte[] getBytes() {
+    return byteStream.getBuffer();
+  }
+  
+  @Override
+  public void commit() {
+    if (state == State.PENDING) {
+      state = State.COMMITTED;
+      notifyFetchComplete();
+    }
+  }
+
+  @Override
+  public void abort() {
+    if (state == State.PENDING) {
+      state = State.ABORTED;
+      notifyFetchFailure();
+    }
+  }
+  
+  @Override
+  public void free() {
+    Preconditions.checkState(
+        state == State.COMMITTED || state == State.ABORTED,
+        "FetchedInput can only be freed after it is committed or aborted");
+    if (state == State.COMMITTED) {
+      state = State.FREED;
+      this.byteStream = null;
+      notifyFreedResource();
+    }
+  }
+
+  @Override
+  public String toString() {
+    return "MemoryFetchedInput [inputAttemptIdentifier="
+        + inputAttemptIdentifier + ", size=" + size + ", type=" + type
+        + ", id=" + id + ", state=" + state + "]";
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/ShuffleUtils.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/ShuffleUtils.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/ShuffleUtils.java
new file mode 100644
index 0000000..7479e7e
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/ShuffleUtils.java
@@ -0,0 +1,81 @@
+/**
+ * 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.library.shuffle.common;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import javax.crypto.SecretKey;
+
+import org.apache.hadoop.io.DataInputByteBuffer;
+import org.apache.hadoop.io.DataOutputBuffer;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.tez.runtime.library.common.security.JobTokenIdentifier;
+import org.apache.tez.runtime.library.common.security.JobTokenSecretManager;
+
+public class ShuffleUtils {
+
+  public static String SHUFFLE_HANDLER_SERVICE_ID = "mapreduce.shuffle";
+
+  public static SecretKey getJobTokenSecretFromTokenBytes(ByteBuffer meta)
+      throws IOException {
+    DataInputByteBuffer in = new DataInputByteBuffer();
+    in.reset(meta);
+    Token<JobTokenIdentifier> jt = new Token<JobTokenIdentifier>();
+    jt.readFields(in);
+    SecretKey sk = JobTokenSecretManager.createSecretKey(jt.getPassword());
+    return sk;
+  }
+
+  public static ByteBuffer convertJobTokenToBytes(
+      Token<JobTokenIdentifier> jobToken) throws IOException {
+    DataOutputBuffer dob = new DataOutputBuffer();
+    jobToken.write(dob);
+    ByteBuffer bb = ByteBuffer.wrap(dob.getData(), 0, dob.getLength());
+    return bb;
+  }
+
+  public static int deserializeShuffleProviderMetaData(ByteBuffer meta)
+      throws IOException {
+    DataInputByteBuffer in = new DataInputByteBuffer();
+    try {
+      in.reset(meta);
+      int port = in.readInt();
+      return port;
+    } finally {
+      in.close();
+    }
+  }
+  
+  // TODO NEWTEZ handle ssl shuffle
+  public static StringBuilder constructBaseURIForShuffleHandler(String host, int port, int partition, ApplicationId appId) {
+    StringBuilder sb = new StringBuilder("http://");
+    sb.append(host);
+    sb.append(":");
+    sb.append(String.valueOf(port));
+    sb.append("/");
+    sb.append("mapOutput?job=");
+    sb.append(appId.toString().replace("application", "job"));
+    sb.append("&reduce=");
+    sb.append(String.valueOf(partition));
+    sb.append("&map=");
+    return sb;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/proto/ShufflePayloads.proto
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/proto/ShufflePayloads.proto b/tez-runtime-library/src/main/proto/ShufflePayloads.proto
new file mode 100644
index 0000000..1fb000f
--- /dev/null
+++ b/tez-runtime-library/src/main/proto/ShufflePayloads.proto
@@ -0,0 +1,33 @@
+/**
+ * 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.
+ */
+
+option java_package = "org.apache.tez.runtime.library.shuffle.impl";
+option java_outer_classname = "ShuffleUserPayloads";
+option java_generate_equals_and_hash = true;
+
+message DataMovementEventPayloadProto {
+  optional bool output_generated = 1;
+  optional string host = 2;
+  optional int32 port = 3;
+  optional string path_component = 4;
+  optional int32 run_duration = 5;
+} 
+
+message InputInformationEventPayloadProto {
+  optional int32 partition_range = 1;
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-yarn-client/src/main/java/org/apache/tez/mapreduce/YARNRunner.java
----------------------------------------------------------------------
diff --git a/tez-yarn-client/src/main/java/org/apache/tez/mapreduce/YARNRunner.java b/tez-yarn-client/src/main/java/org/apache/tez/mapreduce/YARNRunner.java
index 6496b55..a6d250f 100644
--- a/tez-yarn-client/src/main/java/org/apache/tez/mapreduce/YARNRunner.java
+++ b/tez-yarn-client/src/main/java/org/apache/tez/mapreduce/YARNRunner.java
@@ -95,8 +95,6 @@ import org.apache.tez.dag.api.Vertex;
 import org.apache.tez.dag.api.VertexLocationHint.TaskLocationHint;
 import org.apache.tez.dag.api.client.DAGClient;
 import org.apache.tez.dag.api.client.DAGStatus;
-import org.apache.tez.engine.lib.input.ShuffledMergedInputLegacy;
-import org.apache.tez.engine.lib.output.OnFileSortedOutput;
 import org.apache.tez.mapreduce.hadoop.DeprecatedKeys;
 import org.apache.tez.mapreduce.hadoop.MRHelpers;
 import org.apache.tez.mapreduce.hadoop.MRJobConfig;
@@ -104,6 +102,8 @@ import org.apache.tez.mapreduce.hadoop.MultiStageMRConfToTezTranslator;
 import org.apache.tez.mapreduce.hadoop.MultiStageMRConfigUtil;
 import org.apache.tez.mapreduce.processor.map.MapProcessor;
 import org.apache.tez.mapreduce.processor.reduce.ReduceProcessor;
+import org.apache.tez.runtime.library.input.ShuffledMergedInputLegacy;
+import org.apache.tez.runtime.library.output.OnFileSortedOutput;
 
 import com.google.common.annotations.VisibleForTesting;
 


[42/50] [abbrv] Rename tez-engine-api to tez-runtime-api and tez-engine is split into 2: - tez-engine-library for user-visible Input/Output/Processor implementations - tez-engine-internals for framework internals

Posted by ss...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/MapOutput.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/MapOutput.java b/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/MapOutput.java
deleted file mode 100644
index b2a0b54..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/MapOutput.java
+++ /dev/null
@@ -1,227 +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.engine.common.shuffle.impl;
-
-import java.io.IOException;
-import java.io.OutputStream;
-import java.util.Comparator;
-import java.util.concurrent.atomic.AtomicInteger;
-
-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.io.BoundedByteArrayOutputStream;
-import org.apache.tez.engine.common.InputAttemptIdentifier;
-import org.apache.tez.engine.common.task.local.output.TezTaskOutputFiles;
-
-
-class MapOutput {
-  private static final Log LOG = LogFactory.getLog(MapOutput.class);
-  private static AtomicInteger ID = new AtomicInteger(0);
-  
-  public static enum Type {
-    WAIT,
-    MEMORY,
-    DISK
-  }
-  
-  private InputAttemptIdentifier attemptIdentifier;
-  private final int id;
-  
-  private final MergeManager merger;
-  
-  private final long size;
-  
-  private final byte[] memory;
-  private BoundedByteArrayOutputStream byteStream;
-  
-  private final FileSystem localFS;
-  private final Path tmpOutputPath;
-  private final Path outputPath;
-  private final OutputStream disk; 
-  
-  private final Type type;
-  
-  private final boolean primaryMapOutput;
-  
-  MapOutput(InputAttemptIdentifier attemptIdentifier, MergeManager merger, long size, 
-            Configuration conf, LocalDirAllocator localDirAllocator,
-            int fetcher, boolean primaryMapOutput, 
-            TezTaskOutputFiles mapOutputFile)
-         throws IOException {
-    this.id = ID.incrementAndGet();
-    this.attemptIdentifier = attemptIdentifier;
-    this.merger = merger;
-
-    type = Type.DISK;
-
-    memory = null;
-    byteStream = null;
-
-    this.size = size;
-    
-    this.localFS = FileSystem.getLocal(conf);
-    outputPath =
-      mapOutputFile.getInputFileForWrite(this.attemptIdentifier.getInputIdentifier().getSrcTaskIndex(), size);
-    tmpOutputPath = outputPath.suffix(String.valueOf(fetcher));
-
-    disk = localFS.create(tmpOutputPath);
-    
-    this.primaryMapOutput = primaryMapOutput;
-  }
-  
-  MapOutput(InputAttemptIdentifier attemptIdentifier, MergeManager merger, int size, 
-            boolean primaryMapOutput) {
-    this.id = ID.incrementAndGet();
-    this.attemptIdentifier = attemptIdentifier;
-    this.merger = merger;
-
-    type = Type.MEMORY;
-    byteStream = new BoundedByteArrayOutputStream(size);
-    memory = byteStream.getBuffer();
-
-    this.size = size;
-    
-    localFS = null;
-    disk = null;
-    outputPath = null;
-    tmpOutputPath = null;
-    
-    this.primaryMapOutput = primaryMapOutput;
-  }
-
-  public MapOutput(InputAttemptIdentifier attemptIdentifier) {
-    this.id = ID.incrementAndGet();
-    this.attemptIdentifier = attemptIdentifier;
-
-    type = Type.WAIT;
-    merger = null;
-    memory = null;
-    byteStream = null;
-    
-    size = -1;
-    
-    localFS = null;
-    disk = null;
-    outputPath = null;
-    tmpOutputPath = null;
-
-    this.primaryMapOutput = false;
-}
-  
-  public boolean isPrimaryMapOutput() {
-    return primaryMapOutput;
-  }
-
-  @Override
-  public boolean equals(Object obj) {
-    if (obj instanceof MapOutput) {
-      return id == ((MapOutput)obj).id;
-    }
-    return false;
-  }
-
-  @Override
-  public int hashCode() {
-    return id;
-  }
-
-  public Path getOutputPath() {
-    return outputPath;
-  }
-
-  public byte[] getMemory() {
-    return memory;
-  }
-
-  public BoundedByteArrayOutputStream getArrayStream() {
-    return byteStream;
-  }
-  
-  public OutputStream getDisk() {
-    return disk;
-  }
-
-  public InputAttemptIdentifier getAttemptIdentifier() {
-    return this.attemptIdentifier;
-  }
-
-  public Type getType() {
-    return type;
-  }
-
-  public long getSize() {
-    return size;
-  }
-
-  public void commit() throws IOException {
-    if (type == Type.MEMORY) {
-      merger.closeInMemoryFile(this);
-    } else if (type == Type.DISK) {
-      localFS.rename(tmpOutputPath, outputPath);
-      merger.closeOnDiskFile(outputPath);
-    } else {
-      throw new IOException("Cannot commit MapOutput of type WAIT!");
-    }
-  }
-  
-  public void abort() {
-    if (type == Type.MEMORY) {
-      merger.unreserve(memory.length);
-    } else if (type == Type.DISK) {
-      try {
-        localFS.delete(tmpOutputPath, false);
-      } catch (IOException ie) {
-        LOG.info("failure to clean up " + tmpOutputPath, ie);
-      }
-    } else {
-      throw new IllegalArgumentException
-                   ("Cannot commit MapOutput with of type WAIT!");
-    }
-  }
-  
-  public String toString() {
-    return "MapOutput( AttemptIdentifier: " + attemptIdentifier + ", Type: " + type + ")";
-  }
-  
-  public static class MapOutputComparator 
-  implements Comparator<MapOutput> {
-    public int compare(MapOutput o1, MapOutput o2) {
-      if (o1.id == o2.id) { 
-        return 0;
-      }
-      
-      if (o1.size < o2.size) {
-        return -1;
-      } else if (o1.size > o2.size) {
-        return 1;
-      }
-      
-      if (o1.id < o2.id) {
-        return -1;
-      } else {
-        return 1;
-      
-      }
-    }
-  }
-  
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/MergeManager.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/MergeManager.java b/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/MergeManager.java
deleted file mode 100644
index b8792fb..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/MergeManager.java
+++ /dev/null
@@ -1,782 +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.engine.common.shuffle.impl;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.List;
-import java.util.Set;
-import java.util.TreeSet;
-
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.classification.InterfaceAudience.Private;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.ChecksumFileSystem;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.LocalDirAllocator;
-import org.apache.hadoop.fs.LocalFileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.DataInputBuffer;
-import org.apache.hadoop.io.RawComparator;
-import org.apache.hadoop.io.compress.CompressionCodec;
-import org.apache.hadoop.io.compress.DefaultCodec;
-import org.apache.hadoop.util.Progressable;
-import org.apache.hadoop.util.ReflectionUtils;
-import org.apache.tez.common.Constants;
-import org.apache.tez.common.TezJobConfig;
-import org.apache.tez.common.counters.TezCounter;
-import org.apache.tez.engine.api.TezInputContext;
-import org.apache.tez.engine.common.ConfigUtils;
-import org.apache.tez.engine.common.InputAttemptIdentifier;
-import org.apache.tez.engine.common.combine.Combiner;
-import org.apache.tez.engine.common.sort.impl.IFile;
-import org.apache.tez.engine.common.sort.impl.TezMerger;
-import org.apache.tez.engine.common.sort.impl.TezRawKeyValueIterator;
-import org.apache.tez.engine.common.sort.impl.IFile.Writer;
-import org.apache.tez.engine.common.sort.impl.TezMerger.Segment;
-import org.apache.tez.engine.common.task.local.output.TezTaskOutputFiles;
-import org.apache.tez.engine.hadoop.compat.NullProgressable;
-
-@InterfaceAudience.Private
-@InterfaceStability.Unstable
-@SuppressWarnings(value={"rawtypes"})
-public class MergeManager {
-  
-  private static final Log LOG = LogFactory.getLog(MergeManager.class);
-
-  private final Configuration conf;
-  private final FileSystem localFS;
-  private final FileSystem rfs;
-  private final LocalDirAllocator localDirAllocator;
-  
-  private final  TezTaskOutputFiles mapOutputFile;
-  private final Progressable nullProgressable = new NullProgressable();
-  private final Combiner combiner;  
-  
-  Set<MapOutput> inMemoryMergedMapOutputs = 
-    new TreeSet<MapOutput>(new MapOutput.MapOutputComparator());
-  private final IntermediateMemoryToMemoryMerger memToMemMerger;
-
-  Set<MapOutput> inMemoryMapOutputs = 
-    new TreeSet<MapOutput>(new MapOutput.MapOutputComparator());
-  private final InMemoryMerger inMemoryMerger;
-  
-  Set<Path> onDiskMapOutputs = new TreeSet<Path>();
-  private final OnDiskMerger onDiskMerger;
-  
-  private final long memoryLimit;
-  private long usedMemory;
-  private long commitMemory;
-  private final long maxSingleShuffleLimit;
-  
-  private final int memToMemMergeOutputsThreshold; 
-  private final long mergeThreshold;
-  
-  private final int ioSortFactor;
-
-  private final ExceptionReporter exceptionReporter;
-  
-  private final TezInputContext inputContext;
-
-  private final TezCounter spilledRecordsCounter;
-
-  private final TezCounter reduceCombineInputCounter;
-
-  private final TezCounter mergedMapOutputsCounter;
-  
-  private final CompressionCodec codec;
-  
-  private volatile boolean finalMergeComplete = false;
-
-  public MergeManager(Configuration conf, 
-                      FileSystem localFS,
-                      LocalDirAllocator localDirAllocator,  
-                      TezInputContext inputContext,
-                      Combiner combiner,
-                      TezCounter spilledRecordsCounter,
-                      TezCounter reduceCombineInputCounter,
-                      TezCounter mergedMapOutputsCounter,
-                      ExceptionReporter exceptionReporter) {
-    this.inputContext = inputContext;
-    this.conf = conf;
-    this.localDirAllocator = localDirAllocator;
-    this.exceptionReporter = exceptionReporter;
-    
-    this.combiner = combiner;
-
-    this.reduceCombineInputCounter = reduceCombineInputCounter;
-    this.spilledRecordsCounter = spilledRecordsCounter;
-    this.mergedMapOutputsCounter = mergedMapOutputsCounter;
-    this.mapOutputFile = new TezTaskOutputFiles(conf, inputContext.getUniqueIdentifier());
-    
-    this.localFS = localFS;
-    this.rfs = ((LocalFileSystem)localFS).getRaw();
-
-    if (ConfigUtils.isIntermediateInputCompressed(conf)) {
-      Class<? extends CompressionCodec> codecClass =
-          ConfigUtils.getIntermediateInputCompressorClass(conf, DefaultCodec.class);
-      codec = ReflectionUtils.newInstance(codecClass, conf);
-    } else {
-      codec = null;
-    }
-
-    final float maxInMemCopyUse =
-      conf.getFloat(
-          TezJobConfig.TEZ_ENGINE_SHUFFLE_INPUT_BUFFER_PERCENT, 
-          TezJobConfig.DEFAULT_TEZ_ENGINE_SHUFFLE_INPUT_BUFFER_PERCENT);
-    if (maxInMemCopyUse > 1.0 || maxInMemCopyUse < 0.0) {
-      throw new IllegalArgumentException("Invalid value for " +
-          TezJobConfig.TEZ_ENGINE_SHUFFLE_INPUT_BUFFER_PERCENT + ": " +
-          maxInMemCopyUse);
-    }
-
-    // Allow unit tests to fix Runtime memory
-    this.memoryLimit = 
-      (long)(conf.getLong(Constants.TEZ_ENGINE_TASK_MEMORY,
-          Math.min(Runtime.getRuntime().maxMemory(), Integer.MAX_VALUE))
-        * maxInMemCopyUse);
- 
-    this.ioSortFactor = 
-        conf.getInt(
-            TezJobConfig.TEZ_ENGINE_IO_SORT_FACTOR, 
-            TezJobConfig.DEFAULT_TEZ_ENGINE_IO_SORT_FACTOR);
-
-    final float singleShuffleMemoryLimitPercent =
-        conf.getFloat(
-            TezJobConfig.TEZ_ENGINE_SHUFFLE_MEMORY_LIMIT_PERCENT,
-            TezJobConfig.DEFAULT_TEZ_ENGINE_SHUFFLE_MEMORY_LIMIT_PERCENT);
-    if (singleShuffleMemoryLimitPercent <= 0.0f
-        || singleShuffleMemoryLimitPercent > 1.0f) {
-      throw new IllegalArgumentException("Invalid value for "
-          + TezJobConfig.TEZ_ENGINE_SHUFFLE_MEMORY_LIMIT_PERCENT + ": "
-          + singleShuffleMemoryLimitPercent);
-    }
-
-    this.maxSingleShuffleLimit = 
-      (long)(memoryLimit * singleShuffleMemoryLimitPercent);
-    this.memToMemMergeOutputsThreshold = 
-            conf.getInt(
-                TezJobConfig.TEZ_ENGINE_SHUFFLE_MEMTOMEM_SEGMENTS, 
-                ioSortFactor);
-    this.mergeThreshold = 
-        (long)(this.memoryLimit * 
-               conf.getFloat(
-                   TezJobConfig.TEZ_ENGINE_SHUFFLE_MERGE_PERCENT, 
-                   TezJobConfig.DEFAULT_TEZ_ENGINE_SHUFFLE_MERGE_PERCENT));
-    LOG.info("MergerManager: memoryLimit=" + memoryLimit + ", " +
-             "maxSingleShuffleLimit=" + maxSingleShuffleLimit + ", " +
-             "mergeThreshold=" + mergeThreshold + ", " + 
-             "ioSortFactor=" + ioSortFactor + ", " +
-             "memToMemMergeOutputsThreshold=" + memToMemMergeOutputsThreshold);
-
-    if (this.maxSingleShuffleLimit >= this.mergeThreshold) {
-      throw new RuntimeException("Invlaid configuration: "
-          + "maxSingleShuffleLimit should be less than mergeThreshold"
-          + "maxSingleShuffleLimit: " + this.maxSingleShuffleLimit
-          + "mergeThreshold: " + this.mergeThreshold);
-    }
-
-    boolean allowMemToMemMerge = 
-      conf.getBoolean(
-          TezJobConfig.TEZ_ENGINE_SHUFFLE_ENABLE_MEMTOMEM, 
-          TezJobConfig.DEFAULT_TEZ_ENGINE_SHUFFLE_ENABLE_MEMTOMEM);
-    if (allowMemToMemMerge) {
-      this.memToMemMerger = 
-        new IntermediateMemoryToMemoryMerger(this,
-                                             memToMemMergeOutputsThreshold);
-      this.memToMemMerger.start();
-    } else {
-      this.memToMemMerger = null;
-    }
-    
-    this.inMemoryMerger = new InMemoryMerger(this);
-    this.inMemoryMerger.start();
-    
-    this.onDiskMerger = new OnDiskMerger(this);
-    this.onDiskMerger.start();
-  }
-
-  public void waitForInMemoryMerge() throws InterruptedException {
-    inMemoryMerger.waitForMerge();
-  }
-  
-  private boolean canShuffleToMemory(long requestedSize) {
-    return (requestedSize < maxSingleShuffleLimit); 
-  }
-
-  final private MapOutput stallShuffle = new MapOutput(null);
-
-  public synchronized MapOutput reserve(InputAttemptIdentifier srcAttemptIdentifier, 
-                                             long requestedSize,
-                                             int fetcher
-                                             ) throws IOException {
-    if (!canShuffleToMemory(requestedSize)) {
-      LOG.info(srcAttemptIdentifier + ": Shuffling to disk since " + requestedSize + 
-               " is greater than maxSingleShuffleLimit (" + 
-               maxSingleShuffleLimit + ")");
-      return new MapOutput(srcAttemptIdentifier, this, requestedSize, conf, 
-                                localDirAllocator, fetcher, true,
-                                mapOutputFile);
-    }
-    
-    // Stall shuffle if we are above the memory limit
-
-    // It is possible that all threads could just be stalling and not make
-    // progress at all. This could happen when:
-    //
-    // requested size is causing the used memory to go above limit &&
-    // requested size < singleShuffleLimit &&
-    // current used size < mergeThreshold (merge will not get triggered)
-    //
-    // To avoid this from happening, we allow exactly one thread to go past
-    // the memory limit. We check (usedMemory > memoryLimit) and not
-    // (usedMemory + requestedSize > memoryLimit). When this thread is done
-    // fetching, this will automatically trigger a merge thereby unlocking
-    // all the stalled threads
-    
-    if (usedMemory > memoryLimit) {
-      LOG.debug(srcAttemptIdentifier + ": Stalling shuffle since usedMemory (" + usedMemory
-          + ") is greater than memoryLimit (" + memoryLimit + ")." + 
-          " CommitMemory is (" + commitMemory + ")"); 
-      return stallShuffle;
-    }
-    
-    // Allow the in-memory shuffle to progress
-    LOG.debug(srcAttemptIdentifier + ": Proceeding with shuffle since usedMemory ("
-        + usedMemory + ") is lesser than memoryLimit (" + memoryLimit + ")."
-        + "CommitMemory is (" + commitMemory + ")"); 
-    return unconditionalReserve(srcAttemptIdentifier, requestedSize, true);
-  }
-  
-  /**
-   * Unconditional Reserve is used by the Memory-to-Memory thread
-   * @return
-   */
-  private synchronized MapOutput unconditionalReserve(
-      InputAttemptIdentifier srcAttemptIdentifier, long requestedSize, boolean primaryMapOutput) {
-    usedMemory += requestedSize;
-    return new MapOutput(srcAttemptIdentifier, this, (int)requestedSize, 
-        primaryMapOutput);
-  }
-  
-  synchronized void unreserve(long size) {
-    commitMemory -= size;
-    usedMemory -= size;
-  }
-
-  public synchronized void closeInMemoryFile(MapOutput mapOutput) { 
-    inMemoryMapOutputs.add(mapOutput);
-    LOG.info("closeInMemoryFile -> map-output of size: " + mapOutput.getSize()
-        + ", inMemoryMapOutputs.size() -> " + inMemoryMapOutputs.size()
-        + ", commitMemory -> " + commitMemory + ", usedMemory ->" + usedMemory);
-
-    commitMemory+= mapOutput.getSize();
-
-    synchronized (inMemoryMerger) {
-      // Can hang if mergeThreshold is really low.
-      if (!inMemoryMerger.isInProgress() && commitMemory >= mergeThreshold) {
-        LOG.info("Starting inMemoryMerger's merge since commitMemory=" +
-            commitMemory + " > mergeThreshold=" + mergeThreshold + 
-            ". Current usedMemory=" + usedMemory);
-        inMemoryMapOutputs.addAll(inMemoryMergedMapOutputs);
-        inMemoryMergedMapOutputs.clear();
-        inMemoryMerger.startMerge(inMemoryMapOutputs);
-      } 
-    }
-    
-    if (memToMemMerger != null) {
-      synchronized (memToMemMerger) {
-        if (!memToMemMerger.isInProgress() && 
-            inMemoryMapOutputs.size() >= memToMemMergeOutputsThreshold) {
-          memToMemMerger.startMerge(inMemoryMapOutputs);
-        }
-      }
-    }
-  }
-  
-  
-  public synchronized void closeInMemoryMergedFile(MapOutput mapOutput) {
-    inMemoryMergedMapOutputs.add(mapOutput);
-    LOG.info("closeInMemoryMergedFile -> size: " + mapOutput.getSize() + 
-             ", inMemoryMergedMapOutputs.size() -> " + 
-             inMemoryMergedMapOutputs.size());
-  }
-  
-  public synchronized void closeOnDiskFile(Path file) {
-    onDiskMapOutputs.add(file);
-    
-    synchronized (onDiskMerger) {
-      if (!onDiskMerger.isInProgress() && 
-          onDiskMapOutputs.size() >= (2 * ioSortFactor - 1)) {
-        onDiskMerger.startMerge(onDiskMapOutputs);
-      }
-    }
-  }
-
-  /**
-   * Should <b>only</b> be used after the Shuffle phaze is complete, otherwise can
-   * return an invalid state since a merge may not be in progress dur to
-   * inadequate inputs
-   * 
-   * @return true if the merge process is complete, otherwise false
-   */
-  @Private
-  public boolean isMergeComplete() {
-    return finalMergeComplete;
-  }
-  
-  public TezRawKeyValueIterator close() throws Throwable {
-    // Wait for on-going merges to complete
-    if (memToMemMerger != null) { 
-      memToMemMerger.close();
-    }
-    inMemoryMerger.close();
-    onDiskMerger.close();
-    
-    List<MapOutput> memory = 
-      new ArrayList<MapOutput>(inMemoryMergedMapOutputs);
-    memory.addAll(inMemoryMapOutputs);
-    List<Path> disk = new ArrayList<Path>(onDiskMapOutputs);
-    TezRawKeyValueIterator kvIter = finalMerge(conf, rfs, memory, disk);
-    this.finalMergeComplete = true;
-    return kvIter;
-  }
-   
-  void runCombineProcessor(TezRawKeyValueIterator kvIter, Writer writer)
-      throws IOException, InterruptedException {
-    combiner.combine(kvIter, writer);
-  }
-
-  private class IntermediateMemoryToMemoryMerger 
-  extends MergeThread<MapOutput> {
-    
-    public IntermediateMemoryToMemoryMerger(MergeManager manager, 
-                                            int mergeFactor) {
-      super(manager, mergeFactor, exceptionReporter);
-      setName("InMemoryMerger - Thread to do in-memory merge of in-memory " +
-      		    "shuffled map-outputs");
-      setDaemon(true);
-    }
-
-    @Override
-    public void merge(List<MapOutput> inputs) throws IOException {
-      if (inputs == null || inputs.size() == 0) {
-        return;
-      }
-
-      InputAttemptIdentifier dummyMapId = inputs.get(0).getAttemptIdentifier(); 
-      List<Segment> inMemorySegments = new ArrayList<Segment>();
-      long mergeOutputSize = 
-        createInMemorySegments(inputs, inMemorySegments, 0);
-      int noInMemorySegments = inMemorySegments.size();
-      
-      MapOutput mergedMapOutputs = 
-        unconditionalReserve(dummyMapId, mergeOutputSize, false);
-      
-      Writer writer = 
-        new InMemoryWriter(mergedMapOutputs.getArrayStream());
-      
-      LOG.info("Initiating Memory-to-Memory merge with " + noInMemorySegments +
-               " segments of total-size: " + mergeOutputSize);
-
-      TezRawKeyValueIterator rIter = 
-        TezMerger.merge(conf, rfs,
-                       ConfigUtils.getIntermediateInputKeyClass(conf),
-                       ConfigUtils.getIntermediateInputValueClass(conf),
-                       inMemorySegments, inMemorySegments.size(),
-                       new Path(inputContext.getUniqueIdentifier()),
-                       (RawComparator)ConfigUtils.getIntermediateInputKeyComparator(conf),
-                       nullProgressable, null, null, null);
-      TezMerger.writeFile(rIter, writer, nullProgressable, conf);
-      writer.close();
-
-      LOG.info(inputContext.getUniqueIdentifier() +  
-               " Memory-to-Memory merge of the " + noInMemorySegments +
-               " files in-memory complete.");
-
-      // Note the output of the merge
-      closeInMemoryMergedFile(mergedMapOutputs);
-    }
-  }
-  
-  private class InMemoryMerger extends MergeThread<MapOutput> {
-    
-    public InMemoryMerger(MergeManager manager) {
-      super(manager, Integer.MAX_VALUE, exceptionReporter);
-      setName
-      ("InMemoryMerger - Thread to merge in-memory shuffled map-outputs");
-      setDaemon(true);
-    }
-    
-    @Override
-    public void merge(List<MapOutput> inputs) throws IOException, InterruptedException {
-      if (inputs == null || inputs.size() == 0) {
-        return;
-      }
-      
-      //name this output file same as the name of the first file that is 
-      //there in the current list of inmem files (this is guaranteed to
-      //be absent on the disk currently. So we don't overwrite a prev. 
-      //created spill). Also we need to create the output file now since
-      //it is not guaranteed that this file will be present after merge
-      //is called (we delete empty files as soon as we see them
-      //in the merge method)
-
-      //figure out the mapId 
-      InputAttemptIdentifier srcTaskIdentifier = inputs.get(0).getAttemptIdentifier();
-
-      List<Segment> inMemorySegments = new ArrayList<Segment>();
-      long mergeOutputSize = 
-        createInMemorySegments(inputs, inMemorySegments,0);
-      int noInMemorySegments = inMemorySegments.size();
-
-      Path outputPath = mapOutputFile.getInputFileForWrite(
-          srcTaskIdentifier.getInputIdentifier().getSrcTaskIndex(),
-          mergeOutputSize).suffix(Constants.MERGED_OUTPUT_PREFIX);
-
-      Writer writer = null;
-      try {
-        writer =
-            new Writer(conf, rfs, outputPath,
-                (Class)ConfigUtils.getIntermediateInputKeyClass(conf),
-                (Class)ConfigUtils.getIntermediateInputValueClass(conf),
-                codec, null);
-
-        TezRawKeyValueIterator rIter = null;
-        LOG.info("Initiating in-memory merge with " + noInMemorySegments + 
-            " segments...");
-
-        rIter = TezMerger.merge(conf, rfs,
-            (Class)ConfigUtils.getIntermediateInputKeyClass(conf),
-            (Class)ConfigUtils.getIntermediateInputValueClass(conf),
-            inMemorySegments, inMemorySegments.size(),
-            new Path(inputContext.getUniqueIdentifier()),
-            (RawComparator)ConfigUtils.getIntermediateInputKeyComparator(conf),
-            nullProgressable, spilledRecordsCounter, null, null);
-
-        if (null == combiner) {
-          TezMerger.writeFile(rIter, writer, nullProgressable, conf);
-        } else {
-          runCombineProcessor(rIter, writer);
-        }
-        writer.close();
-        writer = null;
-
-        LOG.info(inputContext.getUniqueIdentifier() +  
-            " Merge of the " + noInMemorySegments +
-            " files in-memory complete." +
-            " Local file is " + outputPath + " of size " + 
-            localFS.getFileStatus(outputPath).getLen());
-      } catch (IOException e) { 
-        //make sure that we delete the ondisk file that we created 
-        //earlier when we invoked cloneFileAttributes
-        localFS.delete(outputPath, true);
-        throw e;
-      } finally {
-        if (writer != null) {
-          writer.close();
-        }
-      }
-
-      // Note the output of the merge
-      closeOnDiskFile(outputPath);
-    }
-
-  }
-  
-  private class OnDiskMerger extends MergeThread<Path> {
-    
-    public OnDiskMerger(MergeManager manager) {
-      super(manager, Integer.MAX_VALUE, exceptionReporter);
-      setName("OnDiskMerger - Thread to merge on-disk map-outputs");
-      setDaemon(true);
-    }
-    
-    @Override
-    public void merge(List<Path> inputs) throws IOException {
-      // sanity check
-      if (inputs == null || inputs.isEmpty()) {
-        LOG.info("No ondisk files to merge...");
-        return;
-      }
-      
-      long approxOutputSize = 0;
-      int bytesPerSum = 
-        conf.getInt("io.bytes.per.checksum", 512);
-      
-      LOG.info("OnDiskMerger: We have  " + inputs.size() + 
-               " map outputs on disk. Triggering merge...");
-      
-      // 1. Prepare the list of files to be merged. 
-      for (Path file : inputs) {
-        approxOutputSize += localFS.getFileStatus(file).getLen();
-      }
-
-      // add the checksum length
-      approxOutputSize += 
-        ChecksumFileSystem.getChecksumLength(approxOutputSize, bytesPerSum);
-
-      // 2. Start the on-disk merge process
-      Path outputPath = 
-        localDirAllocator.getLocalPathForWrite(inputs.get(0).toString(), 
-            approxOutputSize, conf).suffix(Constants.MERGED_OUTPUT_PREFIX);
-      Writer writer = 
-        new Writer(conf, rfs, outputPath, 
-                        (Class)ConfigUtils.getIntermediateInputKeyClass(conf), 
-                        (Class)ConfigUtils.getIntermediateInputValueClass(conf),
-                        codec, null);
-      TezRawKeyValueIterator iter  = null;
-      Path tmpDir = new Path(inputContext.getUniqueIdentifier());
-      try {
-        iter = TezMerger.merge(conf, rfs,
-                            (Class)ConfigUtils.getIntermediateInputKeyClass(conf), 
-                            (Class)ConfigUtils.getIntermediateInputValueClass(conf),
-                            codec, inputs.toArray(new Path[inputs.size()]), 
-                            true, ioSortFactor, tmpDir, 
-                            (RawComparator)ConfigUtils.getIntermediateInputKeyComparator(conf), 
-                            nullProgressable, spilledRecordsCounter, null, 
-                            mergedMapOutputsCounter, null);
-
-        TezMerger.writeFile(iter, writer, nullProgressable, conf);
-        writer.close();
-      } catch (IOException e) {
-        localFS.delete(outputPath, true);
-        throw e;
-      }
-
-      closeOnDiskFile(outputPath);
-
-      LOG.info(inputContext.getUniqueIdentifier() +
-          " Finished merging " + inputs.size() + 
-          " map output files on disk of total-size " + 
-          approxOutputSize + "." + 
-          " Local output file is " + outputPath + " of size " +
-          localFS.getFileStatus(outputPath).getLen());
-    }
-  }
-  
-  private long createInMemorySegments(List<MapOutput> inMemoryMapOutputs,
-                                      List<Segment> inMemorySegments, 
-                                      long leaveBytes
-                                      ) throws IOException {
-    long totalSize = 0L;
-    // We could use fullSize could come from the RamManager, but files can be
-    // closed but not yet present in inMemoryMapOutputs
-    long fullSize = 0L;
-    for (MapOutput mo : inMemoryMapOutputs) {
-      fullSize += mo.getMemory().length;
-    }
-    while(fullSize > leaveBytes) {
-      MapOutput mo = inMemoryMapOutputs.remove(0);
-      byte[] data = mo.getMemory();
-      long size = data.length;
-      totalSize += size;
-      fullSize -= size;
-      IFile.Reader reader = new InMemoryReader(MergeManager.this, 
-                                                   mo.getAttemptIdentifier(),
-                                                   data, 0, (int)size);
-      inMemorySegments.add(new Segment(reader, true, 
-                                            (mo.isPrimaryMapOutput() ? 
-                                            mergedMapOutputsCounter : null)));
-    }
-    return totalSize;
-  }
-
-  class RawKVIteratorReader extends IFile.Reader {
-
-    private final TezRawKeyValueIterator kvIter;
-
-    public RawKVIteratorReader(TezRawKeyValueIterator kvIter, long size)
-        throws IOException {
-      super(null, null, size, null, spilledRecordsCounter);
-      this.kvIter = kvIter;
-    }
-    public boolean nextRawKey(DataInputBuffer key) throws IOException {
-      if (kvIter.next()) {
-        final DataInputBuffer kb = kvIter.getKey();
-        final int kp = kb.getPosition();
-        final int klen = kb.getLength() - kp;
-        key.reset(kb.getData(), kp, klen);
-        bytesRead += klen;
-        return true;
-      }
-      return false;
-    }
-    public void nextRawValue(DataInputBuffer value) throws IOException {
-      final DataInputBuffer vb = kvIter.getValue();
-      final int vp = vb.getPosition();
-      final int vlen = vb.getLength() - vp;
-      value.reset(vb.getData(), vp, vlen);
-      bytesRead += vlen;
-    }
-    public long getPosition() throws IOException {
-      return bytesRead;
-    }
-
-    public void close() throws IOException {
-      kvIter.close();
-    }
-  }
-
-  private TezRawKeyValueIterator finalMerge(Configuration job, FileSystem fs,
-                                       List<MapOutput> inMemoryMapOutputs,
-                                       List<Path> onDiskMapOutputs
-                                       ) throws IOException {
-    LOG.info("finalMerge called with " + 
-             inMemoryMapOutputs.size() + " in-memory map-outputs and " + 
-             onDiskMapOutputs.size() + " on-disk map-outputs");
-    
-    final float maxRedPer =
-      job.getFloat(
-          TezJobConfig.TEZ_ENGINE_INPUT_BUFFER_PERCENT,
-          TezJobConfig.DEFAULT_TEZ_ENGINE_INPUT_BUFFER_PERCENT);
-    if (maxRedPer > 1.0 || maxRedPer < 0.0) {
-      throw new IOException(TezJobConfig.TEZ_ENGINE_INPUT_BUFFER_PERCENT +
-                            maxRedPer);
-    }
-    int maxInMemReduce = (int)Math.min(
-        Runtime.getRuntime().maxMemory() * maxRedPer, Integer.MAX_VALUE);
-    
-
-    // merge config params
-    Class keyClass = (Class)ConfigUtils.getIntermediateInputKeyClass(job);
-    Class valueClass = (Class)ConfigUtils.getIntermediateInputValueClass(job);
-    final Path tmpDir = new Path(inputContext.getUniqueIdentifier());
-    final RawComparator comparator =
-      (RawComparator)ConfigUtils.getIntermediateInputKeyComparator(job);
-
-    // segments required to vacate memory
-    List<Segment> memDiskSegments = new ArrayList<Segment>();
-    long inMemToDiskBytes = 0;
-    boolean mergePhaseFinished = false;
-    if (inMemoryMapOutputs.size() > 0) {
-      int srcTaskId = inMemoryMapOutputs.get(0).getAttemptIdentifier().getInputIdentifier().getSrcTaskIndex();
-      inMemToDiskBytes = createInMemorySegments(inMemoryMapOutputs, 
-                                                memDiskSegments,
-                                                maxInMemReduce);
-      final int numMemDiskSegments = memDiskSegments.size();
-      if (numMemDiskSegments > 0 &&
-            ioSortFactor > onDiskMapOutputs.size()) {
-        
-        // If we reach here, it implies that we have less than io.sort.factor
-        // disk segments and this will be incremented by 1 (result of the 
-        // memory segments merge). Since this total would still be 
-        // <= io.sort.factor, we will not do any more intermediate merges,
-        // the merge of all these disk segments would be directly fed to the
-        // reduce method
-        
-        mergePhaseFinished = true;
-        // must spill to disk, but can't retain in-mem for intermediate merge
-        final Path outputPath = 
-          mapOutputFile.getInputFileForWrite(srcTaskId,
-                                             inMemToDiskBytes).suffix(
-                                                 Constants.MERGED_OUTPUT_PREFIX);
-        final TezRawKeyValueIterator rIter = TezMerger.merge(job, fs,
-            keyClass, valueClass, memDiskSegments, numMemDiskSegments,
-            tmpDir, comparator, nullProgressable, spilledRecordsCounter, null, null);
-        final Writer writer = new Writer(job, fs, outputPath,
-            keyClass, valueClass, codec, null);
-        try {
-          TezMerger.writeFile(rIter, writer, nullProgressable, job);
-          // add to list of final disk outputs.
-          onDiskMapOutputs.add(outputPath);
-        } catch (IOException e) {
-          if (null != outputPath) {
-            try {
-              fs.delete(outputPath, true);
-            } catch (IOException ie) {
-              // NOTHING
-            }
-          }
-          throw e;
-        } finally {
-          if (null != writer) {
-            writer.close();
-          }
-        }
-        LOG.info("Merged " + numMemDiskSegments + " segments, " +
-                 inMemToDiskBytes + " bytes to disk to satisfy " +
-                 "reduce memory limit");
-        inMemToDiskBytes = 0;
-        memDiskSegments.clear();
-      } else if (inMemToDiskBytes != 0) {
-        LOG.info("Keeping " + numMemDiskSegments + " segments, " +
-                 inMemToDiskBytes + " bytes in memory for " +
-                 "intermediate, on-disk merge");
-      }
-    }
-
-    // segments on disk
-    List<Segment> diskSegments = new ArrayList<Segment>();
-    long onDiskBytes = inMemToDiskBytes;
-    Path[] onDisk = onDiskMapOutputs.toArray(new Path[onDiskMapOutputs.size()]);
-    for (Path file : onDisk) {
-      onDiskBytes += fs.getFileStatus(file).getLen();
-      LOG.debug("Disk file: " + file + " Length is " + 
-          fs.getFileStatus(file).getLen());
-      diskSegments.add(new Segment(job, fs, file, codec, false,
-                                         (file.toString().endsWith(
-                                             Constants.MERGED_OUTPUT_PREFIX) ?
-                                          null : mergedMapOutputsCounter)
-                                        ));
-    }
-    LOG.info("Merging " + onDisk.length + " files, " +
-             onDiskBytes + " bytes from disk");
-    Collections.sort(diskSegments, new Comparator<Segment>() {
-      public int compare(Segment o1, Segment o2) {
-        if (o1.getLength() == o2.getLength()) {
-          return 0;
-        }
-        return o1.getLength() < o2.getLength() ? -1 : 1;
-      }
-    });
-
-    // build final list of segments from merged backed by disk + in-mem
-    List<Segment> finalSegments = new ArrayList<Segment>();
-    long inMemBytes = createInMemorySegments(inMemoryMapOutputs, 
-                                             finalSegments, 0);
-    LOG.info("Merging " + finalSegments.size() + " segments, " +
-             inMemBytes + " bytes from memory into reduce");
-    if (0 != onDiskBytes) {
-      final int numInMemSegments = memDiskSegments.size();
-      diskSegments.addAll(0, memDiskSegments);
-      memDiskSegments.clear();
-      TezRawKeyValueIterator diskMerge = TezMerger.merge(
-          job, fs, keyClass, valueClass, diskSegments,
-          ioSortFactor, numInMemSegments, tmpDir, comparator,
-          nullProgressable, false, spilledRecordsCounter, null, null);
-      diskSegments.clear();
-      if (0 == finalSegments.size()) {
-        return diskMerge;
-      }
-      finalSegments.add(new Segment(
-            new RawKVIteratorReader(diskMerge, onDiskBytes), true));
-    }
-    return TezMerger.merge(job, fs, keyClass, valueClass,
-                 finalSegments, finalSegments.size(), tmpDir,
-                 comparator, nullProgressable, spilledRecordsCounter, null,
-                 null);
-  
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/MergeThread.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/MergeThread.java b/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/MergeThread.java
deleted file mode 100644
index bab882e..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/MergeThread.java
+++ /dev/null
@@ -1,108 +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.engine.common.shuffle.impl;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Set;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-
-abstract class MergeThread<T> extends Thread {
-  
-  private static final Log LOG = LogFactory.getLog(MergeThread.class);
-
-  private volatile boolean inProgress = false;
-  private List<T> inputs = new ArrayList<T>();
-  protected final MergeManager manager;
-  private final ExceptionReporter reporter;
-  private boolean closed = false;
-  private final int mergeFactor;
-  
-  public MergeThread(MergeManager manager, int mergeFactor,
-                     ExceptionReporter reporter) {
-    this.manager = manager;
-    this.mergeFactor = mergeFactor;
-    this.reporter = reporter;
-  }
-  
-  public synchronized void close() throws InterruptedException {
-    closed = true;
-    waitForMerge();
-    interrupt();
-  }
-
-  public synchronized boolean isInProgress() {
-    return inProgress;
-  }
-  
-  public synchronized void startMerge(Set<T> inputs) {
-    if (!closed) {
-      inProgress = true;
-      this.inputs = new ArrayList<T>();
-      Iterator<T> iter=inputs.iterator();
-      for (int ctr = 0; iter.hasNext() && ctr < mergeFactor; ++ctr) {
-        this.inputs.add(iter.next());
-        iter.remove();
-      }
-      LOG.info(getName() + ": Starting merge with " + this.inputs.size() + 
-               " segments, while ignoring " + inputs.size() + " segments");
-      notifyAll();
-    }
-  }
-
-  public synchronized void waitForMerge() throws InterruptedException {
-    while (inProgress) {
-      wait();
-    }
-  }
-
-  public void run() {
-    while (true) {
-      try {
-        // Wait for notification to start the merge...
-        synchronized (this) {
-          while (!inProgress) {
-            wait();
-          }
-        }
-
-        // Merge
-        merge(inputs);
-      } catch (InterruptedException ie) {
-        return;
-      } catch(Throwable t) {
-        reporter.reportException(t);
-        return;
-      } finally {
-        synchronized (this) {
-          // Clear inputs
-          inputs = null;
-          inProgress = false;        
-          notifyAll();
-        }
-      }
-    }
-  }
-
-  public abstract void merge(List<T> inputs) 
-      throws IOException, InterruptedException;
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/Shuffle.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/Shuffle.java b/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/Shuffle.java
deleted file mode 100644
index 15332a1..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/Shuffle.java
+++ /dev/null
@@ -1,278 +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.engine.common.shuffle.impl;
-
-import java.io.IOException;
-import java.util.List;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.FutureTask;
-import java.util.concurrent.atomic.AtomicInteger;
-
-import javax.crypto.SecretKey;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.LocalDirAllocator;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.tez.common.TezJobConfig;
-import org.apache.tez.common.counters.TaskCounter;
-import org.apache.tez.common.counters.TezCounter;
-import org.apache.tez.dag.api.TezUncheckedException;
-import org.apache.tez.engine.api.Event;
-import org.apache.tez.engine.api.TezInputContext;
-import org.apache.tez.engine.common.TezEngineUtils;
-import org.apache.tez.engine.common.combine.Combiner;
-import org.apache.tez.engine.common.shuffle.server.ShuffleHandler;
-import org.apache.tez.engine.common.sort.impl.TezRawKeyValueIterator;
-import org.apache.tez.engine.shuffle.common.ShuffleUtils;
-
-import com.google.common.base.Preconditions;
-
-@InterfaceAudience.Private
-@InterfaceStability.Unstable
-public class Shuffle implements ExceptionReporter {
-  
-  private static final Log LOG = LogFactory.getLog(Shuffle.class);
-  private static final int PROGRESS_FREQUENCY = 2000;
-  
-  private final Configuration conf;
-  private final TezInputContext inputContext;
-  private final ShuffleClientMetrics metrics;
-
-  private final ShuffleInputEventHandler eventHandler;
-  private final ShuffleScheduler scheduler;
-  private final MergeManager merger;
-  private Throwable throwable = null;
-  private String throwingThreadName = null;
-  private final int numInputs;
-  private final AtomicInteger reduceStartId;
-  private final SecretKey jobTokenSecret;
-  private AtomicInteger reduceRange = new AtomicInteger(
-      TezJobConfig.TEZ_ENGINE_SHUFFLE_PARTITION_RANGE_DEFAULT);
-
-  private FutureTask<TezRawKeyValueIterator> runShuffleFuture;
-
-  public Shuffle(TezInputContext inputContext, Configuration conf, int numInputs) throws IOException {
-    this.inputContext = inputContext;
-    this.conf = conf;
-    this.metrics = new ShuffleClientMetrics(inputContext.getDAGName(),
-        inputContext.getTaskVertexName(), inputContext.getTaskIndex(),
-        this.conf, UserGroupInformation.getCurrentUser().getShortUserName());
-            
-    this.numInputs = numInputs;
-    
-    this.jobTokenSecret = ShuffleUtils
-        .getJobTokenSecretFromTokenBytes(inputContext
-            .getServiceConsumerMetaData(ShuffleHandler.MAPREDUCE_SHUFFLE_SERVICEID));
-    
-    Combiner combiner = TezEngineUtils.instantiateCombiner(conf, inputContext);
-    
-    FileSystem localFS = FileSystem.getLocal(this.conf);
-    LocalDirAllocator localDirAllocator = 
-        new LocalDirAllocator(TezJobConfig.LOCAL_DIRS);
-
-    // TODO TEZ Get rid of Map / Reduce references.
-    TezCounter shuffledMapsCounter = 
-        inputContext.getCounters().findCounter(TaskCounter.SHUFFLED_MAPS);
-    TezCounter reduceShuffleBytes =
-        inputContext.getCounters().findCounter(TaskCounter.REDUCE_SHUFFLE_BYTES);
-    TezCounter failedShuffleCounter =
-        inputContext.getCounters().findCounter(TaskCounter.FAILED_SHUFFLE);
-    TezCounter spilledRecordsCounter = 
-        inputContext.getCounters().findCounter(TaskCounter.SPILLED_RECORDS);
-    TezCounter reduceCombineInputCounter =
-        inputContext.getCounters().findCounter(TaskCounter.COMBINE_INPUT_RECORDS);
-    TezCounter mergedMapOutputsCounter =
-        inputContext.getCounters().findCounter(TaskCounter.MERGED_MAP_OUTPUTS);
-    
-    reduceStartId = new AtomicInteger(inputContext.getTaskIndex());
-    LOG.info("Shuffle assigned reduce start id: " + reduceStartId.get()
-        + " with default reduce range: " + reduceRange.get());
-
-    scheduler = new ShuffleScheduler(
-          this.inputContext,
-          this.conf,
-          this.numInputs,
-          this,
-          shuffledMapsCounter,
-          reduceShuffleBytes,
-          failedShuffleCounter);
-    eventHandler= new ShuffleInputEventHandler(
-          inputContext,
-          this,
-          scheduler);
-    merger = new MergeManager(
-          this.conf,
-          localFS,
-          localDirAllocator,
-          inputContext,
-          combiner,
-          spilledRecordsCounter,
-          reduceCombineInputCounter,
-          mergedMapOutputsCounter,
-          this);
-  }
-
-  public void handleEvents(List<Event> events) {
-    eventHandler.handleEvents(events);
-  }
-  
-  /**
-   * Indicates whether the Shuffle and Merge processing is complete.
-   * @return false if not complete, true if complete or if an error occurred.
-   */
-  public boolean isInputReady() {
-    if (runShuffleFuture == null) {
-      return false;
-    }
-    return runShuffleFuture.isDone();
-    //return scheduler.isDone() && merger.isMergeComplete();
-  }
-
-  /**
-   * Waits for the Shuffle and Merge to complete, and returns an iterator over the input.
-   * @return an iterator over the fetched input.
-   * @throws IOException
-   * @throws InterruptedException
-   */
-  public TezRawKeyValueIterator waitForInput() throws IOException, InterruptedException {
-    Preconditions.checkState(runShuffleFuture != null,
-        "waitForInput can only be called after run");
-    TezRawKeyValueIterator kvIter;
-    try {
-      kvIter = runShuffleFuture.get();
-    } catch (ExecutionException e) {
-      Throwable cause = e.getCause();
-      if (cause instanceof IOException) {
-        throw (IOException) cause;
-      } else if (cause instanceof InterruptedException) {
-        throw (InterruptedException) cause;
-      } else {
-        throw new TezUncheckedException(
-            "Unexpected exception type while running Shuffle and Merge", cause);
-      }
-    }
-    return kvIter;
-  }
-
-  public void run() {
-    RunShuffleCallable runShuffle = new RunShuffleCallable();
-    runShuffleFuture = new FutureTask<TezRawKeyValueIterator>(runShuffle);
-    new Thread(runShuffleFuture, "ShuffleMergeRunner").start();
-  }
-  
-  private class RunShuffleCallable implements Callable<TezRawKeyValueIterator> {
-    @Override
-    public TezRawKeyValueIterator call() throws IOException, InterruptedException {
-      // TODO NEWTEZ Limit # fetchers to number of inputs
-      final int numFetchers = 
-          conf.getInt(
-              TezJobConfig.TEZ_ENGINE_SHUFFLE_PARALLEL_COPIES, 
-              TezJobConfig.DEFAULT_TEZ_ENGINE_SHUFFLE_PARALLEL_COPIES);
-      Fetcher[] fetchers = new Fetcher[numFetchers];
-      for (int i = 0; i < numFetchers; ++i) {
-        fetchers[i] = new Fetcher(conf, scheduler, merger, metrics, Shuffle.this, jobTokenSecret, inputContext);
-        fetchers[i].start();
-      }
-      
-      while (!scheduler.waitUntilDone(PROGRESS_FREQUENCY)) {
-        synchronized (this) {
-          if (throwable != null) {
-            throw new ShuffleError("error in shuffle in " + throwingThreadName,
-                                   throwable);
-          }
-        }
-      }
-      
-      // Stop the map-output fetcher threads
-      for (Fetcher fetcher : fetchers) {
-        fetcher.shutDown();
-      }
-      fetchers = null;
-      
-      // stop the scheduler
-      scheduler.close();
-
-
-      // Finish the on-going merges...
-      TezRawKeyValueIterator kvIter = null;
-      try {
-        kvIter = merger.close();
-      } catch (Throwable e) {
-        throw new ShuffleError("Error while doing final merge " , e);
-      }
-      
-      // Sanity check
-      synchronized (Shuffle.this) {
-        if (throwable != null) {
-          throw new ShuffleError("error in shuffle in " + throwingThreadName,
-                                 throwable);
-        }
-      }
-      return kvIter;
-    }
-  }
-  
-  public int getReduceStartId() {
-    return reduceStartId.get();
-  }
-  
-  public int getReduceRange() {
-    return reduceRange.get();
-  }
-  
-  public synchronized void reportException(Throwable t) {
-    if (throwable == null) {
-      throwable = t;
-      throwingThreadName = Thread.currentThread().getName();
-      // Notify the scheduler so that the reporting thread finds the 
-      // exception immediately.
-      synchronized (scheduler) {
-        scheduler.notifyAll();
-      }
-    }
-  }
-  
-  public static class ShuffleError extends IOException {
-    private static final long serialVersionUID = 5753909320586607881L;
-
-    ShuffleError(String msg, Throwable t) {
-      super(msg, t);
-    }
-  }
-
-  public void setPartitionRange(int range) {
-    if (range == reduceRange.get()) {
-      return;
-    }
-    if (reduceRange.compareAndSet(
-        TezJobConfig.TEZ_ENGINE_SHUFFLE_PARTITION_RANGE_DEFAULT, range)) {
-      LOG.info("Reduce range set to: " + range);
-    } else {
-      TezUncheckedException e = 
-          new TezUncheckedException("Reduce range can be set only once.");
-      reportException(e);
-      throw e; 
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/ShuffleClientMetrics.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/ShuffleClientMetrics.java b/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/ShuffleClientMetrics.java
deleted file mode 100644
index 850dbeb..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/ShuffleClientMetrics.java
+++ /dev/null
@@ -1,91 +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.engine.common.shuffle.impl;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.metrics.MetricsContext;
-import org.apache.hadoop.metrics.MetricsRecord;
-import org.apache.hadoop.metrics.MetricsUtil;
-import org.apache.hadoop.metrics.Updater;
-import org.apache.tez.common.Constants;
-import org.apache.tez.common.TezJobConfig;
-import org.apache.tez.engine.common.TezEngineUtils;
-
-class ShuffleClientMetrics implements Updater {
-
-  private MetricsRecord shuffleMetrics = null;
-  private int numFailedFetches = 0;
-  private int numSuccessFetches = 0;
-  private long numBytes = 0;
-  private int numThreadsBusy = 0;
-  private final int numCopiers;
-  
-  ShuffleClientMetrics(String dagName, String vertexName, int taskIndex, Configuration conf, 
-      String user) {
-    this.numCopiers = 
-        conf.getInt(
-            TezJobConfig.TEZ_ENGINE_SHUFFLE_PARALLEL_COPIES, 
-            TezJobConfig.DEFAULT_TEZ_ENGINE_SHUFFLE_PARALLEL_COPIES);
-
-    MetricsContext metricsContext = MetricsUtil.getContext(Constants.TEZ);
-    this.shuffleMetrics = 
-      MetricsUtil.createRecord(metricsContext, "shuffleInput");
-    this.shuffleMetrics.setTag("user", user);
-    this.shuffleMetrics.setTag("dagName", dagName);
-    this.shuffleMetrics.setTag("taskId", TezEngineUtils.getTaskIdentifier(vertexName, taskIndex));
-    this.shuffleMetrics.setTag("sessionId", 
-        conf.get(
-            TezJobConfig.TEZ_ENGINE_METRICS_SESSION_ID, 
-            TezJobConfig.DEFAULT_TEZ_ENGINE_METRICS_SESSION_ID));
-    metricsContext.registerUpdater(this);
-  }
-  public synchronized void inputBytes(long numBytes) {
-    this.numBytes += numBytes;
-  }
-  public synchronized void failedFetch() {
-    ++numFailedFetches;
-  }
-  public synchronized void successFetch() {
-    ++numSuccessFetches;
-  }
-  public synchronized void threadBusy() {
-    ++numThreadsBusy;
-  }
-  public synchronized void threadFree() {
-    --numThreadsBusy;
-  }
-  public void doUpdates(MetricsContext unused) {
-    synchronized (this) {
-      shuffleMetrics.incrMetric("shuffle_input_bytes", numBytes);
-      shuffleMetrics.incrMetric("shuffle_failed_fetches", 
-                                numFailedFetches);
-      shuffleMetrics.incrMetric("shuffle_success_fetches", 
-                                numSuccessFetches);
-      if (numCopiers != 0) {
-        shuffleMetrics.setMetric("shuffle_fetchers_busy_percent",
-            100*((float)numThreadsBusy/numCopiers));
-      } else {
-        shuffleMetrics.setMetric("shuffle_fetchers_busy_percent", 0);
-      }
-      numBytes = 0;
-      numSuccessFetches = 0;
-      numFailedFetches = 0;
-    }
-    shuffleMetrics.update();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/ShuffleHeader.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/ShuffleHeader.java b/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/ShuffleHeader.java
deleted file mode 100644
index a918ef1..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/ShuffleHeader.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.engine.common.shuffle.impl;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.io.WritableUtils;
-
-/**
- * Shuffle Header information that is sent by the TaskTracker and 
- * deciphered by the Fetcher thread of Reduce task
- *
- */
-@InterfaceAudience.Private
-@InterfaceStability.Stable
-public class ShuffleHeader implements Writable {
-  
-  /** Header info of the shuffle http request/response */
-  public static final String HTTP_HEADER_NAME = "name";
-  public static final String DEFAULT_HTTP_HEADER_NAME = "mapreduce";
-  public static final String HTTP_HEADER_VERSION = "version";
-  public static final String DEFAULT_HTTP_HEADER_VERSION = "1.0.0";
-
-  /**
-   * The longest possible length of task attempt id that we will accept.
-   */
-  private static final int MAX_ID_LENGTH = 1000;
-
-  String mapId;
-  long uncompressedLength;
-  long compressedLength;
-  int forReduce;
-  
-  public ShuffleHeader() { }
-  
-  public ShuffleHeader(String mapId, long compressedLength,
-      long uncompressedLength, int forReduce) {
-    this.mapId = mapId;
-    this.compressedLength = compressedLength;
-    this.uncompressedLength = uncompressedLength;
-    this.forReduce = forReduce;
-  }
-  
-  public String getMapId() {
-    return this.mapId;
-  }
-  
-  public int getPartition() {
-    return this.forReduce;
-  }
-  
-  public long getUncompressedLength() {
-    return uncompressedLength;
-  }
-
-  public long getCompressedLength() {
-    return compressedLength;
-  }
-
-  public void readFields(DataInput in) throws IOException {
-    mapId = WritableUtils.readStringSafely(in, MAX_ID_LENGTH);
-    compressedLength = WritableUtils.readVLong(in);
-    uncompressedLength = WritableUtils.readVLong(in);
-    forReduce = WritableUtils.readVInt(in);
-  }
-
-  public void write(DataOutput out) throws IOException {
-    Text.writeString(out, mapId);
-    WritableUtils.writeVLong(out, compressedLength);
-    WritableUtils.writeVLong(out, uncompressedLength);
-    WritableUtils.writeVInt(out, forReduce);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/ShuffleInputEventHandler.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/ShuffleInputEventHandler.java b/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/ShuffleInputEventHandler.java
deleted file mode 100644
index a8e5fe4..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/ShuffleInputEventHandler.java
+++ /dev/null
@@ -1,134 +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.engine.common.shuffle.impl;
-
-import java.net.URI;
-import java.util.List;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.tez.dag.api.TezUncheckedException;
-import org.apache.tez.engine.api.Event;
-import org.apache.tez.engine.api.TezInputContext;
-import org.apache.tez.engine.api.events.DataMovementEvent;
-import org.apache.tez.engine.api.events.InputFailedEvent;
-import org.apache.tez.engine.api.events.InputInformationEvent;
-import org.apache.tez.engine.common.InputAttemptIdentifier;
-import org.apache.tez.engine.common.shuffle.newimpl.ShuffleUserPayloads.DataMovementEventPayloadProto;
-import org.apache.tez.engine.common.shuffle.newimpl.ShuffleUserPayloads.InputInformationEventPayloadProto;
-
-import com.google.common.base.Preconditions;
-import com.google.protobuf.InvalidProtocolBufferException;
-
-public class ShuffleInputEventHandler {
-  
-  private static final Log LOG = LogFactory.getLog(ShuffleInputEventHandler.class);
-
-  private final ShuffleScheduler scheduler;
-  private final TezInputContext inputContext;
-  private final Shuffle shuffle;
-
-  private int maxMapRuntime = 0;
-  private boolean shuffleRangeSet = false;
-  
-  public ShuffleInputEventHandler(TezInputContext inputContext,
-      Shuffle shuffle, ShuffleScheduler scheduler) {
-    this.inputContext = inputContext;
-    this.shuffle = shuffle;
-    this.scheduler = scheduler;
-  }
-
-  public void handleEvents(List<Event> events) {
-    for (Event event : events) {
-      handleEvent(event);
-    }
-  }
-  
-  
-  private void handleEvent(Event event) {
-    if (event instanceof InputInformationEvent) {
-      processInputInformationEvent((InputInformationEvent) event);
-    }
-    else if (event instanceof DataMovementEvent) {
-      processDataMovementEvent((DataMovementEvent) event);      
-    } else if (event instanceof InputFailedEvent) {
-      processTaskFailedEvent((InputFailedEvent) event);
-    }
-  }
-
-  private void processInputInformationEvent(InputInformationEvent iiEvent) {
-    InputInformationEventPayloadProto inputInfoPayload;
-    try {
-      inputInfoPayload = InputInformationEventPayloadProto.parseFrom(iiEvent.getUserPayload());
-    } catch (InvalidProtocolBufferException e) {
-      throw new TezUncheckedException("Unable to parse InputInformationEvent payload", e);
-    }
-    int partitionRange = inputInfoPayload.getPartitionRange();
-    shuffle.setPartitionRange(partitionRange);
-    this.shuffleRangeSet = true;
-  }
-
-  private void processDataMovementEvent(DataMovementEvent dmEvent) {
-    // FIXME TODO NEWTEZ
-    // Preconditions.checkState(shuffleRangeSet == true, "Shuffle Range must be set before a DataMovementEvent is processed");
-    DataMovementEventPayloadProto shufflePayload;
-    try {
-      shufflePayload = DataMovementEventPayloadProto.parseFrom(dmEvent.getUserPayload());
-    } catch (InvalidProtocolBufferException e) {
-      throw new TezUncheckedException("Unable to parse DataMovementEvent payload", e);
-    } 
-    int partitionId = dmEvent.getSourceIndex();
-    URI baseUri = getBaseURI(shufflePayload.getHost(), shufflePayload.getPort(), partitionId);
-
-    InputAttemptIdentifier srcAttemptIdentifier = new InputAttemptIdentifier(dmEvent.getTargetIndex(), dmEvent.getVersion(), shufflePayload.getPathComponent());
-    scheduler.addKnownMapOutput(shufflePayload.getHost(), partitionId, baseUri.toString(), srcAttemptIdentifier);
-    
-    // TODO NEWTEZ See if this duration hack can be removed.
-    int duration = shufflePayload.getRunDuration();
-    if (duration > maxMapRuntime) {
-      maxMapRuntime = duration;
-      scheduler.informMaxMapRunTime(maxMapRuntime);
-    }
-  }
-  
-  private void processTaskFailedEvent(InputFailedEvent ifEvent) {
-    InputAttemptIdentifier taIdentifier = new InputAttemptIdentifier(ifEvent.getSourceIndex(), ifEvent.getVersion());
-    scheduler.obsoleteMapOutput(taIdentifier);
-    LOG.info("Obsoleting output of src-task: " + taIdentifier);
-  }
-
-  // TODO NEWTEZ Handle encrypted shuffle
-  private URI getBaseURI(String host, int port, int partitionId) {
-    StringBuilder sb = new StringBuilder("http://");
-    sb.append(host);
-    sb.append(":");
-    sb.append(String.valueOf(port));
-    sb.append("/");
-    
-    sb.append("mapOutput?job=");
-    // Required to use the existing ShuffleHandler
-    sb.append(inputContext.getApplicationId().toString().replace("application", "job"));
-    
-    sb.append("&reduce=");
-    sb.append(partitionId);
-    sb.append("&map=");
-    URI u = URI.create(sb.toString());
-    return u;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/ShuffleScheduler.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/ShuffleScheduler.java b/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/ShuffleScheduler.java
deleted file mode 100644
index be75668..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/ShuffleScheduler.java
+++ /dev/null
@@ -1,521 +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.engine.common.shuffle.impl;
-
-import java.io.IOException;
-import java.text.DecimalFormat;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Random;
-import java.util.Set;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.DelayQueue;
-import java.util.concurrent.Delayed;
-import java.util.concurrent.TimeUnit;
-
-import org.apache.commons.lang.mutable.MutableInt;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.io.IntWritable;
-import org.apache.tez.common.TezJobConfig;
-import org.apache.tez.common.counters.TezCounter;
-import org.apache.tez.engine.api.Event;
-import org.apache.tez.engine.api.TezInputContext;
-import org.apache.tez.engine.api.events.InputReadErrorEvent;
-import org.apache.tez.engine.common.InputAttemptIdentifier;
-import org.apache.tez.engine.common.TezEngineUtils;
-
-import com.google.common.collect.Lists;
-
-class ShuffleScheduler {
-  static ThreadLocal<Long> shuffleStart = new ThreadLocal<Long>() {
-    protected Long initialValue() {
-      return 0L;
-    }
-  };
-
-  private static final Log LOG = LogFactory.getLog(ShuffleScheduler.class);
-  private static final int MAX_MAPS_AT_ONCE = 20;
-  private static final long INITIAL_PENALTY = 10000;
-  private static final float PENALTY_GROWTH_RATE = 1.3f;
-  
-  // TODO NEWTEZ May need to be a string if attempting to fetch from multiple inputs.
-  private final Map<Integer, MutableInt> finishedMaps;
-  private final int numInputs;
-  private int remainingMaps;
-  private Map<InputAttemptIdentifier, MapHost> mapLocations = new HashMap<InputAttemptIdentifier, MapHost>();
-  //TODO NEWTEZ Clean this and other maps at some point
-  private ConcurrentMap<String, InputAttemptIdentifier> pathToIdentifierMap = new ConcurrentHashMap<String, InputAttemptIdentifier>(); 
-  private Set<MapHost> pendingHosts = new HashSet<MapHost>();
-  private Set<InputAttemptIdentifier> obsoleteMaps = new HashSet<InputAttemptIdentifier>();
-  
-  private final Random random = new Random(System.currentTimeMillis());
-  private final DelayQueue<Penalty> penalties = new DelayQueue<Penalty>();
-  private final Referee referee = new Referee();
-  private final Map<InputAttemptIdentifier, IntWritable> failureCounts =
-    new HashMap<InputAttemptIdentifier,IntWritable>(); 
-  private final Map<String,IntWritable> hostFailures = 
-    new HashMap<String,IntWritable>();
-  private final TezInputContext inputContext;
-  private final Shuffle shuffle;
-  private final int abortFailureLimit;
-  private final TezCounter shuffledMapsCounter;
-  private final TezCounter reduceShuffleBytes;
-  private final TezCounter failedShuffleCounter;
-  
-  private final long startTime;
-  private long lastProgressTime;
-  
-  private int maxMapRuntime = 0;
-  private int maxFailedUniqueFetches = 5;
-  private int maxFetchFailuresBeforeReporting;
-  
-  private long totalBytesShuffledTillNow = 0;
-  private DecimalFormat  mbpsFormat = new DecimalFormat("0.00");
-
-  private boolean reportReadErrorImmediately = true;
-  
-  public ShuffleScheduler(TezInputContext inputContext,
-                          Configuration conf,
-                          int tasksInDegree,
-                          Shuffle shuffle,
-                          TezCounter shuffledMapsCounter,
-                          TezCounter reduceShuffleBytes,
-                          TezCounter failedShuffleCounter) {
-    this.inputContext = inputContext;
-    this.numInputs = tasksInDegree;
-    abortFailureLimit = Math.max(30, tasksInDegree / 10);
-    remainingMaps = tasksInDegree;
-  //TODO NEWTEZ May need to be a string or a more usable construct if attempting to fetch from multiple inputs. Define a taskId / taskAttemptId pair
-    finishedMaps = new HashMap<Integer, MutableInt>(remainingMaps);
-    this.shuffle = shuffle;
-    this.shuffledMapsCounter = shuffledMapsCounter;
-    this.reduceShuffleBytes = reduceShuffleBytes;
-    this.failedShuffleCounter = failedShuffleCounter;
-    this.startTime = System.currentTimeMillis();
-    this.lastProgressTime = startTime;
-    referee.start();
-    this.maxFailedUniqueFetches = Math.min(tasksInDegree,
-        this.maxFailedUniqueFetches);
-    this.maxFetchFailuresBeforeReporting = 
-        conf.getInt(
-            TezJobConfig.TEZ_ENGINE_SHUFFLE_FETCH_FAILURES, 
-            TezJobConfig.DEFAULT_TEZ_ENGINE_SHUFFLE_FETCH_FAILURES_LIMIT);
-    this.reportReadErrorImmediately = 
-        conf.getBoolean(
-            TezJobConfig.TEZ_ENGINE_SHUFFLE_NOTIFY_READERROR, 
-            TezJobConfig.DEFAULT_TEZ_ENGINE_SHUFFLE_NOTIFY_READERROR);
-  }
-
-  public synchronized void copySucceeded(InputAttemptIdentifier srcAttemptIdentifier, 
-                                         MapHost host,
-                                         long bytes,
-                                         long milis,
-                                         MapOutput output
-                                         ) throws IOException {
-    String taskIdentifier = TezEngineUtils.getTaskAttemptIdentifier(srcAttemptIdentifier.getInputIdentifier().getSrcTaskIndex(), srcAttemptIdentifier.getAttemptNumber());
-    failureCounts.remove(taskIdentifier);
-    hostFailures.remove(host.getHostName());
-    
-    if (!isFinishedTaskTrue(srcAttemptIdentifier.getInputIdentifier().getSrcTaskIndex())) {
-      output.commit();
-      if(incrementTaskCopyAndCheckCompletion(srcAttemptIdentifier.getInputIdentifier().getSrcTaskIndex())) {
-        shuffledMapsCounter.increment(1);
-        if (--remainingMaps == 0) {
-          notifyAll();
-        }
-      }
-
-      // update the status
-      lastProgressTime = System.currentTimeMillis();
-      totalBytesShuffledTillNow += bytes;
-      logProgress();
-      reduceShuffleBytes.increment(bytes);
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("src task: "
-            + TezEngineUtils.getTaskAttemptIdentifier(
-                inputContext.getSourceVertexName(), srcAttemptIdentifier.getInputIdentifier().getSrcTaskIndex(),
-                srcAttemptIdentifier.getAttemptNumber()) + " done");
-      }
-    }
-    // TODO NEWTEZ Should this be releasing the output, if not committed ? Possible memory leak in case of speculation.
-  }
-
-  private void logProgress() {
-    float mbs = (float) totalBytesShuffledTillNow / (1024 * 1024);
-    int mapsDone = numInputs - remainingMaps;
-    long secsSinceStart = (System.currentTimeMillis() - startTime) / 1000 + 1;
-
-    float transferRate = mbs / secsSinceStart;
-    LOG.info("copy(" + mapsDone + " of " + numInputs + " at "
-        + mbpsFormat.format(transferRate) + " MB/s)");
-  }
-
-  public synchronized void copyFailed(InputAttemptIdentifier srcAttempt,
-                                      MapHost host,
-                                      boolean readError) {
-    host.penalize();
-    int failures = 1;
-    if (failureCounts.containsKey(srcAttempt)) {
-      IntWritable x = failureCounts.get(srcAttempt);
-      x.set(x.get() + 1);
-      failures = x.get();
-    } else {
-      failureCounts.put(srcAttempt, new IntWritable(1));      
-    }
-    String hostname = host.getHostName();
-    if (hostFailures.containsKey(hostname)) {
-      IntWritable x = hostFailures.get(hostname);
-      x.set(x.get() + 1);
-    } else {
-      hostFailures.put(hostname, new IntWritable(1));
-    }
-    if (failures >= abortFailureLimit) {
-      try {
-        throw new IOException(failures
-            + " failures downloading "
-            + TezEngineUtils.getTaskAttemptIdentifier(
-                inputContext.getSourceVertexName(), srcAttempt.getInputIdentifier().getSrcTaskIndex(),
-                srcAttempt.getAttemptNumber()));
-      } catch (IOException ie) {
-        shuffle.reportException(ie);
-      }
-    }
-    
-    checkAndInformJobTracker(failures, srcAttempt, readError);
-
-    checkReducerHealth();
-    
-    long delay = (long) (INITIAL_PENALTY *
-        Math.pow(PENALTY_GROWTH_RATE, failures));
-    
-    penalties.add(new Penalty(host, delay));
-    
-    failedShuffleCounter.increment(1);
-  }
-  
-  // Notify the JobTracker  
-  // after every read error, if 'reportReadErrorImmediately' is true or
-  // after every 'maxFetchFailuresBeforeReporting' failures
-  private void checkAndInformJobTracker(
-      int failures, InputAttemptIdentifier srcAttempt, boolean readError) {
-    if ((reportReadErrorImmediately && readError)
-        || ((failures % maxFetchFailuresBeforeReporting) == 0)) {
-      LOG.info("Reporting fetch failure for "
-          + TezEngineUtils.getTaskAttemptIdentifier(
-              inputContext.getSourceVertexName(), srcAttempt.getInputIdentifier().getSrcTaskIndex(),
-              srcAttempt.getAttemptNumber()) + " to jobtracker.");
-
-      List<Event> failedEvents = Lists.newArrayListWithCapacity(1);
-      failedEvents.add(new InputReadErrorEvent("Fetch failure for "
-          + TezEngineUtils.getTaskAttemptIdentifier(
-              inputContext.getSourceVertexName(), srcAttempt.getInputIdentifier().getSrcTaskIndex(),
-              srcAttempt.getAttemptNumber()) + " to jobtracker.", srcAttempt.getInputIdentifier()
-          .getSrcTaskIndex(), srcAttempt.getAttemptNumber()));
-
-      inputContext.sendEvents(failedEvents);      
-      //status.addFailedDependency(mapId);
-    }
-  }
-    
-  private void checkReducerHealth() {
-    final float MAX_ALLOWED_FAILED_FETCH_ATTEMPT_PERCENT = 0.5f;
-    final float MIN_REQUIRED_PROGRESS_PERCENT = 0.5f;
-    final float MAX_ALLOWED_STALL_TIME_PERCENT = 0.5f;
-
-    long totalFailures = failedShuffleCounter.getValue();
-    int doneMaps = numInputs - remainingMaps;
-    
-    boolean reducerHealthy =
-      (((float)totalFailures / (totalFailures + doneMaps))
-          < MAX_ALLOWED_FAILED_FETCH_ATTEMPT_PERCENT);
-    
-    // check if the reducer has progressed enough
-    boolean reducerProgressedEnough =
-      (((float)doneMaps / numInputs)
-          >= MIN_REQUIRED_PROGRESS_PERCENT);
-
-    // check if the reducer is stalled for a long time
-    // duration for which the reducer is stalled
-    int stallDuration =
-      (int)(System.currentTimeMillis() - lastProgressTime);
-    
-    // duration for which the reducer ran with progress
-    int shuffleProgressDuration =
-      (int)(lastProgressTime - startTime);
-
-    // min time the reducer should run without getting killed
-    int minShuffleRunDuration =
-      (shuffleProgressDuration > maxMapRuntime)
-      ? shuffleProgressDuration
-          : maxMapRuntime;
-    
-    boolean reducerStalled =
-      (((float)stallDuration / minShuffleRunDuration)
-          >= MAX_ALLOWED_STALL_TIME_PERCENT);
-
-    // kill if not healthy and has insufficient progress
-    if ((failureCounts.size() >= maxFailedUniqueFetches ||
-        failureCounts.size() == (numInputs - doneMaps))
-        && !reducerHealthy
-        && (!reducerProgressedEnough || reducerStalled)) {
-      LOG.fatal("Shuffle failed with too many fetch failures " +
-      "and insufficient progress!");
-      String errorMsg = "Exceeded MAX_FAILED_UNIQUE_FETCHES; bailing-out.";
-      shuffle.reportException(new IOException(errorMsg));
-    }
-
-  }
-  
-  public synchronized void tipFailed(int srcTaskIndex) {
-    if (!isFinishedTaskTrue(srcTaskIndex)) {
-      setFinishedTaskTrue(srcTaskIndex);
-      if (--remainingMaps == 0) {
-        notifyAll();
-      }
-      logProgress();
-    }
-  }
-  
-  public synchronized void addKnownMapOutput(String hostName,
-                                             int partitionId,
-                                             String hostUrl,
-                                             InputAttemptIdentifier srcAttempt) {
-    String identifier = MapHost.createIdentifier(hostName, partitionId);
-    MapHost host = mapLocations.get(identifier);
-    if (host == null) {
-      host = new MapHost(partitionId, hostName, hostUrl);
-      assert identifier.equals(host.getIdentifier());
-      mapLocations.put(srcAttempt, host);
-    }
-    host.addKnownMap(srcAttempt);
-    pathToIdentifierMap.put(srcAttempt.getPathComponent(), srcAttempt);
-
-    // Mark the host as pending
-    if (host.getState() == MapHost.State.PENDING) {
-      pendingHosts.add(host);
-      notifyAll();
-    }
-  }
-  
-  public synchronized void obsoleteMapOutput(InputAttemptIdentifier srcAttempt) {
-    // The incoming srcAttempt does not contain a path component.
-    obsoleteMaps.add(srcAttempt);
-  }
-  
-  public synchronized void putBackKnownMapOutput(MapHost host,
-                                                 InputAttemptIdentifier srcAttempt) {
-    host.addKnownMap(srcAttempt);
-  }
-
-  public synchronized MapHost getHost() throws InterruptedException {
-      while(pendingHosts.isEmpty()) {
-        wait();
-      }
-      
-      MapHost host = null;
-      Iterator<MapHost> iter = pendingHosts.iterator();
-      int numToPick = random.nextInt(pendingHosts.size());
-      for (int i=0; i <= numToPick; ++i) {
-        host = iter.next();
-      }
-      
-      pendingHosts.remove(host);     
-      host.markBusy();
-      
-      LOG.info("Assigning " + host + " with " + host.getNumKnownMapOutputs() + 
-               " to " + Thread.currentThread().getName());
-      shuffleStart.set(System.currentTimeMillis());
-      
-      return host;
-  }
-  
-  public InputAttemptIdentifier getIdentifierForPathComponent(String pathComponent) {
-    return pathToIdentifierMap.get(pathComponent);
-  }
-  
-  public synchronized List<InputAttemptIdentifier> getMapsForHost(MapHost host) {
-    List<InputAttemptIdentifier> list = host.getAndClearKnownMaps();
-    Iterator<InputAttemptIdentifier> itr = list.iterator();
-    List<InputAttemptIdentifier> result = new ArrayList<InputAttemptIdentifier>();
-    int includedMaps = 0;
-    int totalSize = list.size();
-    // find the maps that we still need, up to the limit
-    while (itr.hasNext()) {
-      InputAttemptIdentifier id = itr.next();
-      if (!obsoleteMaps.contains(id) && !isFinishedTaskTrue(id.getInputIdentifier().getSrcTaskIndex())) {
-        result.add(id);
-        if (++includedMaps >= MAX_MAPS_AT_ONCE) {
-          break;
-        }
-      }
-    }
-    // put back the maps left after the limit
-    while (itr.hasNext()) {
-      InputAttemptIdentifier id = itr.next();
-      if (!obsoleteMaps.contains(id) && !isFinishedTaskTrue(id.getInputIdentifier().getSrcTaskIndex())) {
-        host.addKnownMap(id);
-      }
-    }
-    LOG.info("assigned " + includedMaps + " of " + totalSize + " to " +
-             host + " to " + Thread.currentThread().getName());
-    return result;
-  }
-
-  public synchronized void freeHost(MapHost host) {
-    if (host.getState() != MapHost.State.PENALIZED) {
-      if (host.markAvailable() == MapHost.State.PENDING) {
-        pendingHosts.add(host);
-        notifyAll();
-      }
-    }
-    LOG.info(host + " freed by " + Thread.currentThread().getName() + " in " + 
-             (System.currentTimeMillis()-shuffleStart.get()) + "s");
-  }
-    
-  public synchronized void resetKnownMaps() {
-    mapLocations.clear();
-    obsoleteMaps.clear();
-    pendingHosts.clear();
-    pathToIdentifierMap.clear();
-  }
-
-  /**
-   * Utility method to check if the Shuffle data fetch is complete.
-   * @return
-   */
-  public synchronized boolean isDone() {
-    return remainingMaps == 0;
-  }
-
-  /**
-   * Wait until the shuffle finishes or until the timeout.
-   * @param millis maximum wait time
-   * @return true if the shuffle is done
-   * @throws InterruptedException
-   */
-  public synchronized boolean waitUntilDone(int millis
-                                            ) throws InterruptedException {
-    if (remainingMaps > 0) {
-      wait(millis);
-      return remainingMaps == 0;
-    }
-    return true;
-  }
-  
-  /**
-   * A structure that records the penalty for a host.
-   */
-  private static class Penalty implements Delayed {
-    MapHost host;
-    private long endTime;
-    
-    Penalty(MapHost host, long delay) {
-      this.host = host;
-      this.endTime = System.currentTimeMillis() + delay;
-    }
-
-    public long getDelay(TimeUnit unit) {
-      long remainingTime = endTime - System.currentTimeMillis();
-      return unit.convert(remainingTime, TimeUnit.MILLISECONDS);
-    }
-
-    public int compareTo(Delayed o) {
-      long other = ((Penalty) o).endTime;
-      return endTime == other ? 0 : (endTime < other ? -1 : 1);
-    }
-    
-  }
-  
-  /**
-   * A thread that takes hosts off of the penalty list when the timer expires.
-   */
-  private class Referee extends Thread {
-    public Referee() {
-      setName("ShufflePenaltyReferee");
-      setDaemon(true);
-    }
-
-    public void run() {
-      try {
-        while (true) {
-          // take the first host that has an expired penalty
-          MapHost host = penalties.take().host;
-          synchronized (ShuffleScheduler.this) {
-            if (host.markAvailable() == MapHost.State.PENDING) {
-              pendingHosts.add(host);
-              ShuffleScheduler.this.notifyAll();
-            }
-          }
-        }
-      } catch (InterruptedException ie) {
-        return;
-      } catch (Throwable t) {
-        shuffle.reportException(t);
-      }
-    }
-  }
-  
-  public void close() throws InterruptedException {
-    referee.interrupt();
-    referee.join();
-  }
-
-  public synchronized void informMaxMapRunTime(int duration) {
-    if (duration > maxMapRuntime) {
-      maxMapRuntime = duration;
-    }
-  }
-  
-  void setFinishedTaskTrue(int srcTaskIndex) {
-    synchronized(finishedMaps) {
-      finishedMaps.put(srcTaskIndex, new MutableInt(shuffle.getReduceRange()));
-    }
-  }
-  
-  boolean incrementTaskCopyAndCheckCompletion(int srcTaskIndex) {
-    synchronized(finishedMaps) {
-      MutableInt result = finishedMaps.get(srcTaskIndex);
-      if(result == null) {
-        result = new MutableInt(0);
-        finishedMaps.put(srcTaskIndex, result);
-      }
-      result.increment();
-      return isFinishedTaskTrue(srcTaskIndex);
-    }
-  }
-  
-  boolean isFinishedTaskTrue(int srcTaskIndex) {
-    synchronized (finishedMaps) {
-      MutableInt result = finishedMaps.get(srcTaskIndex);
-      if(result == null) {
-        return false;
-      }
-      if (result.intValue() == shuffle.getReduceRange()) {
-        return true;
-      }
-      return false;      
-    }
-  }
-}


[38/50] [abbrv] Rename tez-engine-api to tez-runtime-api and tez-engine is split into 2: - tez-engine-library for user-visible Input/Output/Processor implementations - tez-engine-internals for framework internals

Posted by ss...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/engine/lib/input/ShuffledMergedInput.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/lib/input/ShuffledMergedInput.java b/tez-engine/src/main/java/org/apache/tez/engine/lib/input/ShuffledMergedInput.java
deleted file mode 100644
index a984b0f..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/lib/input/ShuffledMergedInput.java
+++ /dev/null
@@ -1,179 +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.engine.lib.input;
-
-import java.io.IOException;
-import java.util.Collections;
-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.io.RawComparator;
-import org.apache.tez.common.TezJobConfig;
-import org.apache.tez.common.TezUtils;
-import org.apache.tez.common.counters.TaskCounter;
-import org.apache.tez.common.counters.TezCounter;
-import org.apache.tez.engine.api.Event;
-import org.apache.tez.engine.api.KVReader;
-import org.apache.tez.engine.api.LogicalInput;
-import org.apache.tez.engine.api.TezInputContext;
-import org.apache.tez.engine.common.ConfigUtils;
-import org.apache.tez.engine.common.ValuesIterator;
-import org.apache.tez.engine.common.shuffle.impl.Shuffle;
-import org.apache.tez.engine.common.sort.impl.TezRawKeyValueIterator;
-
-/**
- * <code>ShuffleMergedInput</code> in a {@link LogicalInput} which shuffles
- * intermediate sorted data, merges them and provides key/<values> to the
- * consumer.
- *
- * The Copy and Merge will be triggered by the initialization - which is handled
- * by the Tez framework. Input is not consumable until the Copy and Merge are
- * complete. Methods are provided to check for this, as well as to wait for
- * completion. Attempting to get a reader on a non-complete input will block.
- *
- */
-public class ShuffledMergedInput implements LogicalInput {
-
-  static final Log LOG = LogFactory.getLog(ShuffledMergedInput.class);
-
-  protected TezInputContext inputContext;
-  protected TezRawKeyValueIterator rawIter = null;
-  protected Configuration conf;
-  protected int numInputs = 0;
-  protected Shuffle shuffle;
-  @SuppressWarnings("rawtypes")
-  protected ValuesIterator vIter;
-
-  private TezCounter inputKeyCounter;
-  private TezCounter inputValueCounter;
-
-  @Override
-  public List<Event> initialize(TezInputContext inputContext) throws IOException {
-    this.inputContext = inputContext;
-    this.conf = TezUtils.createConfFromUserPayload(inputContext.getUserPayload());
-
-    this.inputKeyCounter = inputContext.getCounters().findCounter(TaskCounter.REDUCE_INPUT_GROUPS);
-    this.inputValueCounter = inputContext.getCounters().findCounter(TaskCounter.REDUCE_INPUT_RECORDS);
-    this.conf.setStrings(TezJobConfig.LOCAL_DIRS,
-        inputContext.getWorkDirs());
-
-    // Start the shuffle - copy and merge.
-    shuffle = new Shuffle(inputContext, this.conf, numInputs);
-    shuffle.run();
-
-    return Collections.emptyList();
-  }
-
-  /**
-   * Check if the input is ready for consumption
-   *
-   * @return true if the input is ready for consumption, or if an error occurred
-   *         processing fetching the input. false if the shuffle and merge are
-   *         still in progress
-   */
-  public boolean isInputReady() {
-    return shuffle.isInputReady();
-  }
-
-  /**
-   * Waits for the input to become ready for consumption
-   * @throws IOException
-   * @throws InterruptedException
-   */
-  public void waitForInputReady() throws IOException, InterruptedException {
-    rawIter = shuffle.waitForInput();
-    createValuesIterator();
-  }
-
-  @Override
-  public List<Event> close() throws IOException {
-    rawIter.close();
-    return Collections.emptyList();
-  }
-
-  /**
-   * Get a KVReader for the Input.</p> This method will block until the input is
-   * ready - i.e. the copy and merge stages are complete. Users can use the
-   * isInputReady method to check if the input is ready, which gives an
-   * indication of whether this method will block or not.
-   *
-   * NOTE: All values for the current K-V pair must be read prior to invoking
-   * moveToNext. Once moveToNext() is called, the valueIterator from the
-   * previous K-V pair will throw an Exception
-   *
-   * @return a KVReader over the sorted input.
-   */
-  @Override
-  public KVReader getReader() throws IOException {
-    if (rawIter == null) {
-      try {
-        waitForInputReady();
-      } catch (InterruptedException e) {
-        Thread.currentThread().interrupt();
-        throw new IOException("Interrupted while waiting for input ready", e);
-      }
-    }
-    return new KVReader() {
-
-      @Override
-      public boolean next() throws IOException {
-        return vIter.moveToNext();
-      }
-
-      @SuppressWarnings("unchecked")
-      @Override
-      public KVRecord getCurrentKV() {
-        return new KVRecord(vIter.getKey(), vIter.getValues());
-      }
-    };
-  }
-
-  @Override
-  public void handleEvents(List<Event> inputEvents) {
-    shuffle.handleEvents(inputEvents);
-  }
-
-  @Override
-  public void setNumPhysicalInputs(int numInputs) {
-    this.numInputs = numInputs;
-  }
-
-  @SuppressWarnings({ "rawtypes", "unchecked" })
-  protected void createValuesIterator()
-      throws IOException {
-    vIter = new ValuesIterator(rawIter,
-        (RawComparator) ConfigUtils.getIntermediateInputKeyComparator(conf),
-        ConfigUtils.getIntermediateInputKeyClass(conf),
-        ConfigUtils.getIntermediateInputValueClass(conf), conf, inputKeyCounter, inputValueCounter);
-
-  }
-
-  // This functionality is currently broken. If there's inputs which need to be
-  // written to disk, there's a possibility that inputs from the different
-  // sources could clobber each others' output. Also the current structures do
-  // not have adequate information to de-dupe these (vertex name)
-//  public void mergeWith(ShuffledMergedInput other) {
-//    this.numInputs += other.getNumPhysicalInputs();
-//  }
-//
-//  public int getNumPhysicalInputs() {
-//    return this.numInputs;
-//  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/engine/lib/input/ShuffledMergedInputLegacy.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/lib/input/ShuffledMergedInputLegacy.java b/tez-engine/src/main/java/org/apache/tez/engine/lib/input/ShuffledMergedInputLegacy.java
deleted file mode 100644
index f2da031..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/lib/input/ShuffledMergedInputLegacy.java
+++ /dev/null
@@ -1,30 +0,0 @@
-/**
- * <code>ShuffleMergedInput</code> in a {@link LogicalInput} which shuffles
- * intermediate sorted data, merges them and provides key/<values> to the
- * consumer.
- * 
- * The Copy and Merge will be triggered by the initialization - which is handled
- * by the Tez framework. Input is not consumable until the Copy and Merge are
- * complete. Methods are provided to check for this, as well as to wait for
- * completion. Attempting to get a reader on a non-complete input will block.
- * 
- */
-
-package org.apache.tez.engine.lib.input;
-
-import java.io.IOException;
-
-import org.apache.hadoop.classification.InterfaceAudience.LimitedPrivate;
-import org.apache.hadoop.classification.InterfaceAudience.Private;
-import org.apache.tez.engine.common.sort.impl.TezRawKeyValueIterator;
-
-@LimitedPrivate("mapreduce")
-public class ShuffledMergedInputLegacy extends ShuffledMergedInput {
-
-  @Private
-  public TezRawKeyValueIterator getIterator() throws IOException, InterruptedException {
-    // wait for input so that iterator is available
-    waitForInputReady();
-    return rawIter;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/engine/lib/input/ShuffledUnorderedKVInput.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/lib/input/ShuffledUnorderedKVInput.java b/tez-engine/src/main/java/org/apache/tez/engine/lib/input/ShuffledUnorderedKVInput.java
deleted file mode 100644
index 44238fd..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/lib/input/ShuffledUnorderedKVInput.java
+++ /dev/null
@@ -1,76 +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.engine.lib.input;
-
-import java.util.List;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.tez.common.TezJobConfig;
-import org.apache.tez.common.TezUtils;
-import org.apache.tez.engine.api.Event;
-import org.apache.tez.engine.api.LogicalInput;
-import org.apache.tez.engine.api.Reader;
-import org.apache.tez.engine.api.TezInputContext;
-import org.apache.tez.engine.broadcast.input.BroadcastShuffleManager;
-
-import com.google.common.base.Preconditions;
-
-public class ShuffledUnorderedKVInput implements LogicalInput {
-
-  private Configuration conf;
-  private int numInputs = -1;
-  private BroadcastShuffleManager shuffleManager;
-  
-  
-  
-  public ShuffledUnorderedKVInput() {
-  }
-
-  @Override
-  public List<Event> initialize(TezInputContext inputContext) throws Exception {
-    Preconditions.checkArgument(numInputs != -1, "Number of Inputs has not been set");
-    this.conf = TezUtils.createConfFromUserPayload(inputContext.getUserPayload());
-    this.conf.setStrings(TezJobConfig.LOCAL_DIRS, inputContext.getWorkDirs());
-    
-    this.shuffleManager = new BroadcastShuffleManager(inputContext, conf, numInputs);
-    return null;
-  }
-
-  @Override
-  public Reader getReader() throws Exception {
-    // TODO Auto-generated method stub
-    return null;
-  }
-
-  @Override
-  public void handleEvents(List<Event> inputEvents) {
-    shuffleManager.handleEvents(inputEvents);
-  }
-
-  @Override
-  public List<Event> close() throws Exception {
-    this.shuffleManager.shutdown();
-    return null;
-  }
-
-  @Override
-  public void setNumPhysicalInputs(int numInputs) {
-    this.numInputs = numInputs;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/engine/lib/output/InMemorySortedOutput.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/lib/output/InMemorySortedOutput.java b/tez-engine/src/main/java/org/apache/tez/engine/lib/output/InMemorySortedOutput.java
deleted file mode 100644
index 26a01c8..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/lib/output/InMemorySortedOutput.java
+++ /dev/null
@@ -1,81 +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.engine.lib.output;
-
-import java.io.IOException;
-import java.util.Collections;
-import java.util.List;
-
-import org.apache.tez.common.TezUtils;
-import org.apache.tez.engine.api.Event;
-import org.apache.tez.engine.api.KVWriter;
-import org.apache.tez.engine.api.LogicalOutput;
-import org.apache.tez.engine.api.Output;
-import org.apache.tez.engine.api.TezOutputContext;
-import org.apache.tez.engine.api.Writer;
-import org.apache.tez.engine.common.sort.impl.dflt.InMemoryShuffleSorter;
-
-/**
- * {@link InMemorySortedOutput} is an {@link Output} which sorts key/value pairs 
- * written to it and persists it to a file.
- */
-public class InMemorySortedOutput implements LogicalOutput {
-  
-  protected InMemoryShuffleSorter sorter;
-  protected int numTasks;
-  protected TezOutputContext outputContext;
-  
-
-  @Override
-  public List<Event> initialize(TezOutputContext outputContext)
-      throws IOException {
-    this.outputContext = outputContext;
-    this.sorter = new InMemoryShuffleSorter();
-    sorter.initialize(outputContext, TezUtils.createConfFromUserPayload(outputContext.getUserPayload()), numTasks);
-    return Collections.emptyList();
-  }
-
-  @Override
-  public Writer getWriter() throws IOException {
-    return new KVWriter() {
-      
-      @Override
-      public void write(Object key, Object value) throws IOException {
-        sorter.write(key, value);
-      }
-    };
-  }
-
-  @Override
-  public void handleEvents(List<Event> outputEvents) {
-    // No events expected.
-  }
-
-  @Override
-  public void setNumPhysicalOutputs(int numOutputs) {
-    this.numTasks = numOutputs;
-  }
-  
-  @Override
-  public List<Event> close() throws IOException {
-    sorter.flush();
-    sorter.close();
-    // TODO NEWTEZ Event generation
-    return null;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/engine/lib/output/LocalOnFileSorterOutput.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/lib/output/LocalOnFileSorterOutput.java b/tez-engine/src/main/java/org/apache/tez/engine/lib/output/LocalOnFileSorterOutput.java
deleted file mode 100644
index 7fd26d7..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/lib/output/LocalOnFileSorterOutput.java
+++ /dev/null
@@ -1,63 +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.engine.lib.output;
-
-import java.io.IOException;
-import java.util.List;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.tez.engine.api.Event;
-import org.apache.tez.engine.common.task.local.output.TezTaskOutput;
-
-public class LocalOnFileSorterOutput extends OnFileSortedOutput {
-
-  private static final Log LOG = LogFactory.getLog(LocalOnFileSorterOutput.class);
-
-  
-
-  @Override
-  public List<Event> close() throws IOException {
-    LOG.debug("Closing LocalOnFileSorterOutput");
-    super.close();
-
-    TezTaskOutput mapOutputFile = sorter.getMapOutput();
-    FileSystem localFs = FileSystem.getLocal(conf);
-
-    Path src = mapOutputFile.getOutputFile();
-    Path dst =
-        mapOutputFile.getInputFileForWrite(
-            outputContext.getTaskIndex(),
-            localFs.getFileStatus(src).getLen());
-
-    LOG.info("Renaming src = " + src + ", dst = " + dst);
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Renaming src = " + src + ", dst = " + dst);
-    }
-    localFs.rename(src, dst);
-    return null;
-  }
-  
-  @Override
-  protected List<Event> generateDataMovementEventsOnClose() throws IOException {
-    return null;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/engine/lib/output/OnFileSortedOutput.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/lib/output/OnFileSortedOutput.java b/tez-engine/src/main/java/org/apache/tez/engine/lib/output/OnFileSortedOutput.java
deleted file mode 100644
index 9c9eba0..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/lib/output/OnFileSortedOutput.java
+++ /dev/null
@@ -1,123 +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.engine.lib.output;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.Collections;
-import java.util.List;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.yarn.api.ApplicationConstants;
-import org.apache.tez.common.TezJobConfig;
-import org.apache.tez.common.TezUtils;
-import org.apache.tez.engine.api.Event;
-import org.apache.tez.engine.api.KVWriter;
-import org.apache.tez.engine.api.LogicalOutput;
-import org.apache.tez.engine.api.TezOutputContext;
-import org.apache.tez.engine.api.events.DataMovementEvent;
-import org.apache.tez.engine.common.shuffle.newimpl.ShuffleUserPayloads.DataMovementEventPayloadProto;
-import org.apache.tez.engine.common.sort.impl.ExternalSorter;
-import org.apache.tez.engine.common.sort.impl.dflt.DefaultSorter;
-import org.apache.tez.engine.shuffle.common.ShuffleUtils;
-
-import com.google.common.collect.Lists;
-
-/**
- * <code>OnFileSortedOutput</code> is an {@link LogicalOutput} which sorts key/value pairs 
- * written to it and persists it to a file.
- */
-public class OnFileSortedOutput implements LogicalOutput {
-  
-  protected ExternalSorter sorter;
-  protected Configuration conf;
-  protected int numOutputs;
-  protected TezOutputContext outputContext;
-  private long startTime;
-  private long endTime;
-  
-  
-  @Override
-  public List<Event> initialize(TezOutputContext outputContext)
-      throws IOException {
-    this.startTime = System.nanoTime();
-    this.outputContext = outputContext;
-    sorter = new DefaultSorter();
-    this.conf = TezUtils.createConfFromUserPayload(outputContext.getUserPayload());
-    // Initializing this parametr in this conf since it is used in multiple
-    // places (wherever LocalDirAllocator is used) - TezTaskOutputFiles,
-    // TezMerger, etc.
-    this.conf.setStrings(TezJobConfig.LOCAL_DIRS, outputContext.getWorkDirs());
-    sorter.initialize(outputContext, conf, numOutputs);
-    return Collections.emptyList();
-  }
-
-  @Override
-  public KVWriter getWriter() throws IOException {
-    return new KVWriter() {
-      @Override
-      public void write(Object key, Object value) throws IOException {
-        sorter.write(key, value);
-      }
-    };
-  }
-
-  @Override
-  public void handleEvents(List<Event> outputEvents) {
-    // Not expecting any events.
-  }
-
-  @Override
-  public void setNumPhysicalOutputs(int numOutputs) {
-    this.numOutputs = numOutputs;
-  }
-
-  @Override
-  public List<Event> close() throws IOException {
-    sorter.flush();
-    sorter.close();
-    this.endTime = System.nanoTime();
-
-   return generateDataMovementEventsOnClose();
-  }
-  
-  protected List<Event> generateDataMovementEventsOnClose() throws IOException {
-    String host = System.getenv(ApplicationConstants.Environment.NM_HOST
-        .toString());
-    ByteBuffer shuffleMetadata = outputContext
-        .getServiceProviderMetaData(ShuffleUtils.SHUFFLE_HANDLER_SERVICE_ID);
-    int shufflePort = ShuffleUtils.deserializeShuffleProviderMetaData(shuffleMetadata);
-
-    DataMovementEventPayloadProto.Builder payloadBuilder = DataMovementEventPayloadProto
-        .newBuilder();
-    payloadBuilder.setHost(host);
-    payloadBuilder.setPort(shufflePort);
-    payloadBuilder.setPathComponent(outputContext.getUniqueIdentifier());
-    payloadBuilder.setRunDuration((int) ((endTime - startTime) / 1000));
-    DataMovementEventPayloadProto payloadProto = payloadBuilder.build();
-    byte[] payloadBytes = payloadProto.toByteArray();
-
-    List<Event> events = Lists.newArrayListWithCapacity(numOutputs);
-
-    for (int i = 0; i < numOutputs; i++) {
-      DataMovementEvent event = new DataMovementEvent(i, payloadBytes);
-      events.add(event);
-    }
-    return events;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/engine/lib/output/OnFileUnorderedKVOutput.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/lib/output/OnFileUnorderedKVOutput.java b/tez-engine/src/main/java/org/apache/tez/engine/lib/output/OnFileUnorderedKVOutput.java
deleted file mode 100644
index 3ff603f..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/lib/output/OnFileUnorderedKVOutput.java
+++ /dev/null
@@ -1,98 +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.engine.lib.output;
-
-import java.nio.ByteBuffer;
-import java.util.Collections;
-import java.util.List;
-
-import org.apache.hadoop.yarn.api.ApplicationConstants;
-import org.apache.tez.dag.api.TezUncheckedException;
-import org.apache.tez.engine.api.Event;
-import org.apache.tez.engine.api.KVWriter;
-import org.apache.tez.engine.api.LogicalOutput;
-import org.apache.tez.engine.api.TezOutputContext;
-import org.apache.tez.engine.api.events.DataMovementEvent;
-import org.apache.tez.engine.broadcast.output.FileBasedKVWriter;
-import org.apache.tez.engine.common.shuffle.newimpl.ShuffleUserPayloads.DataMovementEventPayloadProto;
-import org.apache.tez.engine.shuffle.common.ShuffleUtils;
-
-import com.google.common.base.Preconditions;
-import com.google.common.collect.Lists;
-
-public class OnFileUnorderedKVOutput implements LogicalOutput {
-
-  private TezOutputContext outputContext;
-  private FileBasedKVWriter kvWriter;
-
-  public OnFileUnorderedKVOutput() {
-  }
-
-  @Override
-  public List<Event> initialize(TezOutputContext outputContext)
-      throws Exception {
-    this.outputContext = outputContext;
-    this.kvWriter = new FileBasedKVWriter(outputContext);
-    return Collections.emptyList();
-  }
-
-  @Override
-  public KVWriter getWriter() throws Exception {
-    return kvWriter;
-  }
-
-  @Override
-  public void handleEvents(List<Event> outputEvents) {
-    throw new TezUncheckedException("Not expecting any events");
-  }
-
-  @Override
-  public List<Event> close() throws Exception {
-    boolean outputGenerated = this.kvWriter.close();
-    DataMovementEventPayloadProto.Builder payloadBuilder = DataMovementEventPayloadProto
-        .newBuilder();
-
-    String host = System.getenv(ApplicationConstants.Environment.NM_HOST
-        .toString());
-    ByteBuffer shuffleMetadata = outputContext
-        .getServiceProviderMetaData(ShuffleUtils.SHUFFLE_HANDLER_SERVICE_ID);
-    int shufflePort = ShuffleUtils
-        .deserializeShuffleProviderMetaData(shuffleMetadata);
-    payloadBuilder.setOutputGenerated(outputGenerated);
-    if (outputGenerated) {
-      payloadBuilder.setHost(host);
-      payloadBuilder.setPort(shufflePort);
-      payloadBuilder.setPathComponent(outputContext.getUniqueIdentifier());
-    }
-    DataMovementEventPayloadProto payloadProto = payloadBuilder.build();
-
-    DataMovementEvent dmEvent = new DataMovementEvent(0,
-        payloadProto.toByteArray());
-    List<Event> events = Lists.newArrayListWithCapacity(1);
-    events.add(dmEvent);
-    return events;
-  }
-
-  @Override
-  public void setNumPhysicalOutputs(int numOutputs) {
-    Preconditions.checkArgument(numOutputs == 1,
-        "Number of outputs can only be 1 for " + this.getClass().getName());
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/engine/newruntime/LogicalIOProcessorRuntimeTask.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/newruntime/LogicalIOProcessorRuntimeTask.java b/tez-engine/src/main/java/org/apache/tez/engine/newruntime/LogicalIOProcessorRuntimeTask.java
deleted file mode 100644
index 29063f9..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/newruntime/LogicalIOProcessorRuntimeTask.java
+++ /dev/null
@@ -1,475 +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.engine.newruntime;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.LinkedHashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.LinkedBlockingQueue;
-
-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.token.Token;
-import org.apache.hadoop.util.StringUtils;
-import org.apache.tez.dag.api.ProcessorDescriptor;
-import org.apache.tez.dag.api.TezUncheckedException;
-import org.apache.tez.dag.records.TezTaskAttemptID;
-import org.apache.tez.engine.api.Event;
-import org.apache.tez.engine.api.Input;
-import org.apache.tez.engine.api.LogicalIOProcessor;
-import org.apache.tez.engine.api.LogicalInput;
-import org.apache.tez.engine.api.LogicalOutput;
-import org.apache.tez.engine.api.Output;
-import org.apache.tez.engine.api.Processor;
-import org.apache.tez.engine.api.TezInputContext;
-import org.apache.tez.engine.api.TezOutputContext;
-import org.apache.tez.engine.api.TezProcessorContext;
-import org.apache.tez.engine.api.impl.EventMetaData;
-import org.apache.tez.engine.api.impl.InputSpec;
-import org.apache.tez.engine.api.impl.OutputSpec;
-import org.apache.tez.engine.api.impl.TaskSpec;
-import org.apache.tez.engine.api.impl.TezEvent;
-import org.apache.tez.engine.api.impl.TezInputContextImpl;
-import org.apache.tez.engine.api.impl.TezOutputContextImpl;
-import org.apache.tez.engine.api.impl.TezProcessorContextImpl;
-import org.apache.tez.engine.api.impl.TezUmbilical;
-import org.apache.tez.engine.api.impl.EventMetaData.EventProducerConsumerType;
-import org.apache.tez.engine.common.security.JobTokenIdentifier;
-import org.apache.tez.engine.shuffle.common.ShuffleUtils;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Preconditions;
-
-@Private
-public class LogicalIOProcessorRuntimeTask extends RuntimeTask {
-
-  private static final Log LOG = LogFactory
-      .getLog(LogicalIOProcessorRuntimeTask.class);
-
-  private final List<InputSpec> inputSpecs;
-  private final List<LogicalInput> inputs;
-
-  private final List<OutputSpec> outputSpecs;
-  private final List<LogicalOutput> outputs;
-
-  private List<TezInputContext> inputContexts;
-  private List<TezOutputContext> outputContexts;
-  private TezProcessorContext processorContext;
-  
-  private final ProcessorDescriptor processorDescriptor;
-  private final LogicalIOProcessor processor;
-
-  private final Map<String, ByteBuffer> serviceConsumerMetadata;
-
-  private Map<String, LogicalInput> inputMap;
-  private Map<String, LogicalOutput> outputMap;
-
-  private LinkedBlockingQueue<TezEvent> eventsToBeProcessed;
-  private Thread eventRouterThread = null;
-
-  private final int appAttemptNumber;
-
-  public LogicalIOProcessorRuntimeTask(TaskSpec taskSpec, int appAttemptNumber,
-      Configuration tezConf, TezUmbilical tezUmbilical,
-      Token<JobTokenIdentifier> jobToken) throws IOException {
-    // TODO Remove jobToken from here post TEZ-421
-    super(taskSpec, tezConf, tezUmbilical);
-    LOG.info("Initializing LogicalIOProcessorRuntimeTask with TaskSpec: "
-        + taskSpec);
-    this.inputContexts = new ArrayList<TezInputContext>(taskSpec.getInputs().size());
-    this.outputContexts = new ArrayList<TezOutputContext>(taskSpec.getOutputs().size());
-    this.inputSpecs = taskSpec.getInputs();
-    this.inputs = createInputs(inputSpecs);
-    this.outputSpecs = taskSpec.getOutputs();
-    this.outputs = createOutputs(outputSpecs);
-    this.processorDescriptor = taskSpec.getProcessorDescriptor();
-    this.processor = createProcessor(processorDescriptor);
-    this.serviceConsumerMetadata = new HashMap<String, ByteBuffer>();
-    this.serviceConsumerMetadata.put(ShuffleUtils.SHUFFLE_HANDLER_SERVICE_ID,
-        ShuffleUtils.convertJobTokenToBytes(jobToken));
-    this.eventsToBeProcessed = new LinkedBlockingQueue<TezEvent>();
-    this.state = State.NEW;
-    this.appAttemptNumber = appAttemptNumber;
-  }
-
-  public void initialize() throws Exception {
-    LOG.info("Initializing LogicalProcessorIORuntimeTask");
-    Preconditions.checkState(this.state == State.NEW, "Already initialized");
-    this.state = State.INITED;
-    inputMap = new LinkedHashMap<String, LogicalInput>(inputs.size());
-    outputMap = new LinkedHashMap<String, LogicalOutput>(outputs.size());
-
-    // TODO Maybe close initialized inputs / outputs in case of failure to
-    // initialize.
-    // Initialize all inputs. TODO: Multi-threaded at some point.
-    for (int i = 0; i < inputs.size(); i++) {
-      String srcVertexName = inputSpecs.get(i).getSourceVertexName();
-      initializeInput(inputs.get(i),
-          inputSpecs.get(i));
-      inputMap.put(srcVertexName, inputs.get(i));
-    }
-
-    // Initialize all outputs. TODO: Multi-threaded at some point.
-    for (int i = 0; i < outputs.size(); i++) {
-      String destVertexName = outputSpecs.get(i).getDestinationVertexName();
-      initializeOutput(outputs.get(i), outputSpecs.get(i));
-      outputMap.put(destVertexName, outputs.get(i));
-    }
-
-    // Initialize processor.
-    initializeLogicalIOProcessor();
-    startRouterThread();
-  }
-
-  public void run() throws Exception {
-    synchronized (this.state) {
-      Preconditions.checkState(this.state == State.INITED,
-          "Can only run while in INITED state. Current: " + this.state);
-      this.state = State.RUNNING;
-    }
-    LogicalIOProcessor lioProcessor = (LogicalIOProcessor) processor;
-    lioProcessor.run(inputMap, outputMap);
-  }
-
-  public void close() throws Exception {
-    try {
-      Preconditions.checkState(this.state == State.RUNNING,
-          "Can only run while in RUNNING state. Current: " + this.state);
-      this.state = State.CLOSED;
-
-      // Close the Inputs.
-      for (int i = 0; i < inputs.size(); i++) {
-        String srcVertexName = inputSpecs.get(i).getSourceVertexName();
-        List<Event> closeInputEvents = inputs.get(i).close();
-        sendTaskGeneratedEvents(closeInputEvents,
-            EventProducerConsumerType.INPUT, taskSpec.getVertexName(),
-            srcVertexName, taskSpec.getTaskAttemptID());
-      }
-
-      // Close the Processor.
-      processor.close();
-
-      // Close the Outputs.
-      for (int i = 0; i < outputs.size(); i++) {
-        String destVertexName = outputSpecs.get(i).getDestinationVertexName();
-        List<Event> closeOutputEvents = outputs.get(i).close();
-        sendTaskGeneratedEvents(closeOutputEvents,
-            EventProducerConsumerType.OUTPUT, taskSpec.getVertexName(),
-            destVertexName, taskSpec.getTaskAttemptID());
-      }
-    } finally {
-      setTaskDone();
-      if (eventRouterThread != null) {
-        eventRouterThread.interrupt();
-      }
-    }
-  }
-
-  private void initializeInput(Input input, InputSpec inputSpec)
-      throws Exception {
-    TezInputContext tezInputContext = createInputContext(inputSpec);
-    inputContexts.add(tezInputContext);
-    if (input instanceof LogicalInput) {
-      ((LogicalInput) input).setNumPhysicalInputs(inputSpec
-          .getPhysicalEdgeCount());
-    }
-    LOG.info("Initializing Input using InputSpec: " + inputSpec);
-    List<Event> events = input.initialize(tezInputContext);
-    sendTaskGeneratedEvents(events, EventProducerConsumerType.INPUT,
-        tezInputContext.getTaskVertexName(),
-        tezInputContext.getSourceVertexName(), taskSpec.getTaskAttemptID());
-  }
-
-  private void initializeOutput(Output output, OutputSpec outputSpec)
-      throws Exception {
-    TezOutputContext tezOutputContext = createOutputContext(outputSpec);
-    outputContexts.add(tezOutputContext);
-    if (output instanceof LogicalOutput) {
-      ((LogicalOutput) output).setNumPhysicalOutputs(outputSpec
-          .getPhysicalEdgeCount());
-    }
-    LOG.info("Initializing Output using OutputSpec: " + outputSpec);
-    List<Event> events = output.initialize(tezOutputContext);
-    sendTaskGeneratedEvents(events, EventProducerConsumerType.OUTPUT,
-        tezOutputContext.getTaskVertexName(),
-        tezOutputContext.getDestinationVertexName(),
-        taskSpec.getTaskAttemptID());
-  }
-
-  private void initializeLogicalIOProcessor() throws Exception {
-    LOG.info("Initializing processor"
-        + ", processorClassName=" + processorDescriptor.getClassName());
-    TezProcessorContext processorContext = createProcessorContext();
-    this.processorContext = processorContext;
-    processor.initialize(processorContext);
-  }
-
-  private TezInputContext createInputContext(InputSpec inputSpec) {
-    TezInputContext inputContext = new TezInputContextImpl(tezConf,
-        appAttemptNumber, tezUmbilical, taskSpec.getVertexName(),
-        inputSpec.getSourceVertexName(), taskSpec.getTaskAttemptID(),
-        tezCounters,
-        inputSpec.getInputDescriptor().getUserPayload() == null ? taskSpec
-            .getProcessorDescriptor().getUserPayload() : inputSpec
-            .getInputDescriptor().getUserPayload(), this,
-        serviceConsumerMetadata);
-    return inputContext;
-  }
-
-  private TezOutputContext createOutputContext(OutputSpec outputSpec) {
-    TezOutputContext outputContext = new TezOutputContextImpl(tezConf,
-        appAttemptNumber, tezUmbilical, taskSpec.getVertexName(),
-        outputSpec.getDestinationVertexName(), taskSpec.getTaskAttemptID(),
-        tezCounters,
-        outputSpec.getOutputDescriptor().getUserPayload() == null ? taskSpec
-            .getProcessorDescriptor().getUserPayload() : outputSpec
-            .getOutputDescriptor().getUserPayload(), this,
-        serviceConsumerMetadata);
-    return outputContext;
-  }
-
-  private TezProcessorContext createProcessorContext() {
-    TezProcessorContext processorContext = new TezProcessorContextImpl(tezConf,
-        appAttemptNumber, tezUmbilical, taskSpec.getVertexName(), taskSpec.getTaskAttemptID(),
-        tezCounters, processorDescriptor.getUserPayload(), this,
-        serviceConsumerMetadata);
-    return processorContext;
-  }
-
-  private List<LogicalInput> createInputs(List<InputSpec> inputSpecs) {
-    List<LogicalInput> inputs = new ArrayList<LogicalInput>(inputSpecs.size());
-    for (InputSpec inputSpec : inputSpecs) {
-      LOG.info("Creating Input from InputSpec: "
-          + inputSpec);
-      Input input = RuntimeUtils.createClazzInstance(inputSpec
-          .getInputDescriptor().getClassName());
-
-      if (input instanceof LogicalInput) {
-        inputs.add((LogicalInput) input);
-      } else {
-        throw new TezUncheckedException(input.getClass().getName()
-            + " is not a sub-type of LogicalInput."
-            + " Only LogicalInput sub-types supported by LogicalIOProcessor.");
-      }
-    }
-    return inputs;
-  }
-
-  private List<LogicalOutput> createOutputs(List<OutputSpec> outputSpecs) {
-    List<LogicalOutput> outputs = new ArrayList<LogicalOutput>(
-        outputSpecs.size());
-    for (OutputSpec outputSpec : outputSpecs) {
-      LOG.info("Creating Output from OutputSpec"
-          + outputSpec);
-      Output output = RuntimeUtils.createClazzInstance(outputSpec
-          .getOutputDescriptor().getClassName());
-      if (output instanceof LogicalOutput) {
-        outputs.add((LogicalOutput) output);
-      } else {
-        throw new TezUncheckedException(output.getClass().getName()
-            + " is not a sub-type of LogicalOutput."
-            + " Only LogicalOutput sub-types supported by LogicalIOProcessor.");
-      }
-    }
-    return outputs;
-  }
-
-  private LogicalIOProcessor createProcessor(
-      ProcessorDescriptor processorDescriptor) {
-    Processor processor = RuntimeUtils.createClazzInstance(processorDescriptor
-        .getClassName());
-    if (!(processor instanceof LogicalIOProcessor)) {
-      throw new TezUncheckedException(processor.getClass().getName()
-          + " is not a sub-type of LogicalIOProcessor."
-          + " Only LogicalOutput sub-types supported by LogicalIOProcessor.");
-    }
-    return (LogicalIOProcessor) processor;
-  }
-
-  private void sendTaskGeneratedEvents(List<Event> events,
-      EventProducerConsumerType generator, String taskVertexName,
-      String edgeVertexName, TezTaskAttemptID taskAttemptID) {
-    if (events == null || events.isEmpty()) {
-      return;
-    }
-    EventMetaData eventMetaData = new EventMetaData(generator,
-        taskVertexName, edgeVertexName, taskAttemptID);
-    List<TezEvent> tezEvents = new ArrayList<TezEvent>(events.size());
-    for (Event e : events) {
-      TezEvent te = new TezEvent(e, eventMetaData);
-      tezEvents.add(te);
-    }
-    if (LOG.isDebugEnabled()) {
-      for (TezEvent e : tezEvents) {
-        LOG.debug("Generated event info"
-            + ", eventMetaData=" + eventMetaData.toString()
-            + ", eventType=" + e.getEventType());
-      }
-    }
-    tezUmbilical.addEvents(tezEvents);
-  }
-
-  private boolean handleEvent(TezEvent e) {
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Handling TezEvent in task"
-          + ", taskAttemptId=" + taskSpec.getTaskAttemptID()
-          + ", eventType=" + e.getEventType()
-          + ", eventSourceInfo=" + e.getSourceInfo()
-          + ", eventDestinationInfo=" + e.getDestinationInfo());
-    }
-    try {
-      switch (e.getDestinationInfo().getEventGenerator()) {
-      case INPUT:
-        LogicalInput input = inputMap.get(
-            e.getDestinationInfo().getEdgeVertexName());
-        if (input != null) {
-          input.handleEvents(Collections.singletonList(e.getEvent()));
-        } else {
-          throw new TezUncheckedException("Unhandled event for invalid target: "
-              + e);
-        }
-        break;
-      case OUTPUT:
-        LogicalOutput output = outputMap.get(
-            e.getDestinationInfo().getEdgeVertexName());
-        if (output != null) {
-          output.handleEvents(Collections.singletonList(e.getEvent()));
-        } else {
-          throw new TezUncheckedException("Unhandled event for invalid target: "
-              + e);
-        }
-        break;
-      case PROCESSOR:
-        processor.handleEvents(Collections.singletonList(e.getEvent()));
-        break;
-      case SYSTEM:
-        LOG.warn("Trying to send a System event in a Task: " + e);
-        break;
-      }
-    } catch (Throwable t) {
-      LOG.warn("Failed to handle event", t);
-      setFatalError(t, "Failed to handle event");
-      EventMetaData sourceInfo = new EventMetaData(
-          e.getDestinationInfo().getEventGenerator(),
-          taskSpec.getVertexName(), e.getDestinationInfo().getEdgeVertexName(),
-          getTaskAttemptID());
-      tezUmbilical.signalFatalError(getTaskAttemptID(),
-          StringUtils.stringifyException(t), sourceInfo);
-      return false;
-    }
-    return true;
-  }
-
-  @Override
-  public synchronized void handleEvents(Collection<TezEvent> events) {
-    if (events == null || events.isEmpty()) {
-      return;
-    }
-    eventCounter.addAndGet(events.size());
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Received events to be processed by task"
-          + ", taskAttemptId=" + taskSpec.getTaskAttemptID()
-          + ", eventCount=" + events.size()
-          + ", newEventCounter=" + eventCounter.get());
-    }
-    eventsToBeProcessed.addAll(events);
-  }
-
-  private void startRouterThread() {
-    eventRouterThread = new Thread(new Runnable() {
-      public void run() {
-        while (!isTaskDone() && !Thread.currentThread().isInterrupted()) {
-          try {
-            TezEvent e = eventsToBeProcessed.take();
-            if (e == null) {
-              continue;
-            }
-            // TODO TODONEWTEZ
-            if (!handleEvent(e)) {
-              LOG.warn("Stopping Event Router thread as failed to handle"
-                  + " event: " + e);
-              return;
-            }
-          } catch (InterruptedException e) {
-            if (!isTaskDone()) {
-              LOG.warn("Event Router thread interrupted. Returning.");
-            }
-            return;
-          }
-        }
-      }
-    });
-
-    eventRouterThread.setName("TezTaskEventRouter["
-        + taskSpec.getTaskAttemptID().toString() + "]");
-    eventRouterThread.start();
-  }
-
-  public synchronized void cleanup() {
-    setTaskDone();
-    if (eventRouterThread != null) {
-      eventRouterThread.interrupt();
-    }
-  }
-  
-  @Private
-  @VisibleForTesting
-  public List<TezInputContext> getInputContexts() {
-    return this.inputContexts;
-  }
-  
-  @Private
-  @VisibleForTesting
-  public List<TezOutputContext> getOutputContexts() {
-    return this.outputContexts;
-  }
-
-  @Private
-  @VisibleForTesting
-  public TezProcessorContext getProcessorContext() {
-    return this.processorContext;
-  }
-  
-  @Private
-  @VisibleForTesting
-  public Map<String, LogicalInput> getInputs() {
-    return this.inputMap;
-  }
-  
-  @Private
-  @VisibleForTesting
-  public Map<String, LogicalOutput> getOutputs() {
-    return this.outputMap;
-  }
-  
-  @Private
-  @VisibleForTesting
-  public LogicalIOProcessor getProcessor() {
-    return this.processor;
-  }
-  
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/engine/newruntime/RuntimeTask.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/newruntime/RuntimeTask.java b/tez-engine/src/main/java/org/apache/tez/engine/newruntime/RuntimeTask.java
deleted file mode 100644
index 22cbc7c..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/newruntime/RuntimeTask.java
+++ /dev/null
@@ -1,106 +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.engine.newruntime;
-
-import java.util.Collection;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicInteger;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.tez.common.counters.TezCounters;
-import org.apache.tez.dag.records.TezTaskAttemptID;
-import org.apache.tez.engine.api.impl.TaskSpec;
-import org.apache.tez.engine.api.impl.TezEvent;
-import org.apache.tez.engine.api.impl.TezUmbilical;
-
-public abstract class RuntimeTask {
-
-  protected AtomicBoolean hasFatalError = new AtomicBoolean(false);
-  protected Throwable fatalError = null;
-  protected String fatalErrorMessage = null;
-  protected float progress;
-  protected final TezCounters tezCounters;
-  protected final TaskSpec taskSpec;
-  protected final Configuration tezConf;
-  protected final TezUmbilical tezUmbilical;
-  protected final AtomicInteger eventCounter;
-  private final AtomicBoolean taskDone;
-
-  protected RuntimeTask(TaskSpec taskSpec, Configuration tezConf,
-      TezUmbilical tezUmbilical) {
-    this.taskSpec = taskSpec;
-    this.tezConf = tezConf;
-    this.tezUmbilical = tezUmbilical;
-    this.tezCounters = new TezCounters();
-    this.eventCounter = new AtomicInteger(0);
-    this.progress = 0.0f;
-    this.taskDone = new AtomicBoolean(false);
-  }
-
-  protected enum State {
-    NEW, INITED, RUNNING, CLOSED;
-  }
-
-  protected State state;
-
-  public String getVertexName() {
-    return taskSpec.getVertexName();
-  }
-
-  public void setFatalError(Throwable t, String message) {
-    hasFatalError.set(true);
-    this.fatalError = t;
-    this.fatalErrorMessage = message;
-  }
-
-  public boolean hadFatalError() {
-    return hasFatalError.get();
-  }
-
-  public synchronized void setProgress(float progress) {
-    this.progress = progress;
-  }
-
-  public synchronized float getProgress() {
-    return this.progress;
-  }
-
-  public TezCounters getCounters() {
-    return this.tezCounters;
-  }
-
-  public TezTaskAttemptID getTaskAttemptID() {
-    return taskSpec.getTaskAttemptID();
-  }
-
-  public abstract void handleEvents(Collection<TezEvent> events);
-
-  public int getEventCounter() {
-    return eventCounter.get();
-  }
-
-  public boolean isTaskDone() {
-    return taskDone.get();
-  }
-
-  protected void setTaskDone() {
-    taskDone.set(true);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/engine/newruntime/RuntimeUtils.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/newruntime/RuntimeUtils.java b/tez-engine/src/main/java/org/apache/tez/engine/newruntime/RuntimeUtils.java
deleted file mode 100644
index 20a029e..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/newruntime/RuntimeUtils.java
+++ /dev/null
@@ -1,62 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tez.engine.newruntime;
-
-import java.util.Map;
-import java.util.concurrent.ConcurrentHashMap;
-
-import org.apache.tez.dag.api.TezUncheckedException;
-
-public class RuntimeUtils {
-
-  private static final Map<String, Class<?>> CLAZZ_CACHE = new ConcurrentHashMap<String, Class<?>>();
-
-  private static Class<?> getClazz(String className) {
-    Class<?> clazz = CLAZZ_CACHE.get(className);
-    if (clazz == null) {
-      try {
-        clazz = Class.forName(className);
-      } catch (ClassNotFoundException e) {
-        throw new TezUncheckedException("Unable to load class: " + className, e);
-      }
-    }
-    return clazz;
-  }
-
-  private static <T> T getNewInstance(Class<T> clazz) {
-    T instance;
-    try {
-      instance = clazz.newInstance();
-    } catch (InstantiationException e) {
-      throw new TezUncheckedException(
-          "Unable to instantiate class with 0 arguments: " + clazz.getName(), e);
-    } catch (IllegalAccessException e) {
-      throw new TezUncheckedException(
-          "Unable to instantiate class with 0 arguments: " + clazz.getName(), e);
-    }
-    return instance;
-  }
-
-  public static <T> T createClazzInstance(String className) {
-    Class<?> clazz = getClazz(className);
-    @SuppressWarnings("unchecked")
-    T instance = (T) getNewInstance(clazz);
-    return instance;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/engine/shuffle/common/DiskFetchedInput.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/shuffle/common/DiskFetchedInput.java b/tez-engine/src/main/java/org/apache/tez/engine/shuffle/common/DiskFetchedInput.java
deleted file mode 100644
index 531e460..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/shuffle/common/DiskFetchedInput.java
+++ /dev/null
@@ -1,111 +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.engine.shuffle.common;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-
-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.tez.engine.common.InputAttemptIdentifier;
-import org.apache.tez.engine.common.task.local.output.TezTaskOutputFiles;
-
-import com.google.common.base.Preconditions;
-
-public class DiskFetchedInput extends FetchedInput {
-
-  private static final Log LOG = LogFactory.getLog(DiskFetchedInput.class);
-  
-  private final FileSystem localFS;
-  private final Path tmpOutputPath;
-  private final Path outputPath;
-
-  public DiskFetchedInput(long size,
-      InputAttemptIdentifier inputAttemptIdentifier,
-      FetchedInputCallback callbackHandler, Configuration conf,
-      LocalDirAllocator localDirAllocator, TezTaskOutputFiles filenameAllocator)
-      throws IOException {
-    super(Type.DISK, size, inputAttemptIdentifier, callbackHandler);
-
-    this.localFS = FileSystem.getLocal(conf);
-    this.outputPath = filenameAllocator.getInputFileForWrite(
-        this.inputAttemptIdentifier.getInputIdentifier().getSrcTaskIndex(), size);
-    this.tmpOutputPath = outputPath.suffix(String.valueOf(id));
-  }
-
-  @Override
-  public OutputStream getOutputStream() throws IOException {
-    return localFS.create(tmpOutputPath);
-  }
-
-  @Override
-  public InputStream getInputStream() throws IOException {
-    return localFS.open(outputPath);
-  }
-
-  @Override
-  public void commit() throws IOException {
-    if (state == State.PENDING) {
-      state = State.COMMITTED;
-      localFS.rename(tmpOutputPath, outputPath);
-      notifyFetchComplete();
-    }
-  }
-
-  @Override
-  public void abort() throws IOException {
-    if (state == State.PENDING) {
-      state = State.ABORTED;
-      // TODO NEWTEZ Maybe defer this to container cleanup
-      localFS.delete(tmpOutputPath, false);
-      notifyFetchFailure();
-    }
-  }
-  
-  @Override
-  public void free() {
-    Preconditions.checkState(
-        state == State.COMMITTED || state == State.ABORTED,
-        "FetchedInput can only be freed after it is committed or aborted");
-    if (state == State.COMMITTED) {
-      state = State.FREED;
-      try {
-        // TODO NEWTEZ Maybe defer this to container cleanup
-        localFS.delete(outputPath, false);
-      } catch (IOException e) {
-        // Ignoring the exception, will eventually be cleaned by container
-        // cleanup.
-        LOG.warn("Failed to remvoe file : " + outputPath.toString());
-      }
-      notifyFreedResource();
-    }
-  }
-
-  @Override
-  public String toString() {
-    return "DiskFetchedInput [outputPath=" + outputPath
-        + ", inputAttemptIdentifier=" + inputAttemptIdentifier + ", size="
-        + size + ", type=" + type + ", id=" + id + ", state=" + state + "]";
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/engine/shuffle/common/FetchResult.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/shuffle/common/FetchResult.java b/tez-engine/src/main/java/org/apache/tez/engine/shuffle/common/FetchResult.java
deleted file mode 100644
index fb0b324..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/shuffle/common/FetchResult.java
+++ /dev/null
@@ -1,70 +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.engine.shuffle.common;
-/**
- * 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.
- */
-
-import org.apache.tez.engine.common.InputAttemptIdentifier;
-
-public class FetchResult {
-
-  private final String host;
-  private final int port;
-  private final int partition;
-  private final Iterable<InputAttemptIdentifier> pendingInputs;
-
-  public FetchResult(String host, int port, int partition,
-      Iterable<InputAttemptIdentifier> pendingInputs) {
-    this.host = host;
-    this.port = port;
-    this.partition = partition;
-    this.pendingInputs = pendingInputs;
-  }
-
-  public String getHost() {
-    return host;
-  }
-
-  public int getPort() {
-    return port;
-  }
-
-  public int getPartition() {
-    return partition;
-  }
-
-  public Iterable<InputAttemptIdentifier> getPendingInputs() {
-    return pendingInputs;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/engine/shuffle/common/FetchedInput.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/shuffle/common/FetchedInput.java b/tez-engine/src/main/java/org/apache/tez/engine/shuffle/common/FetchedInput.java
deleted file mode 100644
index f5339d3..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/shuffle/common/FetchedInput.java
+++ /dev/null
@@ -1,144 +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.engine.shuffle.common;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.util.concurrent.atomic.AtomicInteger;
-
-import org.apache.hadoop.classification.InterfaceAudience.Private;
-import org.apache.tez.engine.common.InputAttemptIdentifier;
-
-@Private
-public abstract class FetchedInput {
-  
-  public static enum Type {
-    WAIT, // TODO NEWTEZ Implement this, only if required.
-    MEMORY,
-    DISK,
-  }
-  
-  protected static enum State {
-    PENDING, COMMITTED, ABORTED, FREED
-  }
-
-  private static AtomicInteger ID_GEN = new AtomicInteger(0);
-
-  protected InputAttemptIdentifier inputAttemptIdentifier;
-  protected final long size;
-  protected final Type type;
-  protected final FetchedInputCallback callback;
-  protected final int id;
-  protected State state;
-
-  public FetchedInput(Type type, long size,
-      InputAttemptIdentifier inputAttemptIdentifier,
-      FetchedInputCallback callbackHandler) {
-    this.type = type;
-    this.size = size;
-    this.inputAttemptIdentifier = inputAttemptIdentifier;
-    this.callback = callbackHandler;
-    this.id = ID_GEN.getAndIncrement();
-    this.state = State.PENDING;
-  }
-
-  public Type getType() {
-    return this.type;
-  }
-
-  public long getSize() {
-    return this.size;
-  }
-
-  public InputAttemptIdentifier getInputAttemptIdentifier() {
-    return this.inputAttemptIdentifier;
-  }
-
-  /**
-   * Inform the Allocator about a committed resource.
-   * This should be called by commit
-   */
-  public void notifyFetchComplete() {
-    this.callback.fetchComplete(this);
-  }
-  
-  /**
-   * Inform the Allocator about a failed resource.
-   * This should be called by abort
-   */
-  public void notifyFetchFailure() {
-    this.callback.fetchFailed(this);
-  }
-  
-  /**
-   * Inform the Allocator about a completed resource being released.
-   * This should be called by free
-   */
-  public void notifyFreedResource() {
-    this.callback.freeResources(this);
-  }
-  
-  /**
-   * Returns the output stream to be used to write fetched data. Users are
-   * expected to close the OutputStream when they're done
-   */
-  public abstract OutputStream getOutputStream() throws IOException;
-
-  /**
-   * Return an input stream to be used to read the previously fetched data.
-   * Users are expected to close the InputStream when they're done
-   */
-  public abstract InputStream getInputStream() throws IOException;
-
-  /**
-   * Commit the output. Should be idempotent
-   */
-  public abstract void commit() throws IOException;
-
-  /**
-   * Abort the output. Should be idempotent
-   */
-  public abstract void abort() throws IOException;
-
-  /**
-   * Called when this input has been consumed, so that resources can be
-   * reclaimed.
-   */
-  public abstract void free();
-  
-  @Override
-  public int hashCode() {
-    return id;
-  }
-
-  @Override
-  public boolean equals(Object obj) {
-    if (this == obj)
-      return true;
-    if (obj == null)
-      return false;
-    if (getClass() != obj.getClass())
-      return false;
-    FetchedInput other = (FetchedInput) obj;
-    if (id != other.id)
-      return false;
-    return true;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/engine/shuffle/common/FetchedInputAllocator.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/shuffle/common/FetchedInputAllocator.java b/tez-engine/src/main/java/org/apache/tez/engine/shuffle/common/FetchedInputAllocator.java
deleted file mode 100644
index 7e573f0..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/shuffle/common/FetchedInputAllocator.java
+++ /dev/null
@@ -1,31 +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.engine.shuffle.common;
-
-import java.io.IOException;
-
-import org.apache.tez.engine.common.InputAttemptIdentifier;
-
-public interface FetchedInputAllocator {
-
-  public FetchedInput allocate(long size,
-      InputAttemptIdentifier inputAttemptIdentifier) throws IOException;
-  
-  
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/engine/shuffle/common/FetchedInputCallback.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/shuffle/common/FetchedInputCallback.java b/tez-engine/src/main/java/org/apache/tez/engine/shuffle/common/FetchedInputCallback.java
deleted file mode 100644
index 2d2d73b..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/shuffle/common/FetchedInputCallback.java
+++ /dev/null
@@ -1,29 +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.engine.shuffle.common;
-
-public interface FetchedInputCallback {
-  
-  public void fetchComplete(FetchedInput fetchedInput);
-  
-  public void fetchFailed(FetchedInput fetchedInput);
-  
-  public void freeResources(FetchedInput fetchedInput);
-  
-}


[35/50] [abbrv] Rename tez-engine-api to tez-runtime-api and tez-engine is split into 2: - tez-engine-library for user-visible Input/Output/Processor implementations - tez-engine-internals for framework internals

Posted by ss...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezHeartbeatResponse.java
----------------------------------------------------------------------
diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezHeartbeatResponse.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezHeartbeatResponse.java
new file mode 100644
index 0000000..10699ac
--- /dev/null
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezHeartbeatResponse.java
@@ -0,0 +1,105 @@
+/**
+ * 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.api.impl;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.hadoop.io.Writable;
+
+public class TezHeartbeatResponse implements Writable {
+
+  private long lastRequestId;
+  private boolean shouldDie = false;
+  private List<TezEvent> events;
+
+  public TezHeartbeatResponse() {
+  }
+
+  public TezHeartbeatResponse(List<TezEvent> events) {
+    this.events = Collections.unmodifiableList(events);
+  }
+
+  public List<TezEvent> getEvents() {
+    return events;
+  }
+
+  public boolean shouldDie() {
+    return shouldDie;
+  }
+
+  public long getLastRequestId() {
+    return lastRequestId;
+  }
+
+  public void setEvents(List<TezEvent> events) {
+    this.events = Collections.unmodifiableList(events);
+  }
+
+  public void setLastRequestId(long lastRequestId ) {
+    this.lastRequestId = lastRequestId;
+  }
+
+  public void setShouldDie() {
+    this.shouldDie = true;
+  }
+
+  @Override
+  public void write(DataOutput out) throws IOException {
+    out.writeLong(lastRequestId);
+    out.writeBoolean(shouldDie);
+    if(events != null) {
+      out.writeBoolean(true);
+      out.writeInt(events.size());
+      for (TezEvent e : events) {
+        e.write(out);
+      }
+    } else {
+      out.writeBoolean(false);
+    }
+  }
+
+  @Override
+  public void readFields(DataInput in) throws IOException {
+    lastRequestId = in.readLong();
+    shouldDie = in.readBoolean();
+    if(in.readBoolean()) {
+      int eventCount = in.readInt();
+      events = new ArrayList<TezEvent>(eventCount);
+      for (int i = 0; i < eventCount; ++i) {
+        TezEvent e = new TezEvent();
+        e.readFields(in);
+        events.add(e);
+      }
+    }
+  }
+
+  @Override
+  public String toString() {
+    return "{ "
+        + " lastRequestId=" + lastRequestId
+        + ", shouldDie=" + shouldDie
+        + ", eventCount=" + (events != null ? events.size() : 0)
+        + " }";
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezInputContextImpl.java
----------------------------------------------------------------------
diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezInputContextImpl.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezInputContextImpl.java
new file mode 100644
index 0000000..9169895
--- /dev/null
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezInputContextImpl.java
@@ -0,0 +1,84 @@
+/**
+ * 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.api.impl;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.tez.common.counters.TezCounters;
+import org.apache.tez.dag.records.TezTaskAttemptID;
+import org.apache.tez.runtime.RuntimeTask;
+import org.apache.tez.runtime.api.Event;
+import org.apache.tez.runtime.api.TezInputContext;
+import org.apache.tez.runtime.api.impl.EventMetaData.EventProducerConsumerType;
+
+public class TezInputContextImpl extends TezTaskContextImpl
+    implements TezInputContext {
+
+  private final byte[] userPayload;
+  private final String sourceVertexName;
+  private final EventMetaData sourceInfo;
+
+  @Private
+  public TezInputContextImpl(Configuration conf, int appAttemptNumber,
+      TezUmbilical tezUmbilical, String taskVertexName,
+      String sourceVertexName, TezTaskAttemptID taskAttemptID,
+      TezCounters counters, byte[] userPayload,
+      RuntimeTask runtimeTask, Map<String, ByteBuffer> serviceConsumerMetadata) {
+    super(conf, appAttemptNumber, taskVertexName, taskAttemptID,
+        counters, runtimeTask, tezUmbilical, serviceConsumerMetadata);
+    this.userPayload = userPayload;
+    this.sourceVertexName = sourceVertexName;
+    this.sourceInfo = new EventMetaData(
+        EventProducerConsumerType.INPUT, taskVertexName, sourceVertexName,
+        taskAttemptID);
+    this.uniqueIdentifier = String.format("%s_%s_%06d_%02d_%s", taskAttemptID
+        .getTaskID().getVertexID().getDAGId().toString(), taskVertexName,
+        getTaskIndex(), getTaskAttemptNumber(), sourceVertexName);
+  }
+
+  @Override
+  public void sendEvents(List<Event> events) {
+    List<TezEvent> tezEvents = new ArrayList<TezEvent>(events.size());
+    for (Event e : events) {
+      TezEvent tEvt = new TezEvent(e, sourceInfo);
+      tezEvents.add(tEvt);
+    }
+    tezUmbilical.addEvents(tezEvents);
+  }
+
+  @Override
+  public byte[] getUserPayload() {
+    return userPayload;
+  }
+
+  @Override
+  public String getSourceVertexName() {
+    return sourceVertexName;
+  }
+
+  @Override
+  public void fatalError(Throwable exception, String message) {
+    super.signalFatalError(exception, message, sourceInfo);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezOutputContextImpl.java
----------------------------------------------------------------------
diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezOutputContextImpl.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezOutputContextImpl.java
new file mode 100644
index 0000000..fd4c3a3
--- /dev/null
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezOutputContextImpl.java
@@ -0,0 +1,85 @@
+/**
+ * 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.api.impl;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.tez.common.counters.TezCounters;
+import org.apache.tez.dag.records.TezTaskAttemptID;
+import org.apache.tez.runtime.RuntimeTask;
+import org.apache.tez.runtime.api.Event;
+import org.apache.tez.runtime.api.TezOutputContext;
+import org.apache.tez.runtime.api.impl.EventMetaData.EventProducerConsumerType;
+
+public class TezOutputContextImpl extends TezTaskContextImpl
+    implements TezOutputContext {
+
+  private final byte[] userPayload;
+  private final String destinationVertexName;
+  private final EventMetaData sourceInfo;
+
+  @Private
+  public TezOutputContextImpl(Configuration conf, int appAttemptNumber,
+      TezUmbilical tezUmbilical, String taskVertexName,
+      String destinationVertexName,
+      TezTaskAttemptID taskAttemptID, TezCounters counters,
+      byte[] userPayload, RuntimeTask runtimeTask,
+      Map<String, ByteBuffer> serviceConsumerMetadata) {
+    super(conf, appAttemptNumber, taskVertexName, taskAttemptID,
+        counters, runtimeTask, tezUmbilical, serviceConsumerMetadata);
+    this.userPayload = userPayload;
+    this.destinationVertexName = destinationVertexName;
+    this.sourceInfo = new EventMetaData(EventProducerConsumerType.OUTPUT,
+        taskVertexName, destinationVertexName, taskAttemptID);
+    this.uniqueIdentifier = String.format("%s_%s_%06d_%02d_%s", taskAttemptID
+        .getTaskID().getVertexID().getDAGId().toString(), taskVertexName,
+        getTaskIndex(), getTaskAttemptNumber(), destinationVertexName);
+  }
+
+  @Override
+  public void sendEvents(List<Event> events) {
+    List<TezEvent> tezEvents = new ArrayList<TezEvent>(events.size());
+    for (Event e : events) {
+      TezEvent tEvt = new TezEvent(e, sourceInfo);
+      tezEvents.add(tEvt);
+    }
+    tezUmbilical.addEvents(tezEvents);
+  }
+
+  @Override
+  public byte[] getUserPayload() {
+    return userPayload;
+  }
+
+  @Override
+  public String getDestinationVertexName() {
+    return destinationVertexName;
+  }
+
+  @Override
+  public void fatalError(Throwable exception, String message) {
+    super.signalFatalError(exception, message, sourceInfo);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezProcessorContextImpl.java
----------------------------------------------------------------------
diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezProcessorContextImpl.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezProcessorContextImpl.java
new file mode 100644
index 0000000..e73baf4
--- /dev/null
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezProcessorContextImpl.java
@@ -0,0 +1,86 @@
+/**
+ * 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.api.impl;
+
+import java.nio.ByteBuffer;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.tez.common.counters.TezCounters;
+import org.apache.tez.dag.records.TezTaskAttemptID;
+import org.apache.tez.runtime.RuntimeTask;
+import org.apache.tez.runtime.api.Event;
+import org.apache.tez.runtime.api.TezProcessorContext;
+import org.apache.tez.runtime.api.impl.EventMetaData.EventProducerConsumerType;
+
+public class TezProcessorContextImpl extends TezTaskContextImpl
+  implements TezProcessorContext {
+
+  private final byte[] userPayload;
+  private final EventMetaData sourceInfo;
+
+  public TezProcessorContextImpl(Configuration conf, int appAttemptNumber,
+      TezUmbilical tezUmbilical, String vertexName,
+      TezTaskAttemptID taskAttemptID, TezCounters counters,
+      byte[] userPayload, RuntimeTask runtimeTask,
+      Map<String, ByteBuffer> serviceConsumerMetadata) {
+    super(conf, appAttemptNumber, vertexName, taskAttemptID,
+        counters, runtimeTask, tezUmbilical, serviceConsumerMetadata);
+    this.userPayload = userPayload;
+    this.sourceInfo = new EventMetaData(EventProducerConsumerType.PROCESSOR,
+        taskVertexName, "", taskAttemptID);
+    this.uniqueIdentifier = String.format("%s_%s_%06d_%02d", taskAttemptID
+        .getTaskID().getVertexID().getDAGId().toString(), taskVertexName,
+        getTaskIndex(), getTaskAttemptNumber());
+  }
+
+  @Override
+  public void sendEvents(List<Event> events) {
+    List<TezEvent> tezEvents = new ArrayList<TezEvent>(events.size());
+    for (Event e : events) {
+      TezEvent tEvt = new TezEvent(e, sourceInfo);
+      tezEvents.add(tEvt);
+    }
+    tezUmbilical.addEvents(tezEvents);
+  }
+
+  @Override
+  public byte[] getUserPayload() {
+    return userPayload;
+  }
+
+  @Override
+  public void setProgress(float progress) {
+    runtimeTask.setProgress(progress);
+  }
+
+  @Override
+  public void fatalError(Throwable exception, String message) {
+    super.signalFatalError(exception, message, sourceInfo);
+  }
+
+  @Override
+  public boolean canCommit() throws IOException {
+    return tezUmbilical.canCommit(this.taskAttemptID);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezTaskContextImpl.java
----------------------------------------------------------------------
diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezTaskContextImpl.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezTaskContextImpl.java
new file mode 100644
index 0000000..ee9e96d
--- /dev/null
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezTaskContextImpl.java
@@ -0,0 +1,145 @@
+/**
+ * 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.api.impl;
+
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.Map;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.util.AuxiliaryServiceHelper;
+import org.apache.tez.common.TezJobConfig;
+import org.apache.tez.common.counters.TezCounters;
+import org.apache.tez.dag.records.TezTaskAttemptID;
+import org.apache.tez.runtime.RuntimeTask;
+import org.apache.tez.runtime.api.TezTaskContext;
+
+public abstract class TezTaskContextImpl implements TezTaskContext {
+
+  private final Configuration conf;
+  protected final String taskVertexName;
+  protected final TezTaskAttemptID taskAttemptID;
+  private final TezCounters counters;
+  private String[] workDirs;
+  protected String uniqueIdentifier;
+  protected final RuntimeTask runtimeTask;
+  protected final TezUmbilical tezUmbilical;
+  private final Map<String, ByteBuffer> serviceConsumerMetadata;
+  private final int appAttemptNumber;
+
+  @Private
+  public TezTaskContextImpl(Configuration conf, int appAttemptNumber,
+      String taskVertexName, TezTaskAttemptID taskAttemptID,
+      TezCounters counters, RuntimeTask runtimeTask,
+      TezUmbilical tezUmbilical, Map<String, ByteBuffer> serviceConsumerMetadata) {
+    this.conf = conf;
+    this.taskVertexName = taskVertexName;
+    this.taskAttemptID = taskAttemptID;
+    this.counters = counters;
+    // TODO Maybe change this to be task id specific at some point. For now
+    // Shuffle code relies on this being a path specified by YARN
+    this.workDirs = this.conf.getStrings(TezJobConfig.LOCAL_DIRS);
+    this.runtimeTask = runtimeTask;
+    this.tezUmbilical = tezUmbilical;
+    this.serviceConsumerMetadata = serviceConsumerMetadata;
+    // TODO NEWTEZ at some point dag attempt should not map to app attempt
+    this.appAttemptNumber = appAttemptNumber;
+  }
+
+  @Override
+  public ApplicationId getApplicationId() {
+    return taskAttemptID.getTaskID().getVertexID().getDAGId()
+        .getApplicationId();
+  }
+
+  @Override
+  public int getTaskIndex() {
+    return taskAttemptID.getTaskID().getId();
+  }
+
+  @Override
+  public int getDAGAttemptNumber() {
+    return appAttemptNumber;
+  }
+
+  @Override
+  public int getTaskAttemptNumber() {
+    return taskAttemptID.getId();
+  }
+
+  @Override
+  public String getDAGName() {
+    // TODO NEWTEZ Change to some form of the DAG name, for now using dagId as
+    // the unique identifier.
+    return taskAttemptID.getTaskID().getVertexID().getDAGId().toString();
+  }
+
+  @Override
+  public String getTaskVertexName() {
+    return taskVertexName;
+  }
+
+
+  @Override
+  public TezCounters getCounters() {
+    return counters;
+  }
+
+  @Override
+  public String[] getWorkDirs() {
+    return Arrays.copyOf(workDirs, workDirs.length);
+  }
+
+  @Override
+  public String getUniqueIdentifier() {
+    return uniqueIdentifier;
+  }
+
+  @Override
+  public ByteBuffer getServiceConsumerMetaData(String serviceName) {
+    return (ByteBuffer) serviceConsumerMetadata.get(serviceName)
+        .asReadOnlyBuffer().rewind();
+  }
+
+  @Override
+  public ByteBuffer getServiceProviderMetaData(String serviceName) {
+    return AuxiliaryServiceHelper.getServiceDataFromEnv(
+        serviceName, System.getenv());
+  }
+
+  protected void signalFatalError(Throwable t, String message,
+      EventMetaData sourceInfo) {
+    runtimeTask.setFatalError(t, message);
+    String diagnostics;
+    if (t != null && message != null) {
+      diagnostics = "exceptionThrown=" + StringUtils.stringifyException(t)
+          + ", errorMessage=" + message;
+    } else if (t == null && message == null) {
+      diagnostics = "Unknown error";
+    } else {
+      diagnostics = t != null ?
+          "exceptionThrown=" + StringUtils.stringifyException(t)
+          : " errorMessage=" + message;
+    }
+    tezUmbilical.signalFatalError(taskAttemptID, diagnostics, sourceInfo);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezUmbilical.java
----------------------------------------------------------------------
diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezUmbilical.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezUmbilical.java
new file mode 100644
index 0000000..addccda
--- /dev/null
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezUmbilical.java
@@ -0,0 +1,36 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.runtime.api.impl;
+
+import java.io.IOException;
+import java.util.Collection;
+
+import org.apache.tez.dag.records.TezTaskAttemptID;
+
+public interface TezUmbilical {
+
+  public void addEvents(Collection<TezEvent> events);
+
+  public void signalFatalError(TezTaskAttemptID taskAttemptID,
+      String diagnostics,
+      EventMetaData sourceInfo);
+
+  public boolean canCommit(TezTaskAttemptID taskAttemptID) throws IOException;
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-internals/src/main/java/org/apache/tez/runtime/common/objectregistry/ObjectRegistryImpl.java
----------------------------------------------------------------------
diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/common/objectregistry/ObjectRegistryImpl.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/common/objectregistry/ObjectRegistryImpl.java
new file mode 100644
index 0000000..a47526b
--- /dev/null
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/common/objectregistry/ObjectRegistryImpl.java
@@ -0,0 +1,68 @@
+/**
+ * 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.common.objectregistry;
+
+import java.util.AbstractMap;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Map.Entry;
+
+import org.apache.tez.runtime.common.objectregistry.ObjectLifeCycle;
+import org.apache.tez.runtime.common.objectregistry.ObjectRegistry;
+
+import com.google.inject.Singleton;
+
+@Singleton
+public class ObjectRegistryImpl implements ObjectRegistry {
+
+  private Map<String, Map.Entry<Object, ObjectLifeCycle>> objectCache =
+      new HashMap<String, Map.Entry<Object, ObjectLifeCycle>>();
+
+  @Override
+  public synchronized Object add(ObjectLifeCycle lifeCycle,
+      String key, Object value) {
+    Map.Entry<Object, ObjectLifeCycle> oldEntry =
+        objectCache.put(key,
+            new AbstractMap.SimpleImmutableEntry<Object, ObjectLifeCycle>(
+                value, lifeCycle));
+    return oldEntry != null ? oldEntry.getKey() : null;
+  }
+
+  @Override
+  public synchronized Object get(String key) {
+    Map.Entry<Object, ObjectLifeCycle> entry =
+        objectCache.get(key);
+    return entry != null ? entry.getKey() : null;
+  }
+
+  @Override
+  public synchronized boolean delete(String key) {
+    return (null != objectCache.remove(key));
+  }
+
+  public synchronized void clearCache(ObjectLifeCycle lifeCycle) {
+    for (Entry<String, Entry<Object, ObjectLifeCycle>> entry :
+      objectCache.entrySet()) {
+      if (entry.getValue().getValue().equals(lifeCycle)) {
+        objectCache.remove(entry.getKey());
+      }
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-internals/src/main/java/org/apache/tez/runtime/common/objectregistry/ObjectRegistryModule.java
----------------------------------------------------------------------
diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/common/objectregistry/ObjectRegistryModule.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/common/objectregistry/ObjectRegistryModule.java
new file mode 100644
index 0000000..97ccf7c
--- /dev/null
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/common/objectregistry/ObjectRegistryModule.java
@@ -0,0 +1,46 @@
+/**
+ * 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.common.objectregistry;
+
+import org.apache.tez.runtime.common.objectregistry.ObjectRegistry;
+import org.apache.tez.runtime.common.objectregistry.ObjectRegistryFactory;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.inject.AbstractModule;
+
+public class ObjectRegistryModule extends AbstractModule {
+
+  private final ObjectRegistry objectRegistry;
+
+  public ObjectRegistryModule(ObjectRegistry objectRegistry) {
+    this.objectRegistry = objectRegistry;
+  }
+
+  @VisibleForTesting
+  public ObjectRegistryModule() {
+    objectRegistry = new ObjectRegistryImpl();
+  }
+
+  @Override
+  protected void configure() {
+    bind(ObjectRegistry.class).toInstance(this.objectRegistry);
+    requestStaticInjection(ObjectRegistryFactory.class);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-internals/src/main/proto/Events.proto
----------------------------------------------------------------------
diff --git a/tez-runtime-internals/src/main/proto/Events.proto b/tez-runtime-internals/src/main/proto/Events.proto
new file mode 100644
index 0000000..558a2b3
--- /dev/null
+++ b/tez-runtime-internals/src/main/proto/Events.proto
@@ -0,0 +1,28 @@
+/**
+ * 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.
+ */
+
+option java_package = "org.apache.tez.runtime.internals.api.events";
+option java_outer_classname = "SystemEventProtos";
+option java_generate_equals_and_hash = true;
+
+message TaskAttemptFailedEventProto {
+  optional string diagnostics = 1;
+}
+
+message TaskAttemptCompletedEventProto {
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-internals/src/test/java/org/apache/tez/runtime/common/objectregistry/TestObjectRegistry.java
----------------------------------------------------------------------
diff --git a/tez-runtime-internals/src/test/java/org/apache/tez/runtime/common/objectregistry/TestObjectRegistry.java b/tez-runtime-internals/src/test/java/org/apache/tez/runtime/common/objectregistry/TestObjectRegistry.java
new file mode 100644
index 0000000..35192e7
--- /dev/null
+++ b/tez-runtime-internals/src/test/java/org/apache/tez/runtime/common/objectregistry/TestObjectRegistry.java
@@ -0,0 +1,60 @@
+/**
+ * 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.common.objectregistry;
+
+import org.apache.tez.runtime.common.objectregistry.ObjectLifeCycle;
+import org.apache.tez.runtime.common.objectregistry.ObjectRegistry;
+import org.apache.tez.runtime.common.objectregistry.ObjectRegistryFactory;
+import org.apache.tez.runtime.common.objectregistry.ObjectRegistryModule;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import com.google.inject.Guice;
+import com.google.inject.Injector;
+
+public class TestObjectRegistry {
+
+  @SuppressWarnings("unused")
+  @Before
+  public void setup() {
+    Injector injector = Guice.createInjector(new ObjectRegistryModule());
+  }
+
+  @Test
+  public void testBasicCRUD() {
+    ObjectRegistry objectRegistry =
+        ObjectRegistryFactory.getObjectRegistry();
+    Assert.assertNotNull(objectRegistry);
+
+    Assert.assertNull(objectRegistry.get("foo"));
+    Assert.assertFalse(objectRegistry.delete("foo"));
+    Integer one = new Integer(1);
+    Integer two_1 = new Integer(2);
+    Integer two_2 = new Integer(3);
+    Assert.assertNull(objectRegistry.add(ObjectLifeCycle.DAG, "one", one));
+    Assert.assertEquals(one, objectRegistry.get("one"));
+    Assert.assertNull(objectRegistry.add(ObjectLifeCycle.DAG, "two", two_1));
+    Assert.assertNotNull(objectRegistry.add(ObjectLifeCycle.SESSION, "two", two_2));
+    Assert.assertNotEquals(two_1, objectRegistry.get("two"));
+    Assert.assertEquals(two_2, objectRegistry.get("two"));
+    Assert.assertTrue(objectRegistry.delete("one"));
+    Assert.assertFalse(objectRegistry.delete("one"));
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/pom.xml
----------------------------------------------------------------------
diff --git a/tez-runtime-library/pom.xml b/tez-runtime-library/pom.xml
new file mode 100644
index 0000000..dcdabe1
--- /dev/null
+++ b/tez-runtime-library/pom.xml
@@ -0,0 +1,79 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  Licensed 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. See accompanying LICENSE file.
+-->
+
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <groupId>org.apache.tez</groupId>
+    <artifactId>tez</artifactId>
+    <version>0.2.0-SNAPSHOT</version>
+  </parent>
+  <artifactId>tez-runtime-library</artifactId>
+
+  <dependencies>
+    <dependency>
+      <groupId>org.apache.tez</groupId>
+      <artifactId>tez-common</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.tez</groupId>
+      <artifactId>tez-api</artifactId>
+    </dependency>
+    <dependency>
+     <groupId>com.google.protobuf</groupId>
+     <artifactId>protobuf-java</artifactId>
+    </dependency>
+  </dependencies>
+
+  <build>
+    <plugins>
+      <plugin>
+        <groupId>org.apache.rat</groupId>
+        <artifactId>apache-rat-plugin</artifactId>
+        <configuration>
+        </configuration>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.hadoop</groupId>
+        <artifactId>hadoop-maven-plugins</artifactId>
+        <executions>
+          <execution>
+            <id>compile-protoc</id>
+            <phase>generate-sources</phase>
+            <goals>
+              <goal>protoc</goal>
+            </goals>
+            <configuration>
+              <protocVersion>${protobuf.version}</protocVersion>
+              <protocCommand>${protoc.path}</protocCommand>
+              <imports>
+                <param>${basedir}/src/main/proto</param>
+              </imports>
+              <source>
+                <directory>${basedir}/src/main/proto</directory>
+                <includes>
+                  <include>ShufflePayloads.proto</include>
+                </includes>
+              </source>
+              <output>${project.build.directory}/generated-sources/java</output>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
+    </plugins>
+  </build>
+</project>

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/hadoop/io/BufferUtils.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/hadoop/io/BufferUtils.java b/tez-runtime-library/src/main/java/org/apache/hadoop/io/BufferUtils.java
new file mode 100644
index 0000000..16f7a8f
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/hadoop/io/BufferUtils.java
@@ -0,0 +1,81 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.io;
+
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+@InterfaceAudience.LimitedPrivate({"HDFS", "MapReduce"})
+@InterfaceStability.Unstable
+public class BufferUtils {
+  public static int compare(DataInputBuffer buf1, DataInputBuffer buf2) {
+    byte[] b1 = buf1.getData();
+    byte[] b2 = buf2.getData();
+    int s1 = buf1.getPosition();
+    int s2 = buf2.getPosition();
+    int l1 = buf1.getLength();
+    int l2 = buf2.getLength();
+    return FastByteComparisons.compareTo(b1, s1, l1, b2, s2, l2);
+  }
+
+  public static int compare(DataOutputBuffer buf1, DataOutputBuffer buf2) {
+    byte[] b1 = buf1.getData();
+    byte[] b2 = buf2.getData();
+    int s1 = 0;
+    int s2 = 0;
+    int l1 = buf1.getLength();
+    int l2 = buf2.getLength();
+    return FastByteComparisons.compareTo(b1, s1, l1, b2, s2, l2);
+  }
+
+  public static int compare(DataInputBuffer buf1, DataOutputBuffer buf2) {
+    byte[] b1 = buf1.getData();
+    byte[] b2 = buf2.getData();
+    int s1 = buf1.getPosition();    
+    int s2 = 0;
+    int l1 = buf1.getLength();
+    int l2 = buf2.getLength();
+    return FastByteComparisons.compareTo(b1, s1, l1, b2, s2, l2);
+  }
+
+  public static int compare(DataOutputBuffer buf1, DataInputBuffer buf2) {
+    return compare(buf2, buf1);
+  }
+
+  public static void copy(DataInputBuffer src, DataOutputBuffer dst) 
+                              throws IOException {
+    byte[] b1 = src.getData();
+    int s1 = src.getPosition();    
+    int l1 = src.getLength();
+    dst.reset();
+    dst.write(b1, s1, l1 - s1);
+  }
+
+  public static void copy(DataOutputBuffer src, DataOutputBuffer dst) 
+                              throws IOException {
+    byte[] b1 = src.getData();
+    int s1 = 0;
+    int l1 = src.getLength();
+    dst.reset();
+    dst.write(b1, s1, l1);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/hadoop/io/HashComparator.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/hadoop/io/HashComparator.java b/tez-runtime-library/src/main/java/org/apache/hadoop/io/HashComparator.java
new file mode 100644
index 0000000..a372e01
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/hadoop/io/HashComparator.java
@@ -0,0 +1,24 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.io;
+
+public interface HashComparator<KEY> {
+
+  int getHashCode(KEY key);
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/api/KVReader.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/api/KVReader.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/api/KVReader.java
new file mode 100644
index 0000000..9c6b380
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/api/KVReader.java
@@ -0,0 +1,81 @@
+/**
+ * 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.library.api;
+
+import java.io.IOException;
+
+import org.apache.tez.runtime.api.Reader;
+
+/**
+ * A key/value(s) pair based {@link Reader}.
+ * 
+ * Example usage
+ * <code>
+ * while (kvReader.next()) {
+ *   KVRecord kvRecord = getCurrentKV();
+ *   Object key =  kvRecord.getKey();
+ *   Iterable values = kvRecord.getValues();
+ * </code>
+ *
+ */
+public interface KVReader extends Reader {
+
+  /**
+   * Moves to the next key/values(s) pair
+   * 
+   * @return true if another key/value(s) pair exists, false if there are no more.
+   * @throws IOException
+   *           if an error occurs
+   */
+  public boolean next() throws IOException;
+
+  /**
+   * Return the current key/value(s) pair. Use moveToNext() to advance.
+   * @return
+   * @throws IOException
+   */
+  public KVRecord getCurrentKV() throws IOException;
+  
+  // TODO NEWTEZ Move this to getCurrentKey and getCurrentValue independently. Otherwise usage pattern seems to be getCurrentKV.getKey, getCurrentKV.getValue
+  
+  // TODO NEWTEZ KVRecord which does not need to return a list!
+  // TODO NEWTEZ Parameterize this
+  /**
+   * Represents a key and an associated set of values
+   *
+   */
+  public static class KVRecord {
+
+    private Object key;
+    private Iterable<Object> values;
+
+    public KVRecord(Object key, Iterable<Object> values) {
+      this.key = key;
+      this.values = values;
+    }
+
+    public Object getKey() {
+      return this.key;
+    }
+
+    public Iterable<Object> getValues() {
+      return this.values;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/api/KVWriter.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/api/KVWriter.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/api/KVWriter.java
new file mode 100644
index 0000000..ff952ed
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/api/KVWriter.java
@@ -0,0 +1,40 @@
+/**
+ * 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.library.api;
+
+import java.io.IOException;
+
+import org.apache.tez.runtime.api.Writer;
+
+/**
+ * A key/value(s) pair based {@link Writer}
+ */
+public interface KVWriter extends Writer {
+  /**
+   * Writes a key/value pair.
+   * 
+   * @param key
+   *          the key to write
+   * @param value
+   *          the value to write
+   * @throws IOException
+   *           if an error occurs
+   */
+  public void write(Object key, Object value) throws IOException;
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/api/Partitioner.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/api/Partitioner.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/api/Partitioner.java
new file mode 100644
index 0000000..680c9b8
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/api/Partitioner.java
@@ -0,0 +1,52 @@
+/**
+ * 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.library.api;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.tez.common.TezJobConfig;
+
+/**
+ * {@link Partitioner} is used by the TEZ framework to partition output
+ * key/value pairs.
+ * 
+ * <b>Partitioner Initialization</b></p> The Partitioner class is picked up
+ * using the TEZ_RUNTIME_PARTITIONER_CLASS attribute in {@link TezJobConfig}
+ * 
+ * TODO NEWTEZ Change construction to first check for a Constructor with a bytep[] payload
+ * 
+ * Partitioners need to provide a single argument ({@link Configuration})
+ * constructor or a 0 argument constructor. If both exist, preference is given
+ * to the single argument constructor. This is primarily for MR support.
+ * 
+ * If using the configuration constructor, TEZ_RUNTIME_NUM_EXPECTED_PARTITIONS
+ * will be set in the configuration, to indicate the max number of expected
+ * partitions.
+ * 
+ */
+public interface Partitioner {
+  
+  /**
+   * Get partition for given key/value.
+   * @param key key
+   * @param value value
+   * @param numPartitions number of partitions
+   * @return
+   */
+  int getPartition(Object key, Object value, int numPartitions);
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/broadcast/input/BroadcastInputManager.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/broadcast/input/BroadcastInputManager.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/broadcast/input/BroadcastInputManager.java
new file mode 100644
index 0000000..cda52da
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/broadcast/input/BroadcastInputManager.java
@@ -0,0 +1,138 @@
+/**
+ * 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.library.broadcast.input;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.LocalDirAllocator;
+import org.apache.tez.common.TezJobConfig;
+import org.apache.tez.dag.api.TezUncheckedException;
+import org.apache.tez.runtime.api.TezInputContext;
+import org.apache.tez.runtime.library.common.Constants;
+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.FetchedInputAllocator;
+import org.apache.tez.runtime.library.shuffle.common.FetchedInputCallback;
+import org.apache.tez.runtime.library.shuffle.common.MemoryFetchedInput;
+
+public class BroadcastInputManager implements FetchedInputAllocator,
+    FetchedInputCallback {
+
+  private final Configuration conf;
+
+  private final TezTaskOutputFiles fileNameAllocator;
+  private final LocalDirAllocator localDirAllocator;
+
+  // Configuration parameters
+  private final long memoryLimit;
+  private final long maxSingleShuffleLimit;
+
+  private long usedMemory = 0;
+
+  public BroadcastInputManager(TezInputContext inputContext, Configuration conf) {
+    this.conf = conf;
+
+    this.fileNameAllocator = new TezTaskOutputFiles(conf,
+        inputContext.getUniqueIdentifier());
+    this.localDirAllocator = new LocalDirAllocator(TezJobConfig.LOCAL_DIRS);
+
+    // Setup configuration
+    final float maxInMemCopyUse = conf.getFloat(
+        TezJobConfig.TEZ_RUNTIME_SHUFFLE_INPUT_BUFFER_PERCENT,
+        TezJobConfig.DEFAULT_TEZ_RUNTIME_SHUFFLE_INPUT_BUFFER_PERCENT);
+    if (maxInMemCopyUse > 1.0 || maxInMemCopyUse < 0.0) {
+      throw new IllegalArgumentException("Invalid value for "
+          + TezJobConfig.TEZ_RUNTIME_SHUFFLE_INPUT_BUFFER_PERCENT + ": "
+          + maxInMemCopyUse);
+    }
+
+    // Allow unit tests to fix Runtime memory
+    this.memoryLimit = (long) (conf.getLong(Constants.TEZ_RUNTIME_TASK_MEMORY,
+        Math.min(Runtime.getRuntime().maxMemory(), Integer.MAX_VALUE)) * maxInMemCopyUse);
+
+    final float singleShuffleMemoryLimitPercent = conf.getFloat(
+        TezJobConfig.TEZ_RUNTIME_SHUFFLE_MEMORY_LIMIT_PERCENT,
+        TezJobConfig.DEFAULT_TEZ_RUNTIME_SHUFFLE_MEMORY_LIMIT_PERCENT);
+    if (singleShuffleMemoryLimitPercent <= 0.0f
+        || singleShuffleMemoryLimitPercent > 1.0f) {
+      throw new IllegalArgumentException("Invalid value for "
+          + TezJobConfig.TEZ_RUNTIME_SHUFFLE_MEMORY_LIMIT_PERCENT + ": "
+          + singleShuffleMemoryLimitPercent);
+    }
+
+    this.maxSingleShuffleLimit = (long) (memoryLimit * singleShuffleMemoryLimitPercent);
+  }
+
+  @Override
+  public synchronized FetchedInput allocate(long size,
+      InputAttemptIdentifier inputAttemptIdentifier) throws IOException {
+    if (size > maxSingleShuffleLimit
+        || this.usedMemory + size > this.memoryLimit) {
+      return new DiskFetchedInput(size, inputAttemptIdentifier, this, conf,
+          localDirAllocator, fileNameAllocator);
+    } else {
+      this.usedMemory += size;
+      return new MemoryFetchedInput(size, inputAttemptIdentifier, this);
+    }
+  }
+
+  @Override
+  public void fetchComplete(FetchedInput fetchedInput) {
+    switch (fetchedInput.getType()) {
+    // Not tracking anything here.
+    case DISK:
+    case MEMORY:
+      break;
+    default:
+      throw new TezUncheckedException("InputType: " + fetchedInput.getType()
+          + " not expected for Broadcast fetch");
+    }
+  }
+
+  @Override
+  public void fetchFailed(FetchedInput fetchedInput) {
+    cleanup(fetchedInput);
+  }
+
+  @Override
+  public void freeResources(FetchedInput fetchedInput) {
+    cleanup(fetchedInput);
+  }
+
+  private void cleanup(FetchedInput fetchedInput) {
+    switch (fetchedInput.getType()) {
+    case DISK:
+      break;
+    case MEMORY:
+      unreserve(fetchedInput.getSize());
+      break;
+    default:
+      throw new TezUncheckedException("InputType: " + fetchedInput.getType()
+          + " not expected for Broadcast fetch");
+    }
+  }
+
+  private synchronized void unreserve(long size) {
+    this.usedMemory -= size;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/broadcast/input/BroadcastKVReader.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/broadcast/input/BroadcastKVReader.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/broadcast/input/BroadcastKVReader.java
new file mode 100644
index 0000000..16e9645
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/broadcast/input/BroadcastKVReader.java
@@ -0,0 +1,225 @@
+/**
+ * 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.library.broadcast.input;
+
+import java.io.IOException;
+import java.util.Iterator;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.DataInputBuffer;
+import org.apache.hadoop.io.compress.CompressionCodec;
+import org.apache.hadoop.io.compress.DefaultCodec;
+import org.apache.hadoop.io.serializer.Deserializer;
+import org.apache.hadoop.io.serializer.SerializationFactory;
+import org.apache.hadoop.util.ReflectionUtils;
+import org.apache.tez.runtime.library.api.KVReader;
+import org.apache.tez.runtime.library.common.ConfigUtils;
+import org.apache.tez.runtime.library.common.shuffle.impl.InMemoryReader;
+import org.apache.tez.runtime.library.common.sort.impl.IFile;
+import org.apache.tez.runtime.library.shuffle.common.FetchedInput;
+import org.apache.tez.runtime.library.shuffle.common.MemoryFetchedInput;
+import org.apache.tez.runtime.library.shuffle.common.FetchedInput.Type;
+
+public class BroadcastKVReader<K, V> implements KVReader {
+
+  private static final Log LOG = LogFactory.getLog(BroadcastKVReader.class);
+  
+  private final BroadcastShuffleManager shuffleManager;
+  private final Configuration conf;
+  private final CompressionCodec codec;
+  
+  private final Class<K> keyClass;
+  private final Class<V> valClass;
+  private final Deserializer<K> keyDeserializer;
+  private final Deserializer<V> valDeserializer;
+  private final DataInputBuffer keyIn;
+  private final DataInputBuffer valIn;
+
+  private final SimpleValueIterator valueIterator;
+  private final SimpleIterable valueIterable;
+  
+  private K key;
+  private V value;
+  
+  private FetchedInput currentFetchedInput;
+  private IFile.Reader currentReader;
+  
+  
+  public BroadcastKVReader(BroadcastShuffleManager shuffleManager,
+      Configuration conf) {
+    this.shuffleManager = shuffleManager;
+    this.conf = conf;
+
+    if (ConfigUtils.isIntermediateInputCompressed(this.conf)) {
+      Class<? extends CompressionCodec> codecClass = ConfigUtils
+          .getIntermediateInputCompressorClass(conf, DefaultCodec.class);
+      codec = ReflectionUtils.newInstance(codecClass, conf);
+    } else {
+      codec = null;
+    }
+
+    this.keyClass = ConfigUtils.getIntermediateInputKeyClass(conf);
+    this.valClass = ConfigUtils.getIntermediateInputKeyClass(conf);
+
+    this.keyIn = new DataInputBuffer();
+    this.valIn = new DataInputBuffer();
+
+    SerializationFactory serializationFactory = new SerializationFactory(conf);
+
+    this.keyDeserializer = serializationFactory.getDeserializer(keyClass); 
+    this.valDeserializer = serializationFactory.getDeserializer(valClass);
+    
+    this.valueIterator = new SimpleValueIterator();
+    this.valueIterable = new SimpleIterable(this.valueIterator);
+  }
+
+  // TODO NEWTEZ Maybe add an interface to check whether next will block.
+  
+  /**
+   * Moves to the next key/values(s) pair
+   * 
+   * @return true if another key/value(s) pair exists, false if there are no
+   *         more.
+   * @throws IOException
+   *           if an error occurs
+   */
+  @Override  
+  public boolean next() throws IOException {
+    if (readNextFromCurrentReader()) {
+      return true;
+    } else {
+      boolean nextInputExists = moveToNextInput();
+      while (nextInputExists) {
+        if(readNextFromCurrentReader()) {
+          return true;
+        }
+        nextInputExists = moveToNextInput();
+      }
+      return false;
+    }
+  }
+
+  @SuppressWarnings("unchecked")
+  @Override
+  public KVRecord getCurrentKV() throws IOException {
+    this.valueIterator.setValue(value);
+    return new KVRecord((Object)key, (Iterable<Object>)this.valueIterable);
+  }
+
+  /**
+   * Tries reading the next key and value from the current reader.
+   * @return true if the current reader has more records
+   * @throws IOException
+   */
+  private boolean readNextFromCurrentReader() throws IOException {
+    // Initial reader.
+    if (this.currentReader == null) {
+      return false;
+    } else {
+      boolean hasMore = this.currentReader.nextRawKey(keyIn);
+      if (hasMore) {
+        this.currentReader.nextRawValue(valIn);
+        this.key = keyDeserializer.deserialize(this.key);
+        this.value = valDeserializer.deserialize(this.value);
+        return true;
+      }
+      return false;
+    }
+  }
+  
+  /**
+   * Moves to the next available input. This method may block if the input is not ready yet.
+   * Also takes care of closing the previous input.
+   * 
+   * @return true if the next input exists, false otherwise
+   * @throws IOException
+   * @throws InterruptedException
+   */
+  private boolean moveToNextInput() throws IOException {
+    if (currentReader != null) { // Close the current reader.
+      currentReader.close();
+      currentFetchedInput.free();
+    }
+    try {
+      currentFetchedInput = shuffleManager.getNextInput();
+    } catch (InterruptedException e) {
+      LOG.warn("Interrupted while waiting for next available input", e);
+      throw new IOException(e);
+    }
+    if (currentFetchedInput == null) {
+      return false; // No more inputs
+    } else {
+      currentReader = openIFileReader(currentFetchedInput);
+      return true;
+    }
+  }
+
+  public IFile.Reader openIFileReader(FetchedInput fetchedInput)
+      throws IOException {
+    if (fetchedInput.getType() == Type.MEMORY) {
+      MemoryFetchedInput mfi = (MemoryFetchedInput) fetchedInput;
+
+      return new InMemoryReader(null, mfi.getInputAttemptIdentifier(),
+          mfi.getBytes(), 0, (int) mfi.getSize());
+    } else {
+      return new IFile.Reader(conf, fetchedInput.getInputStream(),
+          fetchedInput.getSize(), codec, null);
+    }
+  }
+
+  
+  
+  // TODO NEWTEZ Move this into a common class. Also used in MRInput
+  private class SimpleValueIterator implements Iterator<V> {
+
+    private V value;
+
+    public void setValue(V value) {
+      this.value = value;
+    }
+
+    public boolean hasNext() {
+      return value != null;
+    }
+
+    public V next() {
+      V value = this.value;
+      this.value = null;
+      return value;
+    }
+
+    public void remove() {
+      throw new UnsupportedOperationException();
+    }
+  }
+
+  private class SimpleIterable implements Iterable<V> {
+    private final Iterator<V> iterator;
+    public SimpleIterable(Iterator<V> iterator) {
+      this.iterator = iterator;
+    }
+
+    @Override
+    public Iterator<V> iterator() {
+      return iterator;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/broadcast/input/BroadcastShuffleInputEventHandler.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/broadcast/input/BroadcastShuffleInputEventHandler.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/broadcast/input/BroadcastShuffleInputEventHandler.java
new file mode 100644
index 0000000..c64379a
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/broadcast/input/BroadcastShuffleInputEventHandler.java
@@ -0,0 +1,88 @@
+/**
+ * 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.library.broadcast.input;
+
+import java.util.List;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.tez.dag.api.TezUncheckedException;
+import org.apache.tez.runtime.api.Event;
+import org.apache.tez.runtime.api.TezInputContext;
+import org.apache.tez.runtime.api.events.DataMovementEvent;
+import org.apache.tez.runtime.api.events.InputFailedEvent;
+import org.apache.tez.runtime.library.common.InputAttemptIdentifier;
+import org.apache.tez.runtime.library.common.shuffle.impl.ShuffleInputEventHandler;
+import org.apache.tez.runtime.library.shuffle.impl.ShuffleUserPayloads.DataMovementEventPayloadProto;
+
+import com.google.common.base.Preconditions;
+import com.google.protobuf.InvalidProtocolBufferException;
+
+public class BroadcastShuffleInputEventHandler {
+
+  private static final Log LOG = LogFactory.getLog(ShuffleInputEventHandler.class);
+  
+  private final BroadcastShuffleManager shuffleManager;
+  
+  public BroadcastShuffleInputEventHandler(TezInputContext inputContext, BroadcastShuffleManager shuffleManager) {
+    this.shuffleManager = shuffleManager;
+  }
+
+  public void handleEvents(List<Event> events) {
+    for (Event event : events) {
+      handleEvent(event);
+    }
+  }
+  
+  private void handleEvent(Event event) {
+    if (event instanceof DataMovementEvent) {
+      processDataMovementEvent((DataMovementEvent)event);
+    } else if (event instanceof InputFailedEvent) {
+      processInputFailedEvent((InputFailedEvent)event);
+    } else {
+      throw new TezUncheckedException("Unexpected event type: " + event.getClass().getName());
+    }
+  }
+  
+  
+  private void processDataMovementEvent(DataMovementEvent dme) {
+    Preconditions.checkArgument(dme.getSourceIndex() == 0,
+        "Unexpected srcIndex: " + dme.getSourceIndex()
+            + " on DataMovementEvent. Can only be 0");
+    DataMovementEventPayloadProto shufflePayload;
+    try {
+      shufflePayload = DataMovementEventPayloadProto.parseFrom(dme.getUserPayload());
+    } catch (InvalidProtocolBufferException e) {
+      throw new TezUncheckedException("Unable to parse DataMovementEvent payload", e);
+    }
+    if (shufflePayload.getOutputGenerated()) {
+      InputAttemptIdentifier srcAttemptIdentifier = new InputAttemptIdentifier(dme.getTargetIndex(), dme.getVersion(), shufflePayload.getPathComponent());
+      shuffleManager.addKnownInput(shufflePayload.getHost(), shufflePayload.getPort(), srcAttemptIdentifier, 0);
+    } else {
+      shuffleManager.addCompletedInputWithNoData(new InputAttemptIdentifier(dme.getTargetIndex(), dme.getVersion()));
+    }
+  }
+  
+  private void processInputFailedEvent(InputFailedEvent ife) {
+    InputAttemptIdentifier srcAttemptIdentifier = new InputAttemptIdentifier(ife.getTargetIndex(), ife.getVersion());
+    shuffleManager.obsoleteKnownInput(srcAttemptIdentifier);
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/broadcast/input/BroadcastShuffleManager.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/broadcast/input/BroadcastShuffleManager.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/broadcast/input/BroadcastShuffleManager.java
new file mode 100644
index 0000000..2a5c22f
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/broadcast/input/BroadcastShuffleManager.java
@@ -0,0 +1,489 @@
+/**
+ * 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.library.broadcast.input;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.FutureTask;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+
+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.io.compress.CodecPool;
+import org.apache.hadoop.io.compress.CompressionCodec;
+import org.apache.hadoop.io.compress.Decompressor;
+import org.apache.hadoop.io.compress.DefaultCodec;
+import org.apache.hadoop.util.ReflectionUtils;
+import org.apache.tez.common.TezJobConfig;
+import org.apache.tez.runtime.api.Event;
+import org.apache.tez.runtime.api.TezInputContext;
+import org.apache.tez.runtime.api.events.InputReadErrorEvent;
+import org.apache.tez.runtime.library.common.ConfigUtils;
+import org.apache.tez.runtime.library.common.InputAttemptIdentifier;
+import org.apache.tez.runtime.library.common.InputIdentifier;
+import org.apache.tez.runtime.library.common.TezRuntimeUtils;
+import org.apache.tez.runtime.library.common.shuffle.server.ShuffleHandler;
+import org.apache.tez.runtime.library.shuffle.common.FetchResult;
+import org.apache.tez.runtime.library.shuffle.common.FetchedInput;
+import org.apache.tez.runtime.library.shuffle.common.FetchedInputAllocator;
+import org.apache.tez.runtime.library.shuffle.common.Fetcher;
+import org.apache.tez.runtime.library.shuffle.common.FetcherCallback;
+import org.apache.tez.runtime.library.shuffle.common.InputHost;
+import org.apache.tez.runtime.library.shuffle.common.ShuffleUtils;
+import org.apache.tez.runtime.library.shuffle.common.Fetcher.FetcherBuilder;
+
+import com.google.common.collect.Lists;
+import com.google.common.util.concurrent.FutureCallback;
+import com.google.common.util.concurrent.Futures;
+import com.google.common.util.concurrent.ListenableFuture;
+import com.google.common.util.concurrent.ListeningExecutorService;
+import com.google.common.util.concurrent.MoreExecutors;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+
+public class BroadcastShuffleManager implements FetcherCallback {
+
+  private static final Log LOG = LogFactory.getLog(BroadcastShuffleManager.class);
+  
+  private TezInputContext inputContext;
+  private int numInputs;
+  private Configuration conf;
+  
+  private final BroadcastShuffleInputEventHandler inputEventHandler;
+  private final FetchedInputAllocator inputManager;
+  
+  private final ExecutorService fetcherRawExecutor;
+  private final ListeningExecutorService fetcherExecutor;
+
+  private final BlockingQueue<FetchedInput> completedInputs;
+  private final Set<InputIdentifier> completedInputSet;
+  private final Set<InputIdentifier> pendingInputs;
+  private final ConcurrentMap<String, InputHost> knownSrcHosts;
+  private final Set<InputHost> pendingHosts;
+  private final Set<InputAttemptIdentifier> obsoletedInputs;
+  
+  private final AtomicInteger numCompletedInputs = new AtomicInteger(0);
+  
+  private final long startTime;
+  private long lastProgressTime;
+  
+  private FutureTask<Void> runShuffleFuture;
+  
+  // Required to be held when manipulating pendingHosts
+  private ReentrantLock lock = new ReentrantLock();
+  private Condition wakeLoop = lock.newCondition();
+  
+  private final int numFetchers;
+  private final AtomicInteger numRunningFetchers = new AtomicInteger(0);
+  
+  // Parameters required by Fetchers
+  private final SecretKey shuffleSecret;
+  private final int connectionTimeout;
+  private final int readTimeout;
+  private final CompressionCodec codec;
+  private final Decompressor decompressor;
+  
+  private final FetchFutureCallback fetchFutureCallback = new FetchFutureCallback();
+  
+  private volatile Throwable shuffleError;
+  
+  // TODO NEWTEZ Add counters.
+  
+  public BroadcastShuffleManager(TezInputContext inputContext, Configuration conf, int numInputs) throws IOException {
+    this.inputContext = inputContext;
+    this.conf = conf;
+    this.numInputs = numInputs;
+    
+    this.inputEventHandler = new BroadcastShuffleInputEventHandler(inputContext, this);
+    this.inputManager = new BroadcastInputManager(inputContext, conf);
+
+    pendingInputs = Collections.newSetFromMap(new ConcurrentHashMap<InputIdentifier, Boolean>(numInputs));
+    completedInputSet = Collections.newSetFromMap(new ConcurrentHashMap<InputIdentifier, Boolean>(numInputs));
+    completedInputs = new LinkedBlockingQueue<FetchedInput>(numInputs);
+    knownSrcHosts = new ConcurrentHashMap<String, InputHost>();
+    pendingHosts = Collections.newSetFromMap(new ConcurrentHashMap<InputHost, Boolean>());
+    obsoletedInputs = Collections.newSetFromMap(new ConcurrentHashMap<InputAttemptIdentifier, Boolean>());
+    
+    int maxConfiguredFetchers = 
+        conf.getInt(
+            TezJobConfig.TEZ_RUNTIME_SHUFFLE_PARALLEL_COPIES, 
+            TezJobConfig.DEFAULT_TEZ_RUNTIME_SHUFFLE_PARALLEL_COPIES);
+    
+    this.numFetchers = Math.min(maxConfiguredFetchers, numInputs);
+    
+    this.fetcherRawExecutor = Executors.newFixedThreadPool(numFetchers,
+        new ThreadFactoryBuilder().setDaemon(true).setNameFormat("Fetcher #%d")
+            .build());
+    this.fetcherExecutor = MoreExecutors.listeningDecorator(fetcherRawExecutor);
+    
+    this.startTime = System.currentTimeMillis();
+    this.lastProgressTime = startTime;
+    
+    this.shuffleSecret = ShuffleUtils
+        .getJobTokenSecretFromTokenBytes(inputContext
+            .getServiceConsumerMetaData(ShuffleHandler.MAPREDUCE_SHUFFLE_SERVICEID));
+    
+    this.connectionTimeout = conf.getInt(
+        TezJobConfig.TEZ_RUNTIME_SHUFFLE_CONNECT_TIMEOUT,
+        TezJobConfig.DEFAULT_TEZ_RUNTIME_SHUFFLE_STALLED_COPY_TIMEOUT);
+    this.readTimeout = conf.getInt(
+        TezJobConfig.TEZ_RUNTIME_SHUFFLE_READ_TIMEOUT,
+        TezJobConfig.DEFAULT_TEZ_RUNTIME_SHUFFLE_READ_TIMEOUT);
+    
+    if (ConfigUtils.isIntermediateInputCompressed(conf)) {
+      Class<? extends CompressionCodec> codecClass = ConfigUtils
+          .getIntermediateInputCompressorClass(conf, DefaultCodec.class);
+      codec = ReflectionUtils.newInstance(codecClass, conf);
+      decompressor = CodecPool.getDecompressor(codec);
+    } else {
+      codec = null;
+      decompressor = null;
+    }
+  }
+  
+  public void run() {
+    RunBroadcastShuffleCallable callable = new RunBroadcastShuffleCallable();
+    runShuffleFuture = new FutureTask<Void>(callable);
+    new Thread(runShuffleFuture, "ShuffleRunner");
+  }
+  
+  private class RunBroadcastShuffleCallable implements Callable<Void> {
+
+    @Override
+    public Void call() throws Exception {
+      while (numCompletedInputs.get() < numInputs) {
+        if (numRunningFetchers.get() >= numFetchers || pendingHosts.size() == 0) {
+          synchronized(lock) {
+            wakeLoop.await();
+          }
+          if (shuffleError != null) {
+            // InputContext has already been informed of a fatal error.
+            // Initiate shutdown.
+            break;
+          }
+          
+          if (numCompletedInputs.get() < numInputs) {
+            synchronized (lock) {
+              int numFetchersToRun = Math.min(pendingHosts.size(), numFetchers - numRunningFetchers.get());
+              int count = 0;
+              for (Iterator<InputHost> inputHostIter = pendingHosts.iterator() ; inputHostIter.hasNext() ; ) {
+                InputHost inputHost = inputHostIter.next();
+                inputHostIter.remove();
+                if (inputHost.getNumPendingInputs() > 0) {
+                  Fetcher fetcher = constructFetcherForHost(inputHost);
+                  numRunningFetchers.incrementAndGet();
+                  ListenableFuture<FetchResult> future = fetcherExecutor
+                      .submit(fetcher);
+                  Futures.addCallback(future, fetchFutureCallback);
+                  if (++count >= numFetchersToRun) {
+                    break;
+                  }
+                }
+              }
+            }
+          }
+        }
+      }
+      // TODO NEWTEZ Maybe clean up inputs.
+      if (!fetcherExecutor.isShutdown()) {
+        fetcherExecutor.shutdownNow();
+      }
+      return null;
+    }
+  }
+  
+  private Fetcher constructFetcherForHost(InputHost inputHost) {
+    FetcherBuilder fetcherBuilder = new FetcherBuilder(
+        BroadcastShuffleManager.this, inputManager,
+        inputContext.getApplicationId(), shuffleSecret, conf);
+    fetcherBuilder.setConnectionParameters(connectionTimeout, readTimeout);
+    fetcherBuilder.setCompressionParameters(codec, decompressor);
+
+    // Remove obsolete inputs from the list being given to the fetcher. Also
+    // remove from the obsolete list.
+    List<InputAttemptIdentifier> pendingInputsForHost = inputHost
+        .clearAndGetPendingInputs();
+    for (Iterator<InputAttemptIdentifier> inputIter = pendingInputsForHost
+        .iterator(); inputIter.hasNext();) {
+      InputAttemptIdentifier input = inputIter.next();
+      // Avoid adding attempts which have already completed.
+      if (completedInputSet.contains(input.getInputIdentifier())) {
+        inputIter.remove();
+      }
+      // Avoid adding attempts which have been marked as OBSOLETE 
+      if (obsoletedInputs.contains(input)) {
+        inputIter.remove();
+        obsoletedInputs.remove(input);
+      }
+    }
+    // TODO NEWTEZ Maybe limit the number of inputs being given to a single
+    // fetcher, especially in the case where #hosts < #fetchers
+    fetcherBuilder.assignWork(inputHost.getHost(), inputHost.getPort(), 0,
+        inputHost.clearAndGetPendingInputs());
+    return fetcherBuilder.build();
+  }
+  
+  /////////////////// Methods for InputEventHandler
+  
+  public void addKnownInput(String hostName, int port,
+      InputAttemptIdentifier srcAttemptIdentifier, int partition) {
+    InputHost host = knownSrcHosts.get(hostName);
+    if (host == null) {
+      host = new InputHost(hostName, port, inputContext.getApplicationId());
+      InputHost old = knownSrcHosts.putIfAbsent(hostName, host);
+      if (old != null) {
+        host = old;
+      }
+    }
+    host.addKnownInput(srcAttemptIdentifier);
+    synchronized(lock) {
+      pendingHosts.add(host);
+      wakeLoop.signal();
+    }
+  }
+
+  public void addCompletedInputWithNoData(
+      InputAttemptIdentifier srcAttemptIdentifier) {
+    InputIdentifier inputIdentifier = srcAttemptIdentifier.getInputIdentifier();
+    LOG.info("No input data exists for SrcTask: " + inputIdentifier + ". Marking as complete.");
+    if (pendingInputs.remove(inputIdentifier)) {
+      completedInputSet.add(inputIdentifier);
+      completedInputs.add(new NullFetchedInput(srcAttemptIdentifier));
+      numCompletedInputs.incrementAndGet();
+    }
+
+    // Awake the loop to check for termination.
+    synchronized (lock) {
+      wakeLoop.signal();
+    } 
+  }
+
+  public synchronized void obsoleteKnownInput(InputAttemptIdentifier srcAttemptIdentifier) {
+    obsoletedInputs.add(srcAttemptIdentifier);
+    // TODO NEWTEZ Maybe inform the fetcher about this. For now, this is used during the initial fetch list construction.
+  }
+  
+  
+  public void handleEvents(List<Event> events) {
+    inputEventHandler.handleEvents(events);
+  }
+
+  /////////////////// End of Methods for InputEventHandler
+  /////////////////// Methods from FetcherCallbackHandler
+  
+  @Override
+  public void fetchSucceeded(String host,
+      InputAttemptIdentifier srcAttemptIdentifier, FetchedInput fetchedInput, long fetchedBytes,
+      long copyDuration) throws IOException {
+    InputIdentifier inputIdentifier = srcAttemptIdentifier.getInputIdentifier();
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Complete fetch for attempt: " + srcAttemptIdentifier + " to " + fetchedInput.getType());
+    }
+    
+    // Count irrespective of whether this is a copy of an already fetched input
+    synchronized(lock) {
+      lastProgressTime = System.currentTimeMillis();
+    }
+    
+    boolean committed = false;
+    if (!completedInputSet.contains(inputIdentifier)) {
+      synchronized (completedInputSet) {
+        if (!completedInputSet.contains(inputIdentifier)) {
+          fetchedInput.commit();
+          committed = true;
+          pendingInputs.remove(inputIdentifier);
+          completedInputSet.add(inputIdentifier);
+          completedInputs.add(fetchedInput);
+          numCompletedInputs.incrementAndGet();
+        }
+      }
+    }
+    if (!committed) {
+      fetchedInput.abort(); // If this fails, the fetcher may attempt another abort.
+    } else {
+      synchronized(lock) {
+        // Signal the wakeLoop to check for termination.
+        wakeLoop.signal();
+      }
+    }
+    // TODO NEWTEZ Maybe inform fetchers, in case they have an alternate attempt of the same task in their queue.
+  }
+
+  @Override
+  public void fetchFailed(String host,
+      InputAttemptIdentifier srcAttemptIdentifier, boolean connectFailed) {
+    // TODO NEWTEZ. Implement logic to report fetch failures after a threshold.
+    // For now, reporting immediately.
+    InputReadErrorEvent readError = new InputReadErrorEvent(
+        "Fetch failure while fetching from "
+            + TezRuntimeUtils.getTaskAttemptIdentifier(
+                inputContext.getSourceVertexName(),
+                srcAttemptIdentifier.getInputIdentifier().getSrcTaskIndex(),
+                srcAttemptIdentifier.getAttemptNumber()),
+        srcAttemptIdentifier.getInputIdentifier().getSrcTaskIndex(),
+        srcAttemptIdentifier.getAttemptNumber());
+    
+    List<Event> failedEvents = Lists.newArrayListWithCapacity(1);
+    failedEvents.add(readError);
+    inputContext.sendEvents(failedEvents);
+  }
+  /////////////////// End of Methods from FetcherCallbackHandler
+
+  public void shutdown() throws InterruptedException {
+    if (this.fetcherExecutor != null && !this.fetcherExecutor.isShutdown()) {
+      this.fetcherExecutor.shutdown();
+      this.fetcherExecutor.awaitTermination(2000l, TimeUnit.MILLISECONDS);
+      if (!this.fetcherExecutor.isShutdown()) {
+        this.fetcherExecutor.shutdownNow();
+      }
+    }
+  }
+  
+  /////////////////// Methods for walking the available inputs
+  
+  /**
+   * @return true if there is another input ready for consumption.
+   */
+  public boolean newInputAvailable() {
+    FetchedInput head = completedInputs.peek();
+    if (head == null || head instanceof NullFetchedInput) {
+      return false;
+    } else {
+      return true;
+    }
+  }
+
+  /**
+   * @return true if all of the required inputs have been fetched.
+   */
+  public boolean allInputsFetched() {
+    return numCompletedInputs.get() == numInputs;
+  }
+
+  /**
+   * @return the next available input, or null if there are no available inputs.
+   *         This method will block if there are currently no available inputs,
+   *         but more may become available.
+   */
+  public FetchedInput getNextInput() throws InterruptedException {
+    FetchedInput input = null;
+    do {
+      input = completedInputs.peek();
+      if (input == null) {
+        if (allInputsFetched()) {
+          break;
+        } else {
+          input = completedInputs.take(); // block
+        }
+      } else {
+        input = completedInputs.poll();
+      }
+    } while (input instanceof NullFetchedInput);
+    return input;
+  }
+
+  /////////////////// End of methods for walking the available inputs
+  
+  
+  /**
+   * Fake input that is added to the completed input list in case an input does not have any data.
+   *
+   */
+  private class NullFetchedInput extends FetchedInput {
+
+    public NullFetchedInput(InputAttemptIdentifier inputAttemptIdentifier) {
+      super(Type.MEMORY, -1, inputAttemptIdentifier, null);
+    }
+
+    @Override
+    public OutputStream getOutputStream() throws IOException {
+      throw new UnsupportedOperationException("Not supported for NullFetchedInput");
+    }
+
+    @Override
+    public InputStream getInputStream() throws IOException {
+      throw new UnsupportedOperationException("Not supported for NullFetchedInput");
+    }
+
+    @Override
+    public void commit() throws IOException {
+      throw new UnsupportedOperationException("Not supported for NullFetchedInput");
+    }
+
+    @Override
+    public void abort() throws IOException {
+      throw new UnsupportedOperationException("Not supported for NullFetchedInput");
+    }
+
+    @Override
+    public void free() {
+      throw new UnsupportedOperationException("Not supported for NullFetchedInput");
+    }
+  }
+  
+  
+  private class FetchFutureCallback implements FutureCallback<FetchResult> {
+
+    private void doBookKeepingForFetcherComplete() {
+      numRunningFetchers.decrementAndGet();
+      synchronized(lock) {
+        wakeLoop.signal();
+      }
+    }
+    
+    @Override
+    public void onSuccess(FetchResult result) {
+      Iterable<InputAttemptIdentifier> pendingInputs = result.getPendingInputs();
+      if (pendingInputs != null && pendingInputs.iterator().hasNext()) {
+        InputHost inputHost = knownSrcHosts.get(result.getHost());
+        assert inputHost != null;
+        for (InputAttemptIdentifier input : pendingInputs) {
+          inputHost.addKnownInput(input);
+        }
+        pendingHosts.add(inputHost);
+      }
+      doBookKeepingForFetcherComplete();
+    }
+
+    @Override
+    public void onFailure(Throwable t) {
+      LOG.error("Fetcher failed with error: " + t);
+      shuffleError = t;
+      inputContext.fatalError(t, "Fetched failed");
+      doBookKeepingForFetcherComplete();
+    }
+  }
+}


[16/50] [abbrv] TEZ-443. Merge tez-dag-api and tez-engine-api into a single module - tez-api (part of TEZ-398). (sseth)

Posted by ss...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine/src/main/java/org/apache/tez/engine/common/TezEngineUtils.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/TezEngineUtils.java b/tez-engine/src/main/java/org/apache/tez/engine/common/TezEngineUtils.java
index ab78d82..cc29e94 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/TezEngineUtils.java
+++ b/tez-engine/src/main/java/org/apache/tez/engine/common/TezEngineUtils.java
@@ -29,11 +29,11 @@ import org.apache.tez.common.Constants;
 import org.apache.tez.common.TezJobConfig;
 import org.apache.tez.dag.api.TezUncheckedException;
 import org.apache.tez.engine.api.Partitioner;
+import org.apache.tez.engine.api.TezOutputContext;
+import org.apache.tez.engine.api.TezTaskContext;
 import org.apache.tez.engine.common.combine.Combiner;
 import org.apache.tez.engine.common.task.local.output.TezTaskOutput;
 import org.apache.tez.engine.common.task.local.output.TezTaskOutputFiles;
-import org.apache.tez.engine.newapi.TezOutputContext;
-import org.apache.tez.engine.newapi.TezTaskContext;
 
 public class TezEngineUtils {
 

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine/src/main/java/org/apache/tez/engine/common/combine/Combiner.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/combine/Combiner.java b/tez-engine/src/main/java/org/apache/tez/engine/common/combine/Combiner.java
index 6f86d61..b387b36 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/combine/Combiner.java
+++ b/tez-engine/src/main/java/org/apache/tez/engine/common/combine/Combiner.java
@@ -23,7 +23,6 @@ import java.io.IOException;
 import org.apache.hadoop.classification.InterfaceAudience.LimitedPrivate;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.tez.common.TezJobConfig;
-import org.apache.tez.common.TezTaskContext;
 import org.apache.tez.engine.common.sort.impl.IFile.Writer;
 import org.apache.tez.engine.common.sort.impl.TezRawKeyValueIterator;
 

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine/src/main/java/org/apache/tez/engine/common/localshuffle/LocalShuffle.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/localshuffle/LocalShuffle.java b/tez-engine/src/main/java/org/apache/tez/engine/common/localshuffle/LocalShuffle.java
index 1cb89a7..546151f 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/localshuffle/LocalShuffle.java
+++ b/tez-engine/src/main/java/org/apache/tez/engine/common/localshuffle/LocalShuffle.java
@@ -32,12 +32,12 @@ import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.tez.common.TezJobConfig;
 import org.apache.tez.common.counters.TaskCounter;
 import org.apache.tez.common.counters.TezCounter;
+import org.apache.tez.engine.api.TezInputContext;
 import org.apache.tez.engine.common.ConfigUtils;
 import org.apache.tez.engine.common.sort.impl.TezMerger;
 import org.apache.tez.engine.common.sort.impl.TezRawKeyValueIterator;
 import org.apache.tez.engine.common.task.local.output.TezLocalTaskOutputFiles;
 import org.apache.tez.engine.common.task.local.output.TezTaskOutput;
-import org.apache.tez.engine.newapi.TezInputContext;
 
 @SuppressWarnings({"rawtypes"})
 public class LocalShuffle {

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/Fetcher.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/Fetcher.java b/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/Fetcher.java
index 5c71644..b48bb0b 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/Fetcher.java
+++ b/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/Fetcher.java
@@ -46,12 +46,12 @@ import org.apache.hadoop.security.ssl.SSLFactory;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.tez.common.TezJobConfig;
 import org.apache.tez.common.counters.TezCounter;
+import org.apache.tez.engine.api.TezInputContext;
 import org.apache.tez.engine.common.ConfigUtils;
 import org.apache.tez.engine.common.InputAttemptIdentifier;
 import org.apache.tez.engine.common.security.SecureShuffleUtils;
 import org.apache.tez.engine.common.shuffle.impl.MapOutput.Type;
 import org.apache.tez.engine.common.sort.impl.IFileInputStream;
-import org.apache.tez.engine.newapi.TezInputContext;
 
 import com.google.common.annotations.VisibleForTesting;
 

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/MergeManager.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/MergeManager.java b/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/MergeManager.java
index bf2be4e..b8792fb 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/MergeManager.java
+++ b/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/MergeManager.java
@@ -46,6 +46,7 @@ import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.tez.common.Constants;
 import org.apache.tez.common.TezJobConfig;
 import org.apache.tez.common.counters.TezCounter;
+import org.apache.tez.engine.api.TezInputContext;
 import org.apache.tez.engine.common.ConfigUtils;
 import org.apache.tez.engine.common.InputAttemptIdentifier;
 import org.apache.tez.engine.common.combine.Combiner;
@@ -56,7 +57,6 @@ import org.apache.tez.engine.common.sort.impl.IFile.Writer;
 import org.apache.tez.engine.common.sort.impl.TezMerger.Segment;
 import org.apache.tez.engine.common.task.local.output.TezTaskOutputFiles;
 import org.apache.tez.engine.hadoop.compat.NullProgressable;
-import org.apache.tez.engine.newapi.TezInputContext;
 
 @InterfaceAudience.Private
 @InterfaceStability.Unstable

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/Shuffle.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/Shuffle.java b/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/Shuffle.java
index f605b7c..15332a1 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/Shuffle.java
+++ b/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/Shuffle.java
@@ -38,12 +38,12 @@ import org.apache.tez.common.TezJobConfig;
 import org.apache.tez.common.counters.TaskCounter;
 import org.apache.tez.common.counters.TezCounter;
 import org.apache.tez.dag.api.TezUncheckedException;
+import org.apache.tez.engine.api.Event;
+import org.apache.tez.engine.api.TezInputContext;
 import org.apache.tez.engine.common.TezEngineUtils;
 import org.apache.tez.engine.common.combine.Combiner;
 import org.apache.tez.engine.common.shuffle.server.ShuffleHandler;
 import org.apache.tez.engine.common.sort.impl.TezRawKeyValueIterator;
-import org.apache.tez.engine.newapi.Event;
-import org.apache.tez.engine.newapi.TezInputContext;
 import org.apache.tez.engine.shuffle.common.ShuffleUtils;
 
 import com.google.common.base.Preconditions;

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/ShuffleInputEventHandler.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/ShuffleInputEventHandler.java b/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/ShuffleInputEventHandler.java
index 98c7ab1..a8e5fe4 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/ShuffleInputEventHandler.java
+++ b/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/ShuffleInputEventHandler.java
@@ -24,14 +24,14 @@ import java.util.List;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.tez.dag.api.TezUncheckedException;
+import org.apache.tez.engine.api.Event;
+import org.apache.tez.engine.api.TezInputContext;
+import org.apache.tez.engine.api.events.DataMovementEvent;
+import org.apache.tez.engine.api.events.InputFailedEvent;
+import org.apache.tez.engine.api.events.InputInformationEvent;
 import org.apache.tez.engine.common.InputAttemptIdentifier;
 import org.apache.tez.engine.common.shuffle.newimpl.ShuffleUserPayloads.DataMovementEventPayloadProto;
 import org.apache.tez.engine.common.shuffle.newimpl.ShuffleUserPayloads.InputInformationEventPayloadProto;
-import org.apache.tez.engine.newapi.Event;
-import org.apache.tez.engine.newapi.TezInputContext;
-import org.apache.tez.engine.newapi.events.DataMovementEvent;
-import org.apache.tez.engine.newapi.events.InputFailedEvent;
-import org.apache.tez.engine.newapi.events.InputInformationEvent;
 
 import com.google.common.base.Preconditions;
 import com.google.protobuf.InvalidProtocolBufferException;

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/ShuffleScheduler.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/ShuffleScheduler.java b/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/ShuffleScheduler.java
index acbd054..be75668 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/ShuffleScheduler.java
+++ b/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/ShuffleScheduler.java
@@ -40,11 +40,11 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.IntWritable;
 import org.apache.tez.common.TezJobConfig;
 import org.apache.tez.common.counters.TezCounter;
+import org.apache.tez.engine.api.Event;
+import org.apache.tez.engine.api.TezInputContext;
+import org.apache.tez.engine.api.events.InputReadErrorEvent;
 import org.apache.tez.engine.common.InputAttemptIdentifier;
 import org.apache.tez.engine.common.TezEngineUtils;
-import org.apache.tez.engine.newapi.Event;
-import org.apache.tez.engine.newapi.TezInputContext;
-import org.apache.tez.engine.newapi.events.InputReadErrorEvent;
 
 import com.google.common.collect.Lists;
 

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/server/ShuffleHandler.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/server/ShuffleHandler.java b/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/server/ShuffleHandler.java
index f61670e..a6d1c5b 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/server/ShuffleHandler.java
+++ b/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/server/ShuffleHandler.java
@@ -64,12 +64,12 @@ import org.apache.hadoop.yarn.server.api.ApplicationInitializationContext;
 import org.apache.hadoop.yarn.server.api.ApplicationTerminationContext;
 import org.apache.hadoop.yarn.server.api.AuxiliaryService;
 import org.apache.tez.common.TezJobConfig;
+import org.apache.tez.engine.api.TezOutputContext;
 import org.apache.tez.engine.common.security.JobTokenIdentifier;
 import org.apache.tez.engine.common.security.JobTokenSecretManager;
 import org.apache.tez.engine.common.security.SecureShuffleUtils;
 import org.apache.tez.engine.common.shuffle.impl.ShuffleHeader;
 import org.apache.tez.engine.common.sort.impl.ExternalSorter;
-import org.apache.tez.engine.newapi.TezOutputContext;
 import org.apache.tez.engine.shuffle.common.ShuffleUtils;
 import org.jboss.netty.bootstrap.ServerBootstrap;
 import org.jboss.netty.buffer.ChannelBuffers;

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/ExternalSorter.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/ExternalSorter.java b/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/ExternalSorter.java
index 8b4bd4e..5aa0ddf 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/ExternalSorter.java
+++ b/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/ExternalSorter.java
@@ -43,6 +43,7 @@ import org.apache.tez.common.TezJobConfig;
 import org.apache.tez.common.counters.TaskCounter;
 import org.apache.tez.common.counters.TezCounter;
 import org.apache.tez.engine.api.Partitioner;
+import org.apache.tez.engine.api.TezOutputContext;
 import org.apache.tez.engine.common.ConfigUtils;
 import org.apache.tez.engine.common.TezEngineUtils;
 import org.apache.tez.engine.common.combine.Combiner;
@@ -50,7 +51,6 @@ import org.apache.tez.engine.common.shuffle.impl.ShuffleHeader;
 import org.apache.tez.engine.common.sort.impl.IFile.Writer;
 import org.apache.tez.engine.common.task.local.output.TezTaskOutput;
 import org.apache.tez.engine.hadoop.compat.NullProgressable;
-import org.apache.tez.engine.newapi.TezOutputContext;
 
 @SuppressWarnings({"unchecked", "rawtypes"})
 public abstract class ExternalSorter {

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/PipelinedSorter.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/PipelinedSorter.java b/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/PipelinedSorter.java
index 952568e..1bf17a3 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/PipelinedSorter.java
+++ b/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/PipelinedSorter.java
@@ -44,10 +44,10 @@ import org.apache.hadoop.util.IndexedSortable;
 import org.apache.hadoop.util.IndexedSorter;
 import org.apache.hadoop.util.Progress;
 import org.apache.tez.common.TezJobConfig;
+import org.apache.tez.engine.api.TezOutputContext;
 import org.apache.tez.engine.common.ConfigUtils;
 import org.apache.tez.engine.common.sort.impl.IFile.Writer;
 import org.apache.tez.engine.common.sort.impl.TezMerger.Segment;
-import org.apache.tez.engine.newapi.TezOutputContext;
 
 @SuppressWarnings({"unchecked", "rawtypes"})
 public class PipelinedSorter extends ExternalSorter {

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/dflt/DefaultSorter.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/dflt/DefaultSorter.java b/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/dflt/DefaultSorter.java
index 1ad31f7..6b48270 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/dflt/DefaultSorter.java
+++ b/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/dflt/DefaultSorter.java
@@ -40,6 +40,7 @@ import org.apache.hadoop.util.IndexedSortable;
 import org.apache.hadoop.util.Progress;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.tez.common.TezJobConfig;
+import org.apache.tez.engine.api.TezOutputContext;
 import org.apache.tez.engine.common.ConfigUtils;
 import org.apache.tez.engine.common.sort.impl.ExternalSorter;
 import org.apache.tez.engine.common.sort.impl.IFile;
@@ -49,7 +50,6 @@ import org.apache.tez.engine.common.sort.impl.TezRawKeyValueIterator;
 import org.apache.tez.engine.common.sort.impl.TezSpillRecord;
 import org.apache.tez.engine.common.sort.impl.IFile.Writer;
 import org.apache.tez.engine.common.sort.impl.TezMerger.Segment;
-import org.apache.tez.engine.newapi.TezOutputContext;
 
 @SuppressWarnings({"unchecked", "rawtypes"})
 public class DefaultSorter extends ExternalSorter implements IndexedSortable {

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/dflt/InMemoryShuffleSorter.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/dflt/InMemoryShuffleSorter.java b/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/dflt/InMemoryShuffleSorter.java
index 53e6003..e2b3315 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/dflt/InMemoryShuffleSorter.java
+++ b/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/dflt/InMemoryShuffleSorter.java
@@ -29,10 +29,10 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.WritableUtils;
 import org.apache.hadoop.util.DataChecksum;
+import org.apache.tez.engine.api.TezOutputContext;
 import org.apache.tez.engine.common.shuffle.impl.ShuffleHeader;
 import org.apache.tez.engine.common.shuffle.server.ShuffleHandler;
 import org.apache.tez.engine.common.sort.impl.IFile;
-import org.apache.tez.engine.newapi.TezOutputContext;
 
 public class InMemoryShuffleSorter extends DefaultSorter {
 

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine/src/main/java/org/apache/tez/engine/lib/input/LocalMergedInput.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/lib/input/LocalMergedInput.java b/tez-engine/src/main/java/org/apache/tez/engine/lib/input/LocalMergedInput.java
index 8ae6bfe..ed57c61 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/lib/input/LocalMergedInput.java
+++ b/tez-engine/src/main/java/org/apache/tez/engine/lib/input/LocalMergedInput.java
@@ -22,10 +22,10 @@ import java.util.Collections;
 import java.util.List;
 
 import org.apache.tez.common.TezUtils;
+import org.apache.tez.engine.api.Event;
+import org.apache.tez.engine.api.LogicalInput;
+import org.apache.tez.engine.api.TezInputContext;
 import org.apache.tez.engine.common.localshuffle.LocalShuffle;
-import org.apache.tez.engine.newapi.Event;
-import org.apache.tez.engine.newapi.LogicalInput;
-import org.apache.tez.engine.newapi.TezInputContext;
 
 /**
  * <code>LocalMergedInput</code> in an {@link LogicalInput} which shuffles intermediate

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine/src/main/java/org/apache/tez/engine/lib/input/ShuffledMergedInput.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/lib/input/ShuffledMergedInput.java b/tez-engine/src/main/java/org/apache/tez/engine/lib/input/ShuffledMergedInput.java
index c719fba..3db0632 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/lib/input/ShuffledMergedInput.java
+++ b/tez-engine/src/main/java/org/apache/tez/engine/lib/input/ShuffledMergedInput.java
@@ -29,14 +29,14 @@ import org.apache.tez.common.TezJobConfig;
 import org.apache.tez.common.TezUtils;
 import org.apache.tez.common.counters.TaskCounter;
 import org.apache.tez.common.counters.TezCounter;
+import org.apache.tez.engine.api.Event;
 import org.apache.tez.engine.api.KVReader;
+import org.apache.tez.engine.api.LogicalInput;
+import org.apache.tez.engine.api.TezInputContext;
 import org.apache.tez.engine.common.ConfigUtils;
 import org.apache.tez.engine.common.ValuesIterator;
 import org.apache.tez.engine.common.shuffle.impl.Shuffle;
 import org.apache.tez.engine.common.sort.impl.TezRawKeyValueIterator;
-import org.apache.tez.engine.newapi.Event;
-import org.apache.tez.engine.newapi.LogicalInput;
-import org.apache.tez.engine.newapi.TezInputContext;
 
 /**
  * <code>ShuffleMergedInput</code> in a {@link LogicalInput} which shuffles

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine/src/main/java/org/apache/tez/engine/lib/input/ShuffledUnorderedKVInput.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/lib/input/ShuffledUnorderedKVInput.java b/tez-engine/src/main/java/org/apache/tez/engine/lib/input/ShuffledUnorderedKVInput.java
index c2126ca..44238fd 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/lib/input/ShuffledUnorderedKVInput.java
+++ b/tez-engine/src/main/java/org/apache/tez/engine/lib/input/ShuffledUnorderedKVInput.java
@@ -23,11 +23,11 @@ import java.util.List;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.tez.common.TezJobConfig;
 import org.apache.tez.common.TezUtils;
+import org.apache.tez.engine.api.Event;
+import org.apache.tez.engine.api.LogicalInput;
+import org.apache.tez.engine.api.Reader;
+import org.apache.tez.engine.api.TezInputContext;
 import org.apache.tez.engine.broadcast.input.BroadcastShuffleManager;
-import org.apache.tez.engine.newapi.Event;
-import org.apache.tez.engine.newapi.LogicalInput;
-import org.apache.tez.engine.newapi.Reader;
-import org.apache.tez.engine.newapi.TezInputContext;
 
 import com.google.common.base.Preconditions;
 

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine/src/main/java/org/apache/tez/engine/lib/output/InMemorySortedOutput.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/lib/output/InMemorySortedOutput.java b/tez-engine/src/main/java/org/apache/tez/engine/lib/output/InMemorySortedOutput.java
index 218aa21..26a01c8 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/lib/output/InMemorySortedOutput.java
+++ b/tez-engine/src/main/java/org/apache/tez/engine/lib/output/InMemorySortedOutput.java
@@ -22,13 +22,13 @@ import java.util.Collections;
 import java.util.List;
 
 import org.apache.tez.common.TezUtils;
+import org.apache.tez.engine.api.Event;
 import org.apache.tez.engine.api.KVWriter;
+import org.apache.tez.engine.api.LogicalOutput;
+import org.apache.tez.engine.api.Output;
+import org.apache.tez.engine.api.TezOutputContext;
+import org.apache.tez.engine.api.Writer;
 import org.apache.tez.engine.common.sort.impl.dflt.InMemoryShuffleSorter;
-import org.apache.tez.engine.newapi.Event;
-import org.apache.tez.engine.newapi.LogicalOutput;
-import org.apache.tez.engine.newapi.Output;
-import org.apache.tez.engine.newapi.TezOutputContext;
-import org.apache.tez.engine.newapi.Writer;
 
 /**
  * {@link InMemorySortedOutput} is an {@link Output} which sorts key/value pairs 

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine/src/main/java/org/apache/tez/engine/lib/output/LocalOnFileSorterOutput.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/lib/output/LocalOnFileSorterOutput.java b/tez-engine/src/main/java/org/apache/tez/engine/lib/output/LocalOnFileSorterOutput.java
index 963276d..b24e10d 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/lib/output/LocalOnFileSorterOutput.java
+++ b/tez-engine/src/main/java/org/apache/tez/engine/lib/output/LocalOnFileSorterOutput.java
@@ -25,8 +25,8 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.tez.engine.api.Event;
 import org.apache.tez.engine.common.task.local.output.TezTaskOutput;
-import org.apache.tez.engine.newapi.Event;
 
 public class LocalOnFileSorterOutput extends OnFileSortedOutput {
 

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine/src/main/java/org/apache/tez/engine/lib/output/OnFileSortedOutput.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/lib/output/OnFileSortedOutput.java b/tez-engine/src/main/java/org/apache/tez/engine/lib/output/OnFileSortedOutput.java
index 7e0ca37..685722e 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/lib/output/OnFileSortedOutput.java
+++ b/tez-engine/src/main/java/org/apache/tez/engine/lib/output/OnFileSortedOutput.java
@@ -26,14 +26,14 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.yarn.api.ApplicationConstants;
 import org.apache.tez.common.TezJobConfig;
 import org.apache.tez.common.TezUtils;
+import org.apache.tez.engine.api.Event;
 import org.apache.tez.engine.api.KVWriter;
+import org.apache.tez.engine.api.LogicalOutput;
+import org.apache.tez.engine.api.TezOutputContext;
+import org.apache.tez.engine.api.events.DataMovementEvent;
 import org.apache.tez.engine.common.shuffle.newimpl.ShuffleUserPayloads.DataMovementEventPayloadProto;
 import org.apache.tez.engine.common.sort.impl.ExternalSorter;
 import org.apache.tez.engine.common.sort.impl.dflt.DefaultSorter;
-import org.apache.tez.engine.newapi.Event;
-import org.apache.tez.engine.newapi.LogicalOutput;
-import org.apache.tez.engine.newapi.TezOutputContext;
-import org.apache.tez.engine.newapi.events.DataMovementEvent;
 import org.apache.tez.engine.shuffle.common.ShuffleUtils;
 
 import com.google.common.collect.Lists;

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine/src/main/java/org/apache/tez/engine/lib/output/OnFileUnorderedKVOutput.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/lib/output/OnFileUnorderedKVOutput.java b/tez-engine/src/main/java/org/apache/tez/engine/lib/output/OnFileUnorderedKVOutput.java
index 37edde8..3ff603f 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/lib/output/OnFileUnorderedKVOutput.java
+++ b/tez-engine/src/main/java/org/apache/tez/engine/lib/output/OnFileUnorderedKVOutput.java
@@ -24,13 +24,13 @@ import java.util.List;
 
 import org.apache.hadoop.yarn.api.ApplicationConstants;
 import org.apache.tez.dag.api.TezUncheckedException;
+import org.apache.tez.engine.api.Event;
 import org.apache.tez.engine.api.KVWriter;
+import org.apache.tez.engine.api.LogicalOutput;
+import org.apache.tez.engine.api.TezOutputContext;
+import org.apache.tez.engine.api.events.DataMovementEvent;
 import org.apache.tez.engine.broadcast.output.FileBasedKVWriter;
 import org.apache.tez.engine.common.shuffle.newimpl.ShuffleUserPayloads.DataMovementEventPayloadProto;
-import org.apache.tez.engine.newapi.Event;
-import org.apache.tez.engine.newapi.LogicalOutput;
-import org.apache.tez.engine.newapi.TezOutputContext;
-import org.apache.tez.engine.newapi.events.DataMovementEvent;
 import org.apache.tez.engine.shuffle.common.ShuffleUtils;
 
 import com.google.common.base.Preconditions;

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine/src/main/java/org/apache/tez/engine/newruntime/LogicalIOProcessorRuntimeTask.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/newruntime/LogicalIOProcessorRuntimeTask.java b/tez-engine/src/main/java/org/apache/tez/engine/newruntime/LogicalIOProcessorRuntimeTask.java
index 77299de..bfd898b 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/newruntime/LogicalIOProcessorRuntimeTask.java
+++ b/tez-engine/src/main/java/org/apache/tez/engine/newruntime/LogicalIOProcessorRuntimeTask.java
@@ -38,6 +38,16 @@ import org.apache.hadoop.util.StringUtils;
 import org.apache.tez.dag.api.ProcessorDescriptor;
 import org.apache.tez.dag.api.TezUncheckedException;
 import org.apache.tez.dag.records.TezTaskAttemptID;
+import org.apache.tez.engine.api.Event;
+import org.apache.tez.engine.api.Input;
+import org.apache.tez.engine.api.LogicalIOProcessor;
+import org.apache.tez.engine.api.LogicalInput;
+import org.apache.tez.engine.api.LogicalOutput;
+import org.apache.tez.engine.api.Output;
+import org.apache.tez.engine.api.Processor;
+import org.apache.tez.engine.api.TezInputContext;
+import org.apache.tez.engine.api.TezOutputContext;
+import org.apache.tez.engine.api.TezProcessorContext;
 import org.apache.tez.engine.api.impl.EventMetaData;
 import org.apache.tez.engine.api.impl.InputSpec;
 import org.apache.tez.engine.api.impl.OutputSpec;
@@ -49,16 +59,6 @@ import org.apache.tez.engine.api.impl.TezProcessorContextImpl;
 import org.apache.tez.engine.api.impl.TezUmbilical;
 import org.apache.tez.engine.api.impl.EventMetaData.EventProducerConsumerType;
 import org.apache.tez.engine.common.security.JobTokenIdentifier;
-import org.apache.tez.engine.newapi.Event;
-import org.apache.tez.engine.newapi.Input;
-import org.apache.tez.engine.newapi.LogicalIOProcessor;
-import org.apache.tez.engine.newapi.LogicalInput;
-import org.apache.tez.engine.newapi.LogicalOutput;
-import org.apache.tez.engine.newapi.Output;
-import org.apache.tez.engine.newapi.Processor;
-import org.apache.tez.engine.newapi.TezInputContext;
-import org.apache.tez.engine.newapi.TezOutputContext;
-import org.apache.tez.engine.newapi.TezProcessorContext;
 import org.apache.tez.engine.shuffle.common.ShuffleUtils;
 
 import com.google.common.base.Preconditions;

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-mapreduce/pom.xml
----------------------------------------------------------------------
diff --git a/tez-mapreduce/pom.xml b/tez-mapreduce/pom.xml
index ea5016b..2c6b78e 100644
--- a/tez-mapreduce/pom.xml
+++ b/tez-mapreduce/pom.xml
@@ -27,7 +27,7 @@
   <dependencies>
     <dependency>
       <groupId>org.apache.tez</groupId>
-      <artifactId>tez-dag-api</artifactId>
+      <artifactId>tez-api</artifactId>
     </dependency>
     <dependency>
       <groupId>org.apache.tez</groupId>

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-mapreduce/src/main/java/org/apache/tez/common/TezTaskStatus.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/common/TezTaskStatus.java b/tez-mapreduce/src/main/java/org/apache/tez/common/TezTaskStatus.java
new file mode 100644
index 0000000..45ea80e
--- /dev/null
+++ b/tez-mapreduce/src/main/java/org/apache/tez/common/TezTaskStatus.java
@@ -0,0 +1,105 @@
+/**
+* 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.common;
+
+import java.util.List;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.io.Writable;
+import org.apache.tez.common.counters.TezCounters;
+import org.apache.tez.dag.records.TezTaskAttemptID;
+
+// TODO NEWTEZ Get rid of this.
+public interface TezTaskStatus extends Writable {
+
+  //enumeration for reporting current phase of a task.
+  @InterfaceAudience.Private
+  @InterfaceStability.Unstable
+  public static enum Phase{STARTING, MAP, SHUFFLE, SORT, REDUCE, CLEANUP}
+
+  // what state is the task in?
+  @InterfaceAudience.Private
+  @InterfaceStability.Unstable
+  public static enum State {RUNNING, SUCCEEDED, FAILED, UNASSIGNED, KILLED, 
+                            COMMIT_PENDING, FAILED_UNCLEAN, KILLED_UNCLEAN}
+
+  public abstract TezTaskAttemptID getTaskAttemptId();
+
+  public abstract float getProgress();
+
+  public abstract void setProgress(float progress);
+
+  public abstract State getRunState();
+
+  public abstract void setRunState(State runState);
+
+  public abstract String getDiagnosticInfo();
+
+  public abstract void setDiagnosticInfo(String info);
+
+  // TODOTEZDAG Remove stateString / rename
+  public abstract String getStateString();
+
+  public abstract void setStateString(String stateString);
+
+  public abstract long getFinishTime();
+
+  public abstract void setFinishTime(long finishTime);
+  
+  // TODOTEZDAG Can shuffle / merge be made generic ? Otherwise just a single finish time.
+  public abstract long getShuffleFinishTime();
+
+  public abstract void setShuffleFinishTime(long shuffleFinishTime);
+  
+  public abstract long getMapFinishTime();
+
+  public abstract void setMapFinishTime(long mapFinishTime);
+  
+  public abstract long getSortFinishTime();
+  
+  public abstract void setSortFinishTime(long sortFinishTime);
+  
+  public abstract long getStartTime();
+  
+  public abstract void setStartTime(long startTime);
+
+  // TODOTEZDAG Remove phase
+  public abstract Phase getPhase();
+
+  public abstract void setPhase(Phase phase);
+
+  public abstract TezCounters getCounters();
+
+  public abstract void setCounters(TezCounters counters);
+
+  public abstract List<TezTaskAttemptID> getFailedDependencies();
+
+  public abstract void addFailedDependency(TezTaskAttemptID taskAttempttId);
+
+  public abstract void clearStatus();
+
+  public abstract void statusUpdate(float f, String string, TezCounters counters);
+
+  // TODOTEZDAG maybe remove ?
+  public abstract long getLocalOutputSize();
+
+  public abstract void setOutputSize(long l);
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/combine/MRCombiner.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/combine/MRCombiner.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/combine/MRCombiner.java
index 56e88c7..199bbfe 100644
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/combine/MRCombiner.java
+++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/combine/MRCombiner.java
@@ -42,14 +42,14 @@ import org.apache.hadoop.util.Progress;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.tez.common.TezUtils;
 import org.apache.tez.common.counters.TezCounter;
+import org.apache.tez.engine.api.TezInputContext;
+import org.apache.tez.engine.api.TezOutputContext;
+import org.apache.tez.engine.api.TezTaskContext;
 import org.apache.tez.engine.common.ConfigUtils;
 import org.apache.tez.engine.common.ValuesIterator;
 import org.apache.tez.engine.common.combine.Combiner;
 import org.apache.tez.engine.common.sort.impl.IFile.Writer;
 import org.apache.tez.engine.common.sort.impl.TezRawKeyValueIterator;
-import org.apache.tez.engine.newapi.TezInputContext;
-import org.apache.tez.engine.newapi.TezOutputContext;
-import org.apache.tez.engine.newapi.TezTaskContext;
 import org.apache.tez.mapreduce.hadoop.MRConfig;
 import org.apache.tez.mapreduce.hadoop.MRJobConfig;
 import org.apache.tez.mapreduce.hadoop.mapred.MRCounters;

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/TezTypeConverters.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/TezTypeConverters.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/TezTypeConverters.java
index 768d347..889c64c 100644
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/TezTypeConverters.java
+++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/TezTypeConverters.java
@@ -22,7 +22,6 @@ import org.apache.hadoop.mapreduce.Counter;
 import org.apache.hadoop.mapreduce.Counters;
 import org.apache.hadoop.mapreduce.JobID;
 import org.apache.hadoop.mapreduce.TypeConverter;
-import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptCompletionEventStatus;
 import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId;
 import org.apache.tez.common.TezTaskStatus.Phase;
 import org.apache.tez.common.counters.CounterGroup;
@@ -30,7 +29,6 @@ import org.apache.tez.common.counters.TezCounter;
 import org.apache.tez.common.counters.TezCounters;
 import org.apache.tez.dag.records.TezDAGID;
 import org.apache.tez.dag.records.TezTaskAttemptID;
-import org.apache.tez.engine.records.TezDependentTaskCompletionEvent;
 
 public class TezTypeConverters {
 
@@ -66,13 +64,6 @@ public class TezTypeConverters {
     return tezTaskAttemptId;
   }
 
-  public static TezDependentTaskCompletionEvent.Status toTez(
-      TaskAttemptCompletionEventStatus status) {
-    return TezDependentTaskCompletionEvent.Status.valueOf(status.toString());
-  }
-
-  
-  
   public static Counters fromTez(TezCounters tezCounters) {
     if (tezCounters == null) {
       return null;

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/mapred/MRReporter.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/mapred/MRReporter.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/mapred/MRReporter.java
index f5e08dc..2a926d7 100644
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/mapred/MRReporter.java
+++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/mapred/MRReporter.java
@@ -3,8 +3,8 @@ package org.apache.tez.mapreduce.hadoop.mapred;
 import org.apache.hadoop.mapred.Counters.Counter;
 import org.apache.hadoop.mapred.InputSplit;
 import org.apache.hadoop.mapred.Reporter;
-import org.apache.tez.engine.newapi.TezProcessorContext;
-import org.apache.tez.engine.newapi.TezTaskContext;
+import org.apache.tez.engine.api.TezProcessorContext;
+import org.apache.tez.engine.api.TezTaskContext;
 import org.apache.tez.mapreduce.common.Utils;
 
 public class MRReporter implements Reporter {

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/mapreduce/MapContextImpl.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/mapreduce/MapContextImpl.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/mapreduce/MapContextImpl.java
index b0348c9..dcdb3ff 100644
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/mapreduce/MapContextImpl.java
+++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/mapreduce/MapContextImpl.java
@@ -30,7 +30,7 @@ import org.apache.hadoop.mapreduce.OutputCommitter;
 import org.apache.hadoop.mapreduce.RecordReader;
 import org.apache.hadoop.mapreduce.RecordWriter;
 import org.apache.hadoop.mapreduce.TaskAttemptID;
-import org.apache.tez.engine.newapi.TezTaskContext;
+import org.apache.tez.engine.api.TezTaskContext;
 
 /**
  * The context that is given to the {@link Mapper}.

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/mapreduce/TaskAttemptContextImpl.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/mapreduce/TaskAttemptContextImpl.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/mapreduce/TaskAttemptContextImpl.java
index d8548a4..4035c71 100644
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/mapreduce/TaskAttemptContextImpl.java
+++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/mapreduce/TaskAttemptContextImpl.java
@@ -24,7 +24,7 @@ import org.apache.hadoop.mapreduce.Counter;
 import org.apache.hadoop.mapreduce.TaskAttemptID;
 import org.apache.hadoop.mapreduce.TaskID;
 import org.apache.hadoop.mapreduce.TaskType;
-import org.apache.tez.engine.newapi.TezTaskContext;
+import org.apache.tez.engine.api.TezTaskContext;
 import org.apache.tez.mapreduce.common.Utils;
 
 // NOTE: NEWTEZ: This is a copy of org.apache.tez.mapreduce.hadoop.mapred (not mapreduce). mapred likely does not need it's own copy of this class.

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/mapreduce/TaskInputOutputContextImpl.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/mapreduce/TaskInputOutputContextImpl.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/mapreduce/TaskInputOutputContextImpl.java
index c2920dc..05ea89c 100644
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/mapreduce/TaskInputOutputContextImpl.java
+++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/mapreduce/TaskInputOutputContextImpl.java
@@ -29,7 +29,7 @@ import org.apache.hadoop.mapreduce.RecordWriter;
 import org.apache.hadoop.mapreduce.Reducer;
 import org.apache.hadoop.mapreduce.TaskAttemptID;
 import org.apache.hadoop.mapreduce.TaskInputOutputContext;
-import org.apache.tez.engine.newapi.TezTaskContext;
+import org.apache.tez.engine.api.TezTaskContext;
 
 /**
  * A context object that allows input and output from the task. It is only

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/SimpleInput.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/SimpleInput.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/SimpleInput.java
index f03fdc7..598f801 100644
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/SimpleInput.java
+++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/SimpleInput.java
@@ -46,10 +46,10 @@ import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.tez.common.TezUtils;
 import org.apache.tez.common.counters.TaskCounter;
 import org.apache.tez.common.counters.TezCounter;
+import org.apache.tez.engine.api.Event;
 import org.apache.tez.engine.api.KVReader;
-import org.apache.tez.engine.newapi.Event;
-import org.apache.tez.engine.newapi.LogicalInput;
-import org.apache.tez.engine.newapi.TezInputContext;
+import org.apache.tez.engine.api.LogicalInput;
+import org.apache.tez.engine.api.TezInputContext;
 import org.apache.tez.mapreduce.common.Utils;
 import org.apache.tez.mapreduce.hadoop.mapred.MRReporter;
 import org.apache.tez.mapreduce.hadoop.mapreduce.TaskAttemptContextImpl;

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/output/SimpleOutput.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/output/SimpleOutput.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/output/SimpleOutput.java
index dd28ab8..d82c9e2 100644
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/output/SimpleOutput.java
+++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/output/SimpleOutput.java
@@ -26,10 +26,10 @@ import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.tez.common.TezUtils;
 import org.apache.tez.common.counters.TaskCounter;
 import org.apache.tez.common.counters.TezCounter;
+import org.apache.tez.engine.api.Event;
 import org.apache.tez.engine.api.KVWriter;
-import org.apache.tez.engine.newapi.Event;
-import org.apache.tez.engine.newapi.LogicalOutput;
-import org.apache.tez.engine.newapi.TezOutputContext;
+import org.apache.tez.engine.api.LogicalOutput;
+import org.apache.tez.engine.api.TezOutputContext;
 import org.apache.tez.mapreduce.common.Utils;
 import org.apache.tez.mapreduce.hadoop.MRConfig;
 import org.apache.tez.mapreduce.hadoop.MRJobConfig;

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/MRTask.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/MRTask.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/MRTask.java
index af087c9..1a01466 100644
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/MRTask.java
+++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/MRTask.java
@@ -71,12 +71,11 @@ import org.apache.tez.common.counters.TaskCounter;
 import org.apache.tez.common.counters.TezCounter;
 import org.apache.tez.common.counters.TezCounters;
 import org.apache.tez.dag.records.TezDAGID;
+import org.apache.tez.engine.api.LogicalOutput;
+import org.apache.tez.engine.api.TezProcessorContext;
 import org.apache.tez.engine.common.security.JobTokenIdentifier;
 import org.apache.tez.engine.common.security.TokenCache;
 import org.apache.tez.engine.common.sort.impl.TezRawKeyValueIterator;
-import org.apache.tez.engine.newapi.LogicalOutput;
-import org.apache.tez.engine.newapi.TezProcessorContext;
-import org.apache.tez.engine.records.OutputContext;
 import org.apache.tez.mapreduce.hadoop.DeprecatedKeys;
 import org.apache.tez.mapreduce.hadoop.IDConverter;
 import org.apache.tez.mapreduce.hadoop.MRConfig;
@@ -422,14 +421,6 @@ public abstract class MRTask {
       InterruptedException {
   }
 
-  public void outputReady(MRTaskReporter reporter, OutputContext outputContext)
-      throws IOException,
-      InterruptedException {
-    LOG.info("Task: " + taskAttemptId + " reporting outputReady");
-    updateCounters();
-    statusUpdate();
-  }
-
   public void done(LogicalOutput output) throws IOException, InterruptedException {
     updateCounters();
 

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/MRTaskReporter.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/MRTaskReporter.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/MRTaskReporter.java
index d32b4c0..22312f7 100644
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/MRTaskReporter.java
+++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/MRTaskReporter.java
@@ -24,10 +24,10 @@ import org.apache.hadoop.mapred.Counters;
 import org.apache.hadoop.mapred.InputSplit;
 import org.apache.hadoop.mapred.Reporter;
 import org.apache.tez.common.counters.TezCounter;
-import org.apache.tez.engine.newapi.TezInputContext;
-import org.apache.tez.engine.newapi.TezOutputContext;
-import org.apache.tez.engine.newapi.TezProcessorContext;
-import org.apache.tez.engine.newapi.TezTaskContext;
+import org.apache.tez.engine.api.TezInputContext;
+import org.apache.tez.engine.api.TezOutputContext;
+import org.apache.tez.engine.api.TezProcessorContext;
+import org.apache.tez.engine.api.TezTaskContext;
 import org.apache.tez.mapreduce.hadoop.mapred.MRCounters;
 import org.apache.tez.mapreduce.hadoop.mapred.MRReporter;
 

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/map/MapProcessor.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/map/MapProcessor.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/map/MapProcessor.java
index e1ed168..2084146 100644
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/map/MapProcessor.java
+++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/map/MapProcessor.java
@@ -35,14 +35,14 @@ import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.tez.common.counters.TaskCounter;
 import org.apache.tez.common.counters.TezCounter;
 import org.apache.tez.dag.api.TezException;
+import org.apache.tez.engine.api.Event;
 import org.apache.tez.engine.api.KVReader;
 import org.apache.tez.engine.api.KVWriter;
+import org.apache.tez.engine.api.LogicalIOProcessor;
+import org.apache.tez.engine.api.LogicalInput;
+import org.apache.tez.engine.api.LogicalOutput;
+import org.apache.tez.engine.api.TezProcessorContext;
 import org.apache.tez.engine.lib.output.OnFileSortedOutput;
-import org.apache.tez.engine.newapi.Event;
-import org.apache.tez.engine.newapi.LogicalIOProcessor;
-import org.apache.tez.engine.newapi.LogicalInput;
-import org.apache.tez.engine.newapi.LogicalOutput;
-import org.apache.tez.engine.newapi.TezProcessorContext;
 import org.apache.tez.mapreduce.hadoop.mapreduce.MapContextImpl;
 import org.apache.tez.mapreduce.input.SimpleInput;
 import org.apache.tez.mapreduce.input.SimpleInputLegacy;

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/reduce/ReduceProcessor.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/reduce/ReduceProcessor.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/reduce/ReduceProcessor.java
index b7288e8..9210187 100644
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/reduce/ReduceProcessor.java
+++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/reduce/ReduceProcessor.java
@@ -37,16 +37,16 @@ import org.apache.hadoop.util.Progressable;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.tez.common.counters.TaskCounter;
 import org.apache.tez.common.counters.TezCounter;
+import org.apache.tez.engine.api.Event;
 import org.apache.tez.engine.api.KVReader;
 import org.apache.tez.engine.api.KVWriter;
+import org.apache.tez.engine.api.LogicalIOProcessor;
+import org.apache.tez.engine.api.LogicalInput;
+import org.apache.tez.engine.api.LogicalOutput;
+import org.apache.tez.engine.api.TezProcessorContext;
 import org.apache.tez.engine.common.ConfigUtils;
 import org.apache.tez.engine.common.sort.impl.TezRawKeyValueIterator;
 import org.apache.tez.engine.lib.output.OnFileSortedOutput;
-import org.apache.tez.engine.newapi.Event;
-import org.apache.tez.engine.newapi.LogicalIOProcessor;
-import org.apache.tez.engine.newapi.LogicalInput;
-import org.apache.tez.engine.newapi.LogicalOutput;
-import org.apache.tez.engine.newapi.TezProcessorContext;
 import org.apache.tez.mapreduce.input.ShuffledMergedInputLegacy;
 import org.apache.tez.mapreduce.output.SimpleOutput;
 import org.apache.tez.mapreduce.processor.MRTask;

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/TestUmbilicalProtocol.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/TestUmbilicalProtocol.java b/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/TestUmbilicalProtocol.java
index 404dd8c..d5823f7 100644
--- a/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/TestUmbilicalProtocol.java
+++ b/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/TestUmbilicalProtocol.java
@@ -30,8 +30,6 @@ import org.apache.tez.common.records.ProceedToCompletionResponse;
 import org.apache.tez.dag.records.TezTaskAttemptID;
 import org.apache.tez.engine.api.impl.TezHeartbeatRequest;
 import org.apache.tez.engine.api.impl.TezHeartbeatResponse;
-import org.apache.tez.engine.records.OutputContext;
-import org.apache.tez.engine.records.TezTaskDependencyCompletionEventsUpdate;
 
 public class TestUmbilicalProtocol implements TezTaskUmbilicalProtocol {
 
@@ -65,14 +63,6 @@ public class TestUmbilicalProtocol implements TezTaskUmbilicalProtocol {
   }
 
   @Override
-  public TezTaskDependencyCompletionEventsUpdate getDependentTasksCompletionEvents(
-      int fromEventIdx, int maxEventsToFetch,
-      TezTaskAttemptID reduce) {
-    // TODO Auto-generated method stub
-    return null;
-  }
-
-  @Override
   public ContainerTask getTask(ContainerContext containerContext)
       throws IOException {
     // TODO Auto-generated method stub
@@ -86,13 +76,6 @@ public class TestUmbilicalProtocol implements TezTaskUmbilicalProtocol {
   }
 
   @Override
-  public void outputReady(TezTaskAttemptID taskAttemptId,
-      OutputContext outputContext) throws IOException {
-    // TODO Auto-generated method stub
-
-  }
-
-  @Override
   public ProceedToCompletionResponse proceedToCompletion(
       TezTaskAttemptID taskAttemptId) throws IOException {
     return proceedToCompletionResponse;

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-yarn-client/pom.xml
----------------------------------------------------------------------
diff --git a/tez-yarn-client/pom.xml b/tez-yarn-client/pom.xml
index b142701..402126e 100644
--- a/tez-yarn-client/pom.xml
+++ b/tez-yarn-client/pom.xml
@@ -47,7 +47,7 @@
     </dependency>
     <dependency>
       <groupId>org.apache.tez</groupId>
-      <artifactId>tez-dag-api</artifactId>
+      <artifactId>tez-api</artifactId>
     </dependency>
     <dependency>
       <groupId>org.apache.tez</groupId>


[14/50] [abbrv] Rename *.new* packages back to what they should be, remove dead code from the old packages - mapreduce module - tez-engine module (part of TEZ-398). (sseth)

Posted by ss...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezInputContextImpl.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezInputContextImpl.java b/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezInputContextImpl.java
new file mode 100644
index 0000000..2e10a93
--- /dev/null
+++ b/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezInputContextImpl.java
@@ -0,0 +1,84 @@
+/**
+ * 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.engine.api.impl;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.tez.common.counters.TezCounters;
+import org.apache.tez.dag.records.TezTaskAttemptID;
+import org.apache.tez.engine.api.impl.EventMetaData.EventProducerConsumerType;
+import org.apache.tez.engine.newapi.Event;
+import org.apache.tez.engine.newapi.TezInputContext;
+import org.apache.tez.engine.newruntime.RuntimeTask;
+
+public class TezInputContextImpl extends TezTaskContextImpl
+    implements TezInputContext {
+
+  private final byte[] userPayload;
+  private final String sourceVertexName;
+  private final EventMetaData sourceInfo;
+
+  @Private
+  public TezInputContextImpl(Configuration conf, int appAttemptNumber,
+      TezUmbilical tezUmbilical, String taskVertexName,
+      String sourceVertexName, TezTaskAttemptID taskAttemptID,
+      TezCounters counters, byte[] userPayload,
+      RuntimeTask runtimeTask, Map<String, ByteBuffer> serviceConsumerMetadata) {
+    super(conf, appAttemptNumber, taskVertexName, taskAttemptID,
+        counters, runtimeTask, tezUmbilical, serviceConsumerMetadata);
+    this.userPayload = userPayload;
+    this.sourceVertexName = sourceVertexName;
+    this.sourceInfo = new EventMetaData(
+        EventProducerConsumerType.INPUT, taskVertexName, sourceVertexName,
+        taskAttemptID);
+    this.uniqueIdentifier = String.format("%s_%s_%06d_%02d_%s", taskAttemptID
+        .getTaskID().getVertexID().getDAGId().toString(), taskVertexName,
+        getTaskIndex(), getTaskAttemptNumber(), sourceVertexName);
+  }
+
+  @Override
+  public void sendEvents(List<Event> events) {
+    List<TezEvent> tezEvents = new ArrayList<TezEvent>(events.size());
+    for (Event e : events) {
+      TezEvent tEvt = new TezEvent(e, sourceInfo);
+      tezEvents.add(tEvt);
+    }
+    tezUmbilical.addEvents(tezEvents);
+  }
+
+  @Override
+  public byte[] getUserPayload() {
+    return userPayload;
+  }
+
+  @Override
+  public String getSourceVertexName() {
+    return sourceVertexName;
+  }
+
+  @Override
+  public void fatalError(Throwable exception, String message) {
+    super.signalFatalError(exception, message, sourceInfo);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezOutputContextImpl.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezOutputContextImpl.java b/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezOutputContextImpl.java
new file mode 100644
index 0000000..ef58de2
--- /dev/null
+++ b/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezOutputContextImpl.java
@@ -0,0 +1,85 @@
+/**
+ * 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.engine.api.impl;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.tez.common.counters.TezCounters;
+import org.apache.tez.dag.records.TezTaskAttemptID;
+import org.apache.tez.engine.api.impl.EventMetaData.EventProducerConsumerType;
+import org.apache.tez.engine.newapi.Event;
+import org.apache.tez.engine.newapi.TezOutputContext;
+import org.apache.tez.engine.newruntime.RuntimeTask;
+
+public class TezOutputContextImpl extends TezTaskContextImpl
+    implements TezOutputContext {
+
+  private final byte[] userPayload;
+  private final String destinationVertexName;
+  private final EventMetaData sourceInfo;
+
+  @Private
+  public TezOutputContextImpl(Configuration conf, int appAttemptNumber,
+      TezUmbilical tezUmbilical, String taskVertexName,
+      String destinationVertexName,
+      TezTaskAttemptID taskAttemptID, TezCounters counters,
+      byte[] userPayload, RuntimeTask runtimeTask,
+      Map<String, ByteBuffer> serviceConsumerMetadata) {
+    super(conf, appAttemptNumber, taskVertexName, taskAttemptID,
+        counters, runtimeTask, tezUmbilical, serviceConsumerMetadata);
+    this.userPayload = userPayload;
+    this.destinationVertexName = destinationVertexName;
+    this.sourceInfo = new EventMetaData(EventProducerConsumerType.OUTPUT,
+        taskVertexName, destinationVertexName, taskAttemptID);
+    this.uniqueIdentifier = String.format("%s_%s_%06d_%02d_%s", taskAttemptID
+        .getTaskID().getVertexID().getDAGId().toString(), taskVertexName,
+        getTaskIndex(), getTaskAttemptNumber(), destinationVertexName);
+  }
+
+  @Override
+  public void sendEvents(List<Event> events) {
+    List<TezEvent> tezEvents = new ArrayList<TezEvent>(events.size());
+    for (Event e : events) {
+      TezEvent tEvt = new TezEvent(e, sourceInfo);
+      tezEvents.add(tEvt);
+    }
+    tezUmbilical.addEvents(tezEvents);
+  }
+
+  @Override
+  public byte[] getUserPayload() {
+    return userPayload;
+  }
+
+  @Override
+  public String getDestinationVertexName() {
+    return destinationVertexName;
+  }
+
+  @Override
+  public void fatalError(Throwable exception, String message) {
+    super.signalFatalError(exception, message, sourceInfo);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezProcessorContextImpl.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezProcessorContextImpl.java b/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezProcessorContextImpl.java
new file mode 100644
index 0000000..3f20d5c
--- /dev/null
+++ b/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezProcessorContextImpl.java
@@ -0,0 +1,86 @@
+/**
+ * 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.engine.api.impl;
+
+import java.nio.ByteBuffer;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.tez.common.counters.TezCounters;
+import org.apache.tez.dag.records.TezTaskAttemptID;
+import org.apache.tez.engine.api.impl.EventMetaData.EventProducerConsumerType;
+import org.apache.tez.engine.newapi.Event;
+import org.apache.tez.engine.newapi.TezProcessorContext;
+import org.apache.tez.engine.newruntime.RuntimeTask;
+
+public class TezProcessorContextImpl extends TezTaskContextImpl
+  implements TezProcessorContext {
+
+  private final byte[] userPayload;
+  private final EventMetaData sourceInfo;
+
+  public TezProcessorContextImpl(Configuration conf, int appAttemptNumber,
+      TezUmbilical tezUmbilical, String vertexName,
+      TezTaskAttemptID taskAttemptID, TezCounters counters,
+      byte[] userPayload, RuntimeTask runtimeTask,
+      Map<String, ByteBuffer> serviceConsumerMetadata) {
+    super(conf, appAttemptNumber, vertexName, taskAttemptID,
+        counters, runtimeTask, tezUmbilical, serviceConsumerMetadata);
+    this.userPayload = userPayload;
+    this.sourceInfo = new EventMetaData(EventProducerConsumerType.PROCESSOR,
+        taskVertexName, "", taskAttemptID);
+    this.uniqueIdentifier = String.format("%s_%s_%06d_%02d", taskAttemptID
+        .getTaskID().getVertexID().getDAGId().toString(), taskVertexName,
+        getTaskIndex(), getTaskAttemptNumber());
+  }
+
+  @Override
+  public void sendEvents(List<Event> events) {
+    List<TezEvent> tezEvents = new ArrayList<TezEvent>(events.size());
+    for (Event e : events) {
+      TezEvent tEvt = new TezEvent(e, sourceInfo);
+      tezEvents.add(tEvt);
+    }
+    tezUmbilical.addEvents(tezEvents);
+  }
+
+  @Override
+  public byte[] getUserPayload() {
+    return userPayload;
+  }
+
+  @Override
+  public void setProgress(float progress) {
+    runtimeTask.setProgress(progress);
+  }
+
+  @Override
+  public void fatalError(Throwable exception, String message) {
+    super.signalFatalError(exception, message, sourceInfo);
+  }
+
+  @Override
+  public boolean canCommit() throws IOException {
+    return tezUmbilical.canCommit(this.taskAttemptID);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezTaskContextImpl.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezTaskContextImpl.java b/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezTaskContextImpl.java
new file mode 100644
index 0000000..2312c49
--- /dev/null
+++ b/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezTaskContextImpl.java
@@ -0,0 +1,145 @@
+/**
+ * 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.engine.api.impl;
+
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.Map;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.util.AuxiliaryServiceHelper;
+import org.apache.tez.common.TezJobConfig;
+import org.apache.tez.common.counters.TezCounters;
+import org.apache.tez.dag.records.TezTaskAttemptID;
+import org.apache.tez.engine.newapi.TezTaskContext;
+import org.apache.tez.engine.newruntime.RuntimeTask;
+
+public abstract class TezTaskContextImpl implements TezTaskContext {
+
+  private final Configuration conf;
+  protected final String taskVertexName;
+  protected final TezTaskAttemptID taskAttemptID;
+  private final TezCounters counters;
+  private String[] workDirs;
+  protected String uniqueIdentifier;
+  protected final RuntimeTask runtimeTask;
+  protected final TezUmbilical tezUmbilical;
+  private final Map<String, ByteBuffer> serviceConsumerMetadata;
+  private final int appAttemptNumber;
+
+  @Private
+  public TezTaskContextImpl(Configuration conf, int appAttemptNumber,
+      String taskVertexName, TezTaskAttemptID taskAttemptID,
+      TezCounters counters, RuntimeTask runtimeTask,
+      TezUmbilical tezUmbilical, Map<String, ByteBuffer> serviceConsumerMetadata) {
+    this.conf = conf;
+    this.taskVertexName = taskVertexName;
+    this.taskAttemptID = taskAttemptID;
+    this.counters = counters;
+    // TODO Maybe change this to be task id specific at some point. For now
+    // Shuffle code relies on this being a path specified by YARN
+    this.workDirs = this.conf.getStrings(TezJobConfig.LOCAL_DIRS);
+    this.runtimeTask = runtimeTask;
+    this.tezUmbilical = tezUmbilical;
+    this.serviceConsumerMetadata = serviceConsumerMetadata;
+    // TODO NEWTEZ at some point dag attempt should not map to app attempt
+    this.appAttemptNumber = appAttemptNumber;
+  }
+
+  @Override
+  public ApplicationId getApplicationId() {
+    return taskAttemptID.getTaskID().getVertexID().getDAGId()
+        .getApplicationId();
+  }
+
+  @Override
+  public int getTaskIndex() {
+    return taskAttemptID.getTaskID().getId();
+  }
+
+  @Override
+  public int getDAGAttemptNumber() {
+    return appAttemptNumber;
+  }
+
+  @Override
+  public int getTaskAttemptNumber() {
+    return taskAttemptID.getId();
+  }
+
+  @Override
+  public String getDAGName() {
+    // TODO NEWTEZ Change to some form of the DAG name, for now using dagId as
+    // the unique identifier.
+    return taskAttemptID.getTaskID().getVertexID().getDAGId().toString();
+  }
+
+  @Override
+  public String getTaskVertexName() {
+    return taskVertexName;
+  }
+
+
+  @Override
+  public TezCounters getCounters() {
+    return counters;
+  }
+
+  @Override
+  public String[] getWorkDirs() {
+    return Arrays.copyOf(workDirs, workDirs.length);
+  }
+
+  @Override
+  public String getUniqueIdentifier() {
+    return uniqueIdentifier;
+  }
+
+  @Override
+  public ByteBuffer getServiceConsumerMetaData(String serviceName) {
+    return (ByteBuffer) serviceConsumerMetadata.get(serviceName)
+        .asReadOnlyBuffer().rewind();
+  }
+
+  @Override
+  public ByteBuffer getServiceProviderMetaData(String serviceName) {
+    return AuxiliaryServiceHelper.getServiceDataFromEnv(
+        serviceName, System.getenv());
+  }
+
+  protected void signalFatalError(Throwable t, String message,
+      EventMetaData sourceInfo) {
+    runtimeTask.setFatalError(t, message);
+    String diagnostics;
+    if (t != null && message != null) {
+      diagnostics = "exceptionThrown=" + StringUtils.stringifyException(t)
+          + ", errorMessage=" + message;
+    } else if (t == null && message == null) {
+      diagnostics = "Unknown error";
+    } else {
+      diagnostics = t != null ?
+          "exceptionThrown=" + StringUtils.stringifyException(t)
+          : " errorMessage=" + message;
+    }
+    tezUmbilical.signalFatalError(taskAttemptID, diagnostics, sourceInfo);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezUmbilical.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezUmbilical.java b/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezUmbilical.java
new file mode 100644
index 0000000..925d87b
--- /dev/null
+++ b/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezUmbilical.java
@@ -0,0 +1,36 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.engine.api.impl;
+
+import java.io.IOException;
+import java.util.Collection;
+
+import org.apache.tez.dag.records.TezTaskAttemptID;
+
+public interface TezUmbilical {
+
+  public void addEvents(Collection<TezEvent> events);
+
+  public void signalFatalError(TezTaskAttemptID taskAttemptID,
+      String diagnostics,
+      EventMetaData sourceInfo);
+
+  public boolean canCommit(TezTaskAttemptID taskAttemptID) throws IOException;
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/broadcast/input/BroadcastInputManager.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/broadcast/input/BroadcastInputManager.java b/tez-engine/src/main/java/org/apache/tez/engine/broadcast/input/BroadcastInputManager.java
index 78d2e0c..927f0ad 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/broadcast/input/BroadcastInputManager.java
+++ b/tez-engine/src/main/java/org/apache/tez/engine/broadcast/input/BroadcastInputManager.java
@@ -26,7 +26,7 @@ import org.apache.tez.common.Constants;
 import org.apache.tez.common.TezJobConfig;
 import org.apache.tez.dag.api.TezUncheckedException;
 import org.apache.tez.engine.common.InputAttemptIdentifier;
-import org.apache.tez.engine.common.task.local.newoutput.TezTaskOutputFiles;
+import org.apache.tez.engine.common.task.local.output.TezTaskOutputFiles;
 import org.apache.tez.engine.newapi.TezInputContext;
 import org.apache.tez.engine.shuffle.common.DiskFetchedInput;
 import org.apache.tez.engine.shuffle.common.FetchedInput;

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/broadcast/input/BroadcastKVReader.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/broadcast/input/BroadcastKVReader.java b/tez-engine/src/main/java/org/apache/tez/engine/broadcast/input/BroadcastKVReader.java
index b36c240..0b86a8e 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/broadcast/input/BroadcastKVReader.java
+++ b/tez-engine/src/main/java/org/apache/tez/engine/broadcast/input/BroadcastKVReader.java
@@ -30,10 +30,10 @@ import org.apache.hadoop.io.compress.DefaultCodec;
 import org.apache.hadoop.io.serializer.Deserializer;
 import org.apache.hadoop.io.serializer.SerializationFactory;
 import org.apache.hadoop.util.ReflectionUtils;
+import org.apache.tez.engine.api.KVReader;
 import org.apache.tez.engine.common.ConfigUtils;
 import org.apache.tez.engine.common.shuffle.impl.InMemoryReader;
 import org.apache.tez.engine.common.sort.impl.IFile;
-import org.apache.tez.engine.newapi.KVReader;
 import org.apache.tez.engine.shuffle.common.FetchedInput;
 import org.apache.tez.engine.shuffle.common.FetchedInput.Type;
 import org.apache.tez.engine.shuffle.common.MemoryFetchedInput;

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/broadcast/output/FileBasedKVWriter.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/broadcast/output/FileBasedKVWriter.java b/tez-engine/src/main/java/org/apache/tez/engine/broadcast/output/FileBasedKVWriter.java
index 9f3dbbe..84ddd28 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/broadcast/output/FileBasedKVWriter.java
+++ b/tez-engine/src/main/java/org/apache/tez/engine/broadcast/output/FileBasedKVWriter.java
@@ -29,13 +29,13 @@ import org.apache.hadoop.io.compress.DefaultCodec;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.tez.common.TezJobConfig;
 import org.apache.tez.common.TezUtils;
+import org.apache.tez.engine.api.KVWriter;
 import org.apache.tez.engine.common.ConfigUtils;
 import org.apache.tez.engine.common.TezEngineUtils;
 import org.apache.tez.engine.common.sort.impl.IFile;
 import org.apache.tez.engine.common.sort.impl.TezIndexRecord;
 import org.apache.tez.engine.common.sort.impl.TezSpillRecord;
-import org.apache.tez.engine.common.task.local.newoutput.TezTaskOutput;
-import org.apache.tez.engine.newapi.KVWriter;
+import org.apache.tez.engine.common.task.local.output.TezTaskOutput;
 import org.apache.tez.engine.newapi.TezOutputContext;
 
 public class FileBasedKVWriter implements KVWriter {

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/common/TezEngineUtils.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/TezEngineUtils.java b/tez-engine/src/main/java/org/apache/tez/engine/common/TezEngineUtils.java
index 3920ce6..ab78d82 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/TezEngineUtils.java
+++ b/tez-engine/src/main/java/org/apache/tez/engine/common/TezEngineUtils.java
@@ -30,8 +30,8 @@ import org.apache.tez.common.TezJobConfig;
 import org.apache.tez.dag.api.TezUncheckedException;
 import org.apache.tez.engine.api.Partitioner;
 import org.apache.tez.engine.common.combine.Combiner;
-import org.apache.tez.engine.common.task.local.newoutput.TezTaskOutput;
-import org.apache.tez.engine.common.task.local.newoutput.TezTaskOutputFiles;
+import org.apache.tez.engine.common.task.local.output.TezTaskOutput;
+import org.apache.tez.engine.common.task.local.output.TezTaskOutputFiles;
 import org.apache.tez.engine.newapi.TezOutputContext;
 import org.apache.tez.engine.newapi.TezTaskContext;
 

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/common/combine/CombineInput.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/combine/CombineInput.java b/tez-engine/src/main/java/org/apache/tez/engine/common/combine/CombineInput.java
deleted file mode 100644
index bf504bb..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/combine/CombineInput.java
+++ /dev/null
@@ -1,176 +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.engine.common.combine;
-
-import java.io.IOException;
-import java.util.Iterator;
-import java.util.NoSuchElementException;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.io.BytesWritable;
-import org.apache.hadoop.io.DataInputBuffer;
-import org.apache.hadoop.io.RawComparator;
-import org.apache.hadoop.io.serializer.Deserializer;
-import org.apache.hadoop.util.Progressable;
-import org.apache.tez.common.counters.TezCounter;
-import org.apache.tez.engine.api.Input;
-import org.apache.tez.engine.api.Master;
-import org.apache.tez.engine.common.sort.impl.TezRawKeyValueIterator;
-
-@SuppressWarnings({ "unchecked", "rawtypes" })
-public class CombineInput implements Input {
-
-  private final TezRawKeyValueIterator input;
-  private TezCounter inputValueCounter;
-  private TezCounter inputKeyCounter;
-  private RawComparator<Object> comparator;
-  private Object key;                                  // current key
-  private Object value;                              // current value
-  private boolean firstValue = false;                 // first value in key
-  private boolean nextKeyIsSame = false;              // more w/ this key
-  private boolean hasMore;                            // more in file
-  protected Progressable reporter;
-  private Deserializer keyDeserializer;
-  private Deserializer valueDeserializer;
-  private DataInputBuffer buffer = new DataInputBuffer();
-  private BytesWritable currentRawKey = new BytesWritable();
-  private ValueIterable iterable = new ValueIterable();
-  
-  public CombineInput(TezRawKeyValueIterator kvIter) {
-    this.input = kvIter;
-  }
-
-  public void initialize(Configuration conf, Master master) throws IOException,
-      InterruptedException {
-  }
-
-  public boolean hasNext() throws IOException, InterruptedException {
-    while (hasMore && nextKeyIsSame) {
-      nextKeyValue();
-    }
-    if (hasMore) {
-      if (inputKeyCounter != null) {
-        inputKeyCounter.increment(1);
-      }
-      return nextKeyValue();
-    } else {
-      return false;
-    }
-  }
-
-  private boolean nextKeyValue() throws IOException, InterruptedException {
-    if (!hasMore) {
-      key = null;
-      value = null;
-      return false;
-    }
-    firstValue = !nextKeyIsSame;
-    DataInputBuffer nextKey = input.getKey();
-    currentRawKey.set(nextKey.getData(), nextKey.getPosition(), 
-                      nextKey.getLength() - nextKey.getPosition());
-    buffer.reset(currentRawKey.getBytes(), 0, currentRawKey.getLength());
-    key = keyDeserializer.deserialize(key);
-    DataInputBuffer nextVal = input.getValue();
-    buffer.reset(nextVal.getData(), nextVal.getPosition(), nextVal.getLength());
-    value = valueDeserializer.deserialize(value);
-
-    hasMore = input.next();
-    if (hasMore) {
-      nextKey = input.getKey();
-      nextKeyIsSame = comparator.compare(currentRawKey.getBytes(), 0, 
-                                     currentRawKey.getLength(),
-                                     nextKey.getData(),
-                                     nextKey.getPosition(),
-                                     nextKey.getLength() - nextKey.getPosition()
-                                         ) == 0;
-    } else {
-      nextKeyIsSame = false;
-    }
-    inputValueCounter.increment(1);
-    return true;
-  }
-
-  public Object getNextKey() throws IOException, InterruptedException {
-    return key;
-  }
-
-  public Iterable getNextValues() throws IOException,
-      InterruptedException {
-    return iterable;
-  }
-
-  public float getProgress() throws IOException, InterruptedException {
-    return input.getProgress().getProgress();
-  }
-
-  public void close() throws IOException {
-    input.close();
-  }
-
-  public TezRawKeyValueIterator getIterator() {
-    return this.input;
-  }
-
-  protected class ValueIterator implements Iterator<Object> {
-
-
-    public boolean hasNext() {
-      return firstValue || nextKeyIsSame;
-    }
-
-    public Object next() {
-
-      // if this is the first record, we don't need to advance
-      if (firstValue) {
-        firstValue = false;
-        return value;
-      }
-      // if this isn't the first record and the next key is different, they
-      // can't advance it here.
-      if (!nextKeyIsSame) {
-        throw new NoSuchElementException("iterate past last value");
-      }
-      // otherwise, go to the next key/value pair
-      try {
-        nextKeyValue();
-        return value;
-      } catch (IOException ie) {
-        throw new RuntimeException("next value iterator failed", ie);
-      } catch (InterruptedException ie) {
-        // this is bad, but we can't modify the exception list of java.util
-        throw new RuntimeException("next value iterator interrupted", ie);        
-      }
-    }
-
-    public void remove() {
-      throw new UnsupportedOperationException("remove not implemented");
-    }
-  }
-
-
-  
-  protected class ValueIterable implements Iterable<Object> {
-    private ValueIterator iterator = new ValueIterator();
-    public Iterator<Object> iterator() {
-      return iterator;
-    } 
-  }
-  
-
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/common/combine/CombineOutput.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/combine/CombineOutput.java b/tez-engine/src/main/java/org/apache/tez/engine/common/combine/CombineOutput.java
deleted file mode 100644
index 10a1b90..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/combine/CombineOutput.java
+++ /dev/null
@@ -1,55 +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.engine.common.combine;
-
-import java.io.IOException;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.tez.engine.api.Master;
-import org.apache.tez.engine.api.Output;
-import org.apache.tez.engine.common.sort.impl.IFile.Writer;
-import org.apache.tez.engine.records.OutputContext;
-
-public class CombineOutput implements Output {
-
-  private final Writer writer;
-  
-  public CombineOutput(Writer writer) {
-    this.writer = writer;
-  }
-
-  public void initialize(Configuration conf, Master master) throws IOException,
-      InterruptedException {
-    // TODO Auto-generated method stub
-
-  }
-
-  public void write(Object key, Object value) throws IOException,
-      InterruptedException {
-    writer.append(key, value);
-  }
-
-  @Override
-  public OutputContext getOutputContext() {
-    return null;
-  }
-  
-  public void close() throws IOException, InterruptedException {
-    writer.close();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/common/localshuffle/LocalShuffle.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/localshuffle/LocalShuffle.java b/tez-engine/src/main/java/org/apache/tez/engine/common/localshuffle/LocalShuffle.java
index 38b04d3..1cb89a7 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/localshuffle/LocalShuffle.java
+++ b/tez-engine/src/main/java/org/apache/tez/engine/common/localshuffle/LocalShuffle.java
@@ -35,8 +35,8 @@ import org.apache.tez.common.counters.TezCounter;
 import org.apache.tez.engine.common.ConfigUtils;
 import org.apache.tez.engine.common.sort.impl.TezMerger;
 import org.apache.tez.engine.common.sort.impl.TezRawKeyValueIterator;
-import org.apache.tez.engine.common.task.local.newoutput.TezLocalTaskOutputFiles;
-import org.apache.tez.engine.common.task.local.newoutput.TezTaskOutput;
+import org.apache.tez.engine.common.task.local.output.TezLocalTaskOutputFiles;
+import org.apache.tez.engine.common.task.local.output.TezTaskOutput;
 import org.apache.tez.engine.newapi.TezInputContext;
 
 @SuppressWarnings({"rawtypes"})

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/MapOutput.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/MapOutput.java b/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/MapOutput.java
index 46851c7..b2a0b54 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/MapOutput.java
+++ b/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/MapOutput.java
@@ -30,7 +30,7 @@ import org.apache.hadoop.fs.LocalDirAllocator;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.BoundedByteArrayOutputStream;
 import org.apache.tez.engine.common.InputAttemptIdentifier;
-import org.apache.tez.engine.common.task.local.newoutput.TezTaskOutputFiles;
+import org.apache.tez.engine.common.task.local.output.TezTaskOutputFiles;
 
 
 class MapOutput {

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/MergeManager.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/MergeManager.java b/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/MergeManager.java
index ad9bb5f..bf2be4e 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/MergeManager.java
+++ b/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/MergeManager.java
@@ -54,7 +54,7 @@ import org.apache.tez.engine.common.sort.impl.TezMerger;
 import org.apache.tez.engine.common.sort.impl.TezRawKeyValueIterator;
 import org.apache.tez.engine.common.sort.impl.IFile.Writer;
 import org.apache.tez.engine.common.sort.impl.TezMerger.Segment;
-import org.apache.tez.engine.common.task.local.newoutput.TezTaskOutputFiles;
+import org.apache.tez.engine.common.task.local.output.TezTaskOutputFiles;
 import org.apache.tez.engine.hadoop.compat.NullProgressable;
 import org.apache.tez.engine.newapi.TezInputContext;
 

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/common/sort/SortingOutput.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/sort/SortingOutput.java b/tez-engine/src/main/java/org/apache/tez/engine/common/sort/SortingOutput.java
deleted file mode 100644
index 35d7723..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/sort/SortingOutput.java
+++ /dev/null
@@ -1,32 +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.engine.common.sort;
-
-import org.apache.tez.common.RunningTaskContext;
-import org.apache.tez.engine.api.Output;
-
-/**
- * {@link SortingOutput} is an {@link Output} which sorts incoming key/value
- * pairs.
- */
-public interface SortingOutput extends Output {
-  
-  // TODO PreCommit rename
-  public void setTask(RunningTaskContext runningTaskContext);
-  
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/ExternalSorter.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/ExternalSorter.java b/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/ExternalSorter.java
index 1b5e015..8b4bd4e 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/ExternalSorter.java
+++ b/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/ExternalSorter.java
@@ -48,7 +48,7 @@ import org.apache.tez.engine.common.TezEngineUtils;
 import org.apache.tez.engine.common.combine.Combiner;
 import org.apache.tez.engine.common.shuffle.impl.ShuffleHeader;
 import org.apache.tez.engine.common.sort.impl.IFile.Writer;
-import org.apache.tez.engine.common.task.local.newoutput.TezTaskOutput;
+import org.apache.tez.engine.common.task.local.output.TezTaskOutput;
 import org.apache.tez.engine.hadoop.compat.NullProgressable;
 import org.apache.tez.engine.newapi.TezOutputContext;
 

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/common/task/impl/CombineValuesIterator.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/task/impl/CombineValuesIterator.java b/tez-engine/src/main/java/org/apache/tez/engine/common/task/impl/CombineValuesIterator.java
deleted file mode 100644
index ae6a371..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/task/impl/CombineValuesIterator.java
+++ /dev/null
@@ -1,51 +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.engine.common.task.impl;
-
-import java.io.IOException;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.io.RawComparator;
-import org.apache.tez.common.TezTaskReporter;
-import org.apache.tez.common.counters.TezCounter;
-import org.apache.tez.engine.common.sort.impl.TezRawKeyValueIterator;
-
-/** Iterator to return Combined values */
-@InterfaceAudience.Private
-@InterfaceStability.Unstable
-public class CombineValuesIterator<KEY,VALUE>
-extends ValuesIterator<KEY,VALUE> {
-
-  private final TezCounter combineInputCounter;
-
-  public CombineValuesIterator(TezRawKeyValueIterator in,
-      RawComparator<KEY> comparator, Class<KEY> keyClass,
-      Class<VALUE> valClass, Configuration conf, TezTaskReporter reporter,
-      TezCounter combineInputCounter) throws IOException {
-    super(in, comparator, keyClass, valClass, conf, reporter);
-    this.combineInputCounter = combineInputCounter;
-  }
-
-  public VALUE next() {
-    combineInputCounter.increment(1);
-    return super.next();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/common/task/local/newoutput/TezLocalTaskOutputFiles.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/task/local/newoutput/TezLocalTaskOutputFiles.java b/tez-engine/src/main/java/org/apache/tez/engine/common/task/local/newoutput/TezLocalTaskOutputFiles.java
deleted file mode 100644
index bbe4e34..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/task/local/newoutput/TezLocalTaskOutputFiles.java
+++ /dev/null
@@ -1,249 +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.engine.common.task.local.newoutput;
-
-import java.io.IOException;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-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.tez.common.Constants;
-import org.apache.tez.common.TezJobConfig;
-import org.apache.tez.engine.common.InputAttemptIdentifier;
-
-/**
- * Manipulate the working area for the transient store for maps and reduces.
- *
- * This class is used by map and reduce tasks to identify the directories that
- * they need to write to/read from for intermediate files. The callers of
- * these methods are from the Child running the Task.
- */
-@InterfaceAudience.Private
-@InterfaceStability.Unstable
-public class TezLocalTaskOutputFiles extends TezTaskOutput {
-
-  public TezLocalTaskOutputFiles(Configuration conf, String uniqueId) {
-    super(conf, uniqueId);
-  }
-
-  private LocalDirAllocator lDirAlloc =
-    new LocalDirAllocator(TezJobConfig.LOCAL_DIRS);
-
-
-  /**
-   * Return the path to local map output file created earlier
-   *
-   * @return path
-   * @throws IOException
-   */
-  @Override
-  public Path getOutputFile()
-      throws IOException {
-    return lDirAlloc.getLocalPathToRead(Constants.TASK_OUTPUT_DIR + Path.SEPARATOR
-        + Constants.TEZ_ENGINE_TASK_OUTPUT_FILENAME_STRING, conf);
-  }
-
-  /**
-   * Create a local map output file name.
-   *
-   * @param size the size of the file
-   * @return path
-   * @throws IOException
-   */
-  @Override
-  public Path getOutputFileForWrite(long size)
-      throws IOException {
-    return lDirAlloc.getLocalPathForWrite(Constants.TASK_OUTPUT_DIR + Path.SEPARATOR
-        + Constants.TEZ_ENGINE_TASK_OUTPUT_FILENAME_STRING, size, conf);
-  }
-  
-  /**
-   * Create a local map output file name. This should *only* be used if the size
-   * of the file is not known. Otherwise use the equivalent which accepts a size
-   * parameter.
-   * 
-   * @return path
-   * @throws IOException
-   */
-  @Override
-  public Path getOutputFileForWrite() throws IOException {
-    return lDirAlloc.getLocalPathForWrite(Constants.TASK_OUTPUT_DIR
-        + Path.SEPARATOR + Constants.TEZ_ENGINE_TASK_OUTPUT_FILENAME_STRING,
-        conf);
-  }
-
-  /**
-   * Create a local map output file name on the same volume.
-   */
-  @Override
-  public Path getOutputFileForWriteInVolume(Path existing) {
-    return new Path(existing.getParent(), Constants.TEZ_ENGINE_TASK_OUTPUT_FILENAME_STRING);
-  }
-
-  /**
-   * Return the path to a local map output index file created earlier
-   *
-   * @return path
-   * @throws IOException
-   */
-  @Override
-  public Path getOutputIndexFile()
-      throws IOException {
-    return lDirAlloc.getLocalPathToRead(Constants.TASK_OUTPUT_DIR + Path.SEPARATOR
-        + Constants.TEZ_ENGINE_TASK_OUTPUT_FILENAME_STRING + Constants.TEZ_ENGINE_TASK_OUTPUT_INDEX_SUFFIX_STRING,
-        conf);
-  }
-
-  /**
-   * Create a local map output index file name.
-   *
-   * @param size the size of the file
-   * @return path
-   * @throws IOException
-   */
-  @Override
-  public Path getOutputIndexFileForWrite(long size)
-      throws IOException {
-    return lDirAlloc.getLocalPathForWrite(Constants.TASK_OUTPUT_DIR + Path.SEPARATOR
-        + Constants.TEZ_ENGINE_TASK_OUTPUT_FILENAME_STRING + Constants.TEZ_ENGINE_TASK_OUTPUT_INDEX_SUFFIX_STRING,
-        size, conf);
-  }
-
-  /**
-   * Create a local map output index file name on the same volume.
-   */
-  @Override
-  public Path getOutputIndexFileForWriteInVolume(Path existing) {
-    return new Path(existing.getParent(),
-        Constants.TEZ_ENGINE_TASK_OUTPUT_FILENAME_STRING + Constants.TEZ_ENGINE_TASK_OUTPUT_INDEX_SUFFIX_STRING);
-  }
-
-  /**
-   * Return a local map spill file created earlier.
-   *
-   * @param spillNumber the number
-   * @return path
-   * @throws IOException
-   */
-  @Override
-  public Path getSpillFile(int spillNumber)
-      throws IOException {
-    return lDirAlloc.getLocalPathToRead(Constants.TASK_OUTPUT_DIR + "/spill"
-        + spillNumber + ".out", conf);
-  }
-
-  /**
-   * Create a local map spill file name.
-   *
-   * @param spillNumber the number
-   * @param size the size of the file
-   * @return path
-   * @throws IOException
-   */
-  @Override
-  public Path getSpillFileForWrite(int spillNumber, long size)
-      throws IOException {
-    return lDirAlloc.getLocalPathForWrite(Constants.TASK_OUTPUT_DIR + "/spill"
-        + spillNumber + ".out", size, conf);
-  }
-
-  /**
-   * Return a local map spill index file created earlier
-   *
-   * @param spillNumber the number
-   * @return path
-   * @throws IOException
-   */
-  @Override
-  public Path getSpillIndexFile(int spillNumber)
-      throws IOException {
-    return lDirAlloc.getLocalPathToRead(Constants.TASK_OUTPUT_DIR + "/spill"
-        + spillNumber + ".out.index", conf);
-  }
-
-  /**
-   * Create a local map spill index file name.
-   *
-   * @param spillNumber the number
-   * @param size the size of the file
-   * @return path
-   * @throws IOException
-   */
-  @Override
-  public Path getSpillIndexFileForWrite(int spillNumber, long size)
-      throws IOException {
-    return lDirAlloc.getLocalPathForWrite(Constants.TASK_OUTPUT_DIR + "/spill"
-        + spillNumber + ".out.index", size, conf);
-  }
-
-  /**
-   * Return a local reduce input file created earlier
-   *
-   * @param mapId a map task id
-   * @return path
-   * @throws IOException
-   */
-  @Override
-  public Path getInputFile(InputAttemptIdentifier mapId)
-      throws IOException {
-    return lDirAlloc.getLocalPathToRead(String.format(
-        Constants.TEZ_ENGINE_TASK_INPUT_FILE_FORMAT_STRING, 
-        Constants.TASK_OUTPUT_DIR, Integer.valueOf(mapId.getInputIdentifier().getSrcTaskIndex())), conf);
-  }
-
-  /**
-   * Create a local reduce input file name.
-   *
-   * @param mapId a map task id
-   * @param size the size of the file
-   * @return path
-   * @throws IOException
-   */
-  @Override
-  public Path getInputFileForWrite(int taskId,
-                                   long size)
-      throws IOException {
-    return lDirAlloc.getLocalPathForWrite(String.format(
-        Constants.TEZ_ENGINE_TASK_INPUT_FILE_FORMAT_STRING, Constants.TASK_OUTPUT_DIR, taskId),
-        size, conf);
-  }
-
-  /** Removes all of the files related to a task. */
-  @Override
-  public void removeAll()
-      throws IOException {
-    deleteLocalFiles(Constants.TASK_OUTPUT_DIR);
-  }
-
-  private String[] getLocalDirs() throws IOException {
-    return conf.getStrings(TezJobConfig.LOCAL_DIRS);
-  }
-
-  @SuppressWarnings("deprecation")
-  private void deleteLocalFiles(String subdir) throws IOException {
-    String[] localDirs = getLocalDirs();
-    for (int i = 0; i < localDirs.length; i++) {
-      FileSystem.getLocal(conf).delete(new Path(localDirs[i], subdir));
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/common/task/local/newoutput/TezTaskOutput.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/task/local/newoutput/TezTaskOutput.java b/tez-engine/src/main/java/org/apache/tez/engine/common/task/local/newoutput/TezTaskOutput.java
deleted file mode 100644
index 87a5aec..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/task/local/newoutput/TezTaskOutput.java
+++ /dev/null
@@ -1,165 +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.engine.common.task.local.newoutput;
-
-import java.io.IOException;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.Path;
-import org.apache.tez.engine.common.InputAttemptIdentifier;
-
-/**
- * Manipulate the working area for the transient store for maps and reduces.
- *
- * This class is used by map and reduce tasks to identify the directories that
- * they need to write to/read from for intermediate files. The callers of
- * these methods are from child space and see mapreduce.cluster.local.dir as
- * taskTracker/jobCache/jobId/attemptId
- * This class should not be used from TaskTracker space.
- */
-@InterfaceAudience.Private
-@InterfaceStability.Unstable
-public abstract class TezTaskOutput {
-
-  protected Configuration conf;
-  protected String uniqueId;
-
-  public TezTaskOutput(Configuration conf, String uniqueId) {
-    this.conf = conf;
-    this.uniqueId = uniqueId;
-  }
-
-  /**
-   * Return the path to local map output file created earlier
-   *
-   * @return path
-   * @throws IOException
-   */
-  public abstract Path getOutputFile() throws IOException;
-
-  /**
-   * Create a local map output file name.
-   *
-   * @param size the size of the file
-   * @return path
-   * @throws IOException
-   */
-  public abstract Path getOutputFileForWrite(long size) throws IOException;
-
-  /**
-   * Create a local output file name. This method is meant to be used *only* if
-   * the size of the file is not know up front.
-   * 
-   * @return path
-   * @throws IOException
-   */
-  public abstract Path getOutputFileForWrite() throws IOException;
-  
-  /**
-   * Create a local map output file name on the same volume.
-   */
-  public abstract Path getOutputFileForWriteInVolume(Path existing);
-
-  /**
-   * Return the path to a local map output index file created earlier
-   *
-   * @return path
-   * @throws IOException
-   */
-  public abstract Path getOutputIndexFile() throws IOException;
-
-  /**
-   * Create a local map output index file name.
-   *
-   * @param size the size of the file
-   * @return path
-   * @throws IOException
-   */
-  public abstract Path getOutputIndexFileForWrite(long size) throws IOException;
-
-  /**
-   * Create a local map output index file name on the same volume.
-   */
-  public abstract Path getOutputIndexFileForWriteInVolume(Path existing);
-
-  /**
-   * Return a local map spill file created earlier.
-   *
-   * @param spillNumber the number
-   * @return path
-   * @throws IOException
-   */
-  public abstract Path getSpillFile(int spillNumber) throws IOException;
-
-  /**
-   * Create a local map spill file name.
-   *
-   * @param spillNumber the number
-   * @param size the size of the file
-   * @return path
-   * @throws IOException
-   */
-  public abstract Path getSpillFileForWrite(int spillNumber, long size)
-      throws IOException;
-
-  /**
-   * Return a local map spill index file created earlier
-   *
-   * @param spillNumber the number
-   * @return path
-   * @throws IOException
-   */
-  public abstract Path getSpillIndexFile(int spillNumber) throws IOException;
-
-  /**
-   * Create a local map spill index file name.
-   *
-   * @param spillNumber the number
-   * @param size the size of the file
-   * @return path
-   * @throws IOException
-   */
-  public abstract Path getSpillIndexFileForWrite(int spillNumber, long size)
-      throws IOException;
-
-  /**
-   * Return a local reduce input file created earlier
-   *
-   * @param attemptIdentifier The identifier for the source task
-   * @return path
-   * @throws IOException
-   */
-  public abstract Path getInputFile(InputAttemptIdentifier attemptIdentifier) throws IOException;
-
-  /**
-   * Create a local reduce input file name.
-   *
-   * @param taskIdentifier The identifier for the source task
-   * @param size the size of the file
-   * @return path
-   * @throws IOException
-   */
-  public abstract Path getInputFileForWrite(
-      int taskIdentifier, long size) throws IOException;
-
-  /** Removes all of the files related to a task. */
-  public abstract void removeAll() throws IOException;
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/common/task/local/newoutput/TezTaskOutputFiles.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/task/local/newoutput/TezTaskOutputFiles.java b/tez-engine/src/main/java/org/apache/tez/engine/common/task/local/newoutput/TezTaskOutputFiles.java
deleted file mode 100644
index a37f05f..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/task/local/newoutput/TezTaskOutputFiles.java
+++ /dev/null
@@ -1,246 +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.engine.common.task.local.newoutput;
-
-import java.io.IOException;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.LocalDirAllocator;
-import org.apache.hadoop.fs.Path;
-import org.apache.tez.common.Constants;
-import org.apache.tez.common.TezJobConfig;
-import org.apache.tez.engine.common.InputAttemptIdentifier;
-
-/**
- * Manipulate the working area for the transient store for maps and reduces.
- *
- * This class is used by map and reduce tasks to identify the directories that
- * they need to write to/read from for intermediate files. The callers of
- * these methods are from child space and see mapreduce.cluster.local.dir as
- * taskTracker/jobCache/jobId/attemptId
- * This class should not be used from TaskTracker space.
- */
-
-@InterfaceAudience.Private
-@InterfaceStability.Unstable
-public class TezTaskOutputFiles extends TezTaskOutput {
-  
-  public TezTaskOutputFiles(Configuration conf, String uniqueId) {
-    super(conf, uniqueId);
-  }
-
-  private static final Log LOG = LogFactory.getLog(TezTaskOutputFiles.class);
-
-  private static final String SPILL_FILE_PATTERN = "%s_spill_%d.out";
-  private static final String SPILL_INDEX_FILE_PATTERN = SPILL_FILE_PATTERN
-      + ".index";
-
-  
-
-  // assume configured to $localdir/usercache/$user/appcache/$appId
-  private LocalDirAllocator lDirAlloc =
-    new LocalDirAllocator(TezJobConfig.LOCAL_DIRS);
-  
-
-  private Path getAttemptOutputDir() {
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("getAttemptOutputDir: "
-          + Constants.TASK_OUTPUT_DIR + "/"
-          + uniqueId);
-    }
-    return new Path(Constants.TASK_OUTPUT_DIR, uniqueId);
-  }
-
-  /**
-   * Return the path to local map output file created earlier
-   *
-   * @return path
-   * @throws IOException
-   */
-  public Path getOutputFile() throws IOException {
-    Path attemptOutput =
-      new Path(getAttemptOutputDir(), Constants.TEZ_ENGINE_TASK_OUTPUT_FILENAME_STRING);
-    return lDirAlloc.getLocalPathToRead(attemptOutput.toString(), conf);
-  }
-
-  /**
-   * Create a local map output file name.
-   *
-   * @param size the size of the file
-   * @return path
-   * @throws IOException
-   */
-  public Path getOutputFileForWrite(long size) throws IOException {
-    Path attemptOutput =
-      new Path(getAttemptOutputDir(), Constants.TEZ_ENGINE_TASK_OUTPUT_FILENAME_STRING);
-    return lDirAlloc.getLocalPathForWrite(attemptOutput.toString(), size, conf);
-  }
-
-  /**
-   * Create a local map output file name. This should *only* be used if the size
-   * of the file is not known. Otherwise use the equivalent which accepts a size
-   * parameter.
-   * 
-   * @return path
-   * @throws IOException
-   */
-  public Path getOutputFileForWrite() throws IOException {
-    Path attemptOutput =
-      new Path(getAttemptOutputDir(), Constants.TEZ_ENGINE_TASK_OUTPUT_FILENAME_STRING);
-    return lDirAlloc.getLocalPathForWrite(attemptOutput.toString(), conf);
-  }
-
-  /**
-   * Create a local map output file name on the same volume.
-   */
-  public Path getOutputFileForWriteInVolume(Path existing) {
-    Path outputDir = new Path(existing.getParent(), Constants.TASK_OUTPUT_DIR);
-    Path attemptOutputDir = new Path(outputDir, uniqueId);
-    return new Path(attemptOutputDir, Constants.TEZ_ENGINE_TASK_OUTPUT_FILENAME_STRING);
-  }
-
-  /**
-   * Return the path to a local map output index file created earlier
-   *
-   * @return path
-   * @throws IOException
-   */
-  public Path getOutputIndexFile() throws IOException {
-    Path attemptIndexOutput =
-      new Path(getAttemptOutputDir(), Constants.TEZ_ENGINE_TASK_OUTPUT_FILENAME_STRING +
-                                      Constants.TEZ_ENGINE_TASK_OUTPUT_INDEX_SUFFIX_STRING);
-    return lDirAlloc.getLocalPathToRead(attemptIndexOutput.toString(), conf);
-  }
-
-  /**
-   * Create a local map output index file name.
-   *
-   * @param size the size of the file
-   * @return path
-   * @throws IOException
-   */
-  public Path getOutputIndexFileForWrite(long size) throws IOException {
-    Path attemptIndexOutput =
-      new Path(getAttemptOutputDir(), Constants.TEZ_ENGINE_TASK_OUTPUT_FILENAME_STRING +
-                                      Constants.TEZ_ENGINE_TASK_OUTPUT_INDEX_SUFFIX_STRING);
-    return lDirAlloc.getLocalPathForWrite(attemptIndexOutput.toString(),
-        size, conf);
-  }
-
-  /**
-   * Create a local map output index file name on the same volume.
-   */
-  public Path getOutputIndexFileForWriteInVolume(Path existing) {
-    Path outputDir = new Path(existing.getParent(), Constants.TASK_OUTPUT_DIR);
-    Path attemptOutputDir = new Path(outputDir, uniqueId);
-    return new Path(attemptOutputDir, Constants.TEZ_ENGINE_TASK_OUTPUT_FILENAME_STRING +
-                                      Constants.TEZ_ENGINE_TASK_OUTPUT_INDEX_SUFFIX_STRING);
-  }
-
-  /**
-   * Return a local map spill file created earlier.
-   *
-   * @param spillNumber the number
-   * @return path
-   * @throws IOException
-   */
-  public Path getSpillFile(int spillNumber) throws IOException {
-    return lDirAlloc.getLocalPathToRead(
-        String.format(SPILL_FILE_PATTERN,
-            uniqueId, spillNumber), conf);
-  }
-
-  /**
-   * Create a local map spill file name.
-   *
-   * @param spillNumber the number
-   * @param size the size of the file
-   * @return path
-   * @throws IOException
-   */
-  public Path getSpillFileForWrite(int spillNumber, long size)
-      throws IOException {
-    return lDirAlloc.getLocalPathForWrite(
-        String.format(String.format(SPILL_FILE_PATTERN,
-            uniqueId, spillNumber)), size, conf);
-  }
-
-  /**
-   * Return a local map spill index file created earlier
-   *
-   * @param spillNumber the number
-   * @return path
-   * @throws IOException
-   */
-  public Path getSpillIndexFile(int spillNumber) throws IOException {
-    return lDirAlloc.getLocalPathToRead(
-        String.format(SPILL_INDEX_FILE_PATTERN,
-            uniqueId, spillNumber), conf);
-  }
-
-  /**
-   * Create a local map spill index file name.
-   *
-   * @param spillNumber the number
-   * @param size the size of the file
-   * @return path
-   * @throws IOException
-   */
-  public Path getSpillIndexFileForWrite(int spillNumber, long size)
-      throws IOException {
-    return lDirAlloc.getLocalPathForWrite(
-        String.format(SPILL_INDEX_FILE_PATTERN,
-            uniqueId, spillNumber), size, conf);
-  }
-
-  /**
-   * Return a local reduce input file created earlier
-   *
-   * @param attemptIdentifier an identifier for a task. The attempt information is ignored.
-   * @return path
-   * @throws IOException
-   */
-  public Path getInputFile(InputAttemptIdentifier attemptIdentifier) throws IOException {
-    throw new UnsupportedOperationException("Incompatible with LocalRunner");
-  }
-
-  /**
-   * Create a local reduce input file name.
-   *
-   * @param attemptIdentifier an identifier for a task. The attempt information is ignored.
-   * @param size the size of the file
-   * @return path
-   * @throws IOException
-   */
-  public Path getInputFileForWrite(int srcTaskId,
-      long size) throws IOException {
-    return lDirAlloc.getLocalPathForWrite(String.format(
-        uniqueId, getAttemptOutputDir().toString(), srcTaskId),
-        size, conf);
-  }
-
-  /** Removes all of the files related to a task. */
-  public void removeAll() throws IOException {
-    throw new UnsupportedOperationException("Incompatible with LocalRunner");
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/common/task/local/output/TezLocalTaskOutputFiles.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/task/local/output/TezLocalTaskOutputFiles.java b/tez-engine/src/main/java/org/apache/tez/engine/common/task/local/output/TezLocalTaskOutputFiles.java
index 69484af..40e6b1a 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/task/local/output/TezLocalTaskOutputFiles.java
+++ b/tez-engine/src/main/java/org/apache/tez/engine/common/task/local/output/TezLocalTaskOutputFiles.java
@@ -28,7 +28,7 @@ import org.apache.hadoop.fs.LocalDirAllocator;
 import org.apache.hadoop.fs.Path;
 import org.apache.tez.common.Constants;
 import org.apache.tez.common.TezJobConfig;
-import org.apache.tez.dag.records.TezTaskID;
+import org.apache.tez.engine.common.InputAttemptIdentifier;
 
 /**
  * Manipulate the working area for the transient store for maps and reduces.
@@ -41,11 +41,13 @@ import org.apache.tez.dag.records.TezTaskID;
 @InterfaceStability.Unstable
 public class TezLocalTaskOutputFiles extends TezTaskOutput {
 
+  public TezLocalTaskOutputFiles(Configuration conf, String uniqueId) {
+    super(conf, uniqueId);
+  }
+
   private LocalDirAllocator lDirAlloc =
     new LocalDirAllocator(TezJobConfig.LOCAL_DIRS);
 
-  public TezLocalTaskOutputFiles() {
-  }
 
   /**
    * Return the path to local map output file created earlier
@@ -57,7 +59,7 @@ public class TezLocalTaskOutputFiles extends TezTaskOutput {
   public Path getOutputFile()
       throws IOException {
     return lDirAlloc.getLocalPathToRead(Constants.TASK_OUTPUT_DIR + Path.SEPARATOR
-        + Constants.TEZ_ENGINE_TASK_OUTPUT_FILENAME_STRING, getConf());
+        + Constants.TEZ_ENGINE_TASK_OUTPUT_FILENAME_STRING, conf);
   }
 
   /**
@@ -71,7 +73,22 @@ public class TezLocalTaskOutputFiles extends TezTaskOutput {
   public Path getOutputFileForWrite(long size)
       throws IOException {
     return lDirAlloc.getLocalPathForWrite(Constants.TASK_OUTPUT_DIR + Path.SEPARATOR
-        + Constants.TEZ_ENGINE_TASK_OUTPUT_FILENAME_STRING, size, getConf());
+        + Constants.TEZ_ENGINE_TASK_OUTPUT_FILENAME_STRING, size, conf);
+  }
+  
+  /**
+   * Create a local map output file name. This should *only* be used if the size
+   * of the file is not known. Otherwise use the equivalent which accepts a size
+   * parameter.
+   * 
+   * @return path
+   * @throws IOException
+   */
+  @Override
+  public Path getOutputFileForWrite() throws IOException {
+    return lDirAlloc.getLocalPathForWrite(Constants.TASK_OUTPUT_DIR
+        + Path.SEPARATOR + Constants.TEZ_ENGINE_TASK_OUTPUT_FILENAME_STRING,
+        conf);
   }
 
   /**
@@ -93,7 +110,7 @@ public class TezLocalTaskOutputFiles extends TezTaskOutput {
       throws IOException {
     return lDirAlloc.getLocalPathToRead(Constants.TASK_OUTPUT_DIR + Path.SEPARATOR
         + Constants.TEZ_ENGINE_TASK_OUTPUT_FILENAME_STRING + Constants.TEZ_ENGINE_TASK_OUTPUT_INDEX_SUFFIX_STRING,
-        getConf());
+        conf);
   }
 
   /**
@@ -108,7 +125,7 @@ public class TezLocalTaskOutputFiles extends TezTaskOutput {
       throws IOException {
     return lDirAlloc.getLocalPathForWrite(Constants.TASK_OUTPUT_DIR + Path.SEPARATOR
         + Constants.TEZ_ENGINE_TASK_OUTPUT_FILENAME_STRING + Constants.TEZ_ENGINE_TASK_OUTPUT_INDEX_SUFFIX_STRING,
-        size, getConf());
+        size, conf);
   }
 
   /**
@@ -131,7 +148,7 @@ public class TezLocalTaskOutputFiles extends TezTaskOutput {
   public Path getSpillFile(int spillNumber)
       throws IOException {
     return lDirAlloc.getLocalPathToRead(Constants.TASK_OUTPUT_DIR + "/spill"
-        + spillNumber + ".out", getConf());
+        + spillNumber + ".out", conf);
   }
 
   /**
@@ -146,7 +163,7 @@ public class TezLocalTaskOutputFiles extends TezTaskOutput {
   public Path getSpillFileForWrite(int spillNumber, long size)
       throws IOException {
     return lDirAlloc.getLocalPathForWrite(Constants.TASK_OUTPUT_DIR + "/spill"
-        + spillNumber + ".out", size, getConf());
+        + spillNumber + ".out", size, conf);
   }
 
   /**
@@ -160,7 +177,7 @@ public class TezLocalTaskOutputFiles extends TezTaskOutput {
   public Path getSpillIndexFile(int spillNumber)
       throws IOException {
     return lDirAlloc.getLocalPathToRead(Constants.TASK_OUTPUT_DIR + "/spill"
-        + spillNumber + ".out.index", getConf());
+        + spillNumber + ".out.index", conf);
   }
 
   /**
@@ -175,7 +192,7 @@ public class TezLocalTaskOutputFiles extends TezTaskOutput {
   public Path getSpillIndexFileForWrite(int spillNumber, long size)
       throws IOException {
     return lDirAlloc.getLocalPathForWrite(Constants.TASK_OUTPUT_DIR + "/spill"
-        + spillNumber + ".out.index", size, getConf());
+        + spillNumber + ".out.index", size, conf);
   }
 
   /**
@@ -186,11 +203,11 @@ public class TezLocalTaskOutputFiles extends TezTaskOutput {
    * @throws IOException
    */
   @Override
-  public Path getInputFile(int mapId)
+  public Path getInputFile(InputAttemptIdentifier mapId)
       throws IOException {
     return lDirAlloc.getLocalPathToRead(String.format(
         Constants.TEZ_ENGINE_TASK_INPUT_FILE_FORMAT_STRING, 
-        Constants.TASK_OUTPUT_DIR, Integer.valueOf(mapId)), getConf());
+        Constants.TASK_OUTPUT_DIR, Integer.valueOf(mapId.getInputIdentifier().getSrcTaskIndex())), conf);
   }
 
   /**
@@ -202,12 +219,12 @@ public class TezLocalTaskOutputFiles extends TezTaskOutput {
    * @throws IOException
    */
   @Override
-  public Path getInputFileForWrite(TezTaskID mapId,
+  public Path getInputFileForWrite(int taskId,
                                    long size)
       throws IOException {
     return lDirAlloc.getLocalPathForWrite(String.format(
-        Constants.TEZ_ENGINE_TASK_INPUT_FILE_FORMAT_STRING, Constants.TASK_OUTPUT_DIR, mapId.getId()),
-        size, getConf());
+        Constants.TEZ_ENGINE_TASK_INPUT_FILE_FORMAT_STRING, Constants.TASK_OUTPUT_DIR, taskId),
+        size, conf);
   }
 
   /** Removes all of the files related to a task. */
@@ -217,20 +234,15 @@ public class TezLocalTaskOutputFiles extends TezTaskOutput {
     deleteLocalFiles(Constants.TASK_OUTPUT_DIR);
   }
 
-  @Override
-  public void setConf(Configuration conf) {
-    super.setConf(conf);
-  }
-
   private String[] getLocalDirs() throws IOException {
-    return getConf().getStrings(TezJobConfig.LOCAL_DIRS);
+    return conf.getStrings(TezJobConfig.LOCAL_DIRS);
   }
 
   @SuppressWarnings("deprecation")
   private void deleteLocalFiles(String subdir) throws IOException {
     String[] localDirs = getLocalDirs();
     for (int i = 0; i < localDirs.length; i++) {
-      FileSystem.getLocal(getConf()).delete(new Path(localDirs[i], subdir));
+      FileSystem.getLocal(conf).delete(new Path(localDirs[i], subdir));
     }
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/common/task/local/output/TezTaskOutput.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/task/local/output/TezTaskOutput.java b/tez-engine/src/main/java/org/apache/tez/engine/common/task/local/output/TezTaskOutput.java
index 50d270b..e1d83ad 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/task/local/output/TezTaskOutput.java
+++ b/tez-engine/src/main/java/org/apache/tez/engine/common/task/local/output/TezTaskOutput.java
@@ -23,9 +23,8 @@ import java.io.IOException;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.conf.Configurable;
 import org.apache.hadoop.fs.Path;
-import org.apache.tez.dag.records.TezTaskID;
+import org.apache.tez.engine.common.InputAttemptIdentifier;
 
 /**
  * Manipulate the working area for the transient store for maps and reduces.
@@ -38,12 +37,14 @@ import org.apache.tez.dag.records.TezTaskID;
  */
 @InterfaceAudience.Private
 @InterfaceStability.Unstable
-public abstract class TezTaskOutput implements Configurable {
+public abstract class TezTaskOutput {
 
   protected Configuration conf;
   protected String uniqueId;
 
-  public TezTaskOutput() {
+  public TezTaskOutput(Configuration conf, String uniqueId) {
+    this.conf = conf;
+    this.uniqueId = uniqueId;
   }
 
   /**
@@ -64,6 +65,15 @@ public abstract class TezTaskOutput implements Configurable {
   public abstract Path getOutputFileForWrite(long size) throws IOException;
 
   /**
+   * Create a local output file name. This method is meant to be used *only* if
+   * the size of the file is not know up front.
+   * 
+   * @return path
+   * @throws IOException
+   */
+  public abstract Path getOutputFileForWrite() throws IOException;
+  
+  /**
    * Create a local map output file name on the same volume.
    */
   public abstract Path getOutputFileForWriteInVolume(Path existing);
@@ -133,42 +143,23 @@ public abstract class TezTaskOutput implements Configurable {
   /**
    * Return a local reduce input file created earlier
    *
-   * @param mapId a map task id
+   * @param attemptIdentifier The identifier for the source task
    * @return path
    * @throws IOException
    */
-  public abstract Path getInputFile(int mapId) throws IOException;
+  public abstract Path getInputFile(InputAttemptIdentifier attemptIdentifier) throws IOException;
 
   /**
    * Create a local reduce input file name.
    *
-   * @param mapId a map task id
+   * @param taskIdentifier The identifier for the source task
    * @param size the size of the file
    * @return path
    * @throws IOException
    */
   public abstract Path getInputFileForWrite(
-      TezTaskID mapId, long size) throws IOException;
+      int taskIdentifier, long size) throws IOException;
 
   /** Removes all of the files related to a task. */
   public abstract void removeAll() throws IOException;
-
-  public void setUniqueIdentifier(String uniqueId) {
-    this.uniqueId = uniqueId;
-  }
-  
-  public String getUniqueIdentifier() {
-    return this.uniqueId;
-  }
-  
-  @Override
-  public void setConf(Configuration conf) {
-    this.conf = conf;
-  }
-
-  @Override
-  public Configuration getConf() {
-    return conf;
-  }
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/common/task/local/output/TezTaskOutputFiles.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/task/local/output/TezTaskOutputFiles.java b/tez-engine/src/main/java/org/apache/tez/engine/common/task/local/output/TezTaskOutputFiles.java
index b7874f0..b8f051b 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/task/local/output/TezTaskOutputFiles.java
+++ b/tez-engine/src/main/java/org/apache/tez/engine/common/task/local/output/TezTaskOutputFiles.java
@@ -29,7 +29,7 @@ import org.apache.hadoop.fs.LocalDirAllocator;
 import org.apache.hadoop.fs.Path;
 import org.apache.tez.common.Constants;
 import org.apache.tez.common.TezJobConfig;
-import org.apache.tez.dag.records.TezTaskID;
+import org.apache.tez.engine.common.InputAttemptIdentifier;
 
 /**
  * Manipulate the working area for the transient store for maps and reduces.
@@ -40,32 +40,35 @@ import org.apache.tez.dag.records.TezTaskID;
  * taskTracker/jobCache/jobId/attemptId
  * This class should not be used from TaskTracker space.
  */
+
 @InterfaceAudience.Private
 @InterfaceStability.Unstable
 public class TezTaskOutputFiles extends TezTaskOutput {
+  
+  public TezTaskOutputFiles(Configuration conf, String uniqueId) {
+    super(conf, uniqueId);
+  }
 
   private static final Log LOG = LogFactory.getLog(TezTaskOutputFiles.class);
-  private Configuration conf;
 
   private static final String SPILL_FILE_PATTERN = "%s_spill_%d.out";
   private static final String SPILL_INDEX_FILE_PATTERN = SPILL_FILE_PATTERN
       + ".index";
 
-  public TezTaskOutputFiles() {
-  }
+  
 
   // assume configured to $localdir/usercache/$user/appcache/$appId
   private LocalDirAllocator lDirAlloc =
     new LocalDirAllocator(TezJobConfig.LOCAL_DIRS);
+  
 
   private Path getAttemptOutputDir() {
     if (LOG.isDebugEnabled()) {
       LOG.debug("getAttemptOutputDir: "
           + Constants.TASK_OUTPUT_DIR + "/"
-          + conf.get(Constants.TEZ_ENGINE_TASK_ATTEMPT_ID));
+          + uniqueId);
     }
-    return new Path(Constants.TASK_OUTPUT_DIR,
-        conf.get(Constants.TEZ_ENGINE_TASK_ATTEMPT_ID));
+    return new Path(Constants.TASK_OUTPUT_DIR, uniqueId);
   }
 
   /**
@@ -94,12 +97,25 @@ public class TezTaskOutputFiles extends TezTaskOutput {
   }
 
   /**
+   * Create a local map output file name. This should *only* be used if the size
+   * of the file is not known. Otherwise use the equivalent which accepts a size
+   * parameter.
+   * 
+   * @return path
+   * @throws IOException
+   */
+  public Path getOutputFileForWrite() throws IOException {
+    Path attemptOutput =
+      new Path(getAttemptOutputDir(), Constants.TEZ_ENGINE_TASK_OUTPUT_FILENAME_STRING);
+    return lDirAlloc.getLocalPathForWrite(attemptOutput.toString(), conf);
+  }
+
+  /**
    * Create a local map output file name on the same volume.
    */
   public Path getOutputFileForWriteInVolume(Path existing) {
     Path outputDir = new Path(existing.getParent(), Constants.TASK_OUTPUT_DIR);
-    Path attemptOutputDir = new Path(outputDir,
-        conf.get(Constants.TEZ_ENGINE_TASK_ATTEMPT_ID));
+    Path attemptOutputDir = new Path(outputDir, uniqueId);
     return new Path(attemptOutputDir, Constants.TEZ_ENGINE_TASK_OUTPUT_FILENAME_STRING);
   }
 
@@ -136,8 +152,7 @@ public class TezTaskOutputFiles extends TezTaskOutput {
    */
   public Path getOutputIndexFileForWriteInVolume(Path existing) {
     Path outputDir = new Path(existing.getParent(), Constants.TASK_OUTPUT_DIR);
-    Path attemptOutputDir = new Path(outputDir,
-        conf.get(Constants.TEZ_ENGINE_TASK_ATTEMPT_ID));
+    Path attemptOutputDir = new Path(outputDir, uniqueId);
     return new Path(attemptOutputDir, Constants.TEZ_ENGINE_TASK_OUTPUT_FILENAME_STRING +
                                       Constants.TEZ_ENGINE_TASK_OUTPUT_INDEX_SUFFIX_STRING);
   }
@@ -152,7 +167,7 @@ public class TezTaskOutputFiles extends TezTaskOutput {
   public Path getSpillFile(int spillNumber) throws IOException {
     return lDirAlloc.getLocalPathToRead(
         String.format(SPILL_FILE_PATTERN,
-            conf.get(Constants.TEZ_ENGINE_TASK_ATTEMPT_ID), spillNumber), conf);
+            uniqueId, spillNumber), conf);
   }
 
   /**
@@ -167,7 +182,7 @@ public class TezTaskOutputFiles extends TezTaskOutput {
       throws IOException {
     return lDirAlloc.getLocalPathForWrite(
         String.format(String.format(SPILL_FILE_PATTERN,
-            conf.get(Constants.TEZ_ENGINE_TASK_ATTEMPT_ID), spillNumber)), size, conf);
+            uniqueId, spillNumber)), size, conf);
   }
 
   /**
@@ -180,7 +195,7 @@ public class TezTaskOutputFiles extends TezTaskOutput {
   public Path getSpillIndexFile(int spillNumber) throws IOException {
     return lDirAlloc.getLocalPathToRead(
         String.format(SPILL_INDEX_FILE_PATTERN,
-            conf.get(Constants.TEZ_ENGINE_TASK_ATTEMPT_ID), spillNumber), conf);
+            uniqueId, spillNumber), conf);
   }
 
   /**
@@ -195,33 +210,32 @@ public class TezTaskOutputFiles extends TezTaskOutput {
       throws IOException {
     return lDirAlloc.getLocalPathForWrite(
         String.format(SPILL_INDEX_FILE_PATTERN,
-            conf.get(Constants.TEZ_ENGINE_TASK_ATTEMPT_ID), spillNumber), size, conf);
+            uniqueId, spillNumber), size, conf);
   }
 
   /**
    * Return a local reduce input file created earlier
    *
-   * @param mapId a map task id
+   * @param attemptIdentifier an identifier for a task. The attempt information is ignored.
    * @return path
    * @throws IOException
    */
-  public Path getInputFile(int mapId) throws IOException {
+  public Path getInputFile(InputAttemptIdentifier attemptIdentifier) throws IOException {
     throw new UnsupportedOperationException("Incompatible with LocalRunner");
   }
 
   /**
    * Create a local reduce input file name.
    *
-   * @param mapId a map task id
+   * @param attemptIdentifier an identifier for a task. The attempt information is ignored.
    * @param size the size of the file
    * @return path
    * @throws IOException
    */
-  public Path getInputFileForWrite(TezTaskID mapId,
+  public Path getInputFileForWrite(int srcTaskId,
       long size) throws IOException {
     return lDirAlloc.getLocalPathForWrite(String.format(
-        Constants.TEZ_ENGINE_TASK_INPUT_FILE_FORMAT_STRING,
-        getAttemptOutputDir().toString(), mapId.getId()),
+        uniqueId, getAttemptOutputDir().toString(), srcTaskId),
         size, conf);
   }
 
@@ -229,13 +243,4 @@ public class TezTaskOutputFiles extends TezTaskOutput {
   public void removeAll() throws IOException {
     throw new UnsupportedOperationException("Incompatible with LocalRunner");
   }
-
-  public void setConf(Configuration conf) {
-    this.conf = conf;
-  }
-
-  public Configuration getConf() {
-    return conf;
-  }
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/lib/input/ShuffledMergedInput.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/lib/input/ShuffledMergedInput.java b/tez-engine/src/main/java/org/apache/tez/engine/lib/input/ShuffledMergedInput.java
index eccd119..c719fba 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/lib/input/ShuffledMergedInput.java
+++ b/tez-engine/src/main/java/org/apache/tez/engine/lib/input/ShuffledMergedInput.java
@@ -29,12 +29,12 @@ import org.apache.tez.common.TezJobConfig;
 import org.apache.tez.common.TezUtils;
 import org.apache.tez.common.counters.TaskCounter;
 import org.apache.tez.common.counters.TezCounter;
+import org.apache.tez.engine.api.KVReader;
 import org.apache.tez.engine.common.ConfigUtils;
 import org.apache.tez.engine.common.ValuesIterator;
 import org.apache.tez.engine.common.shuffle.impl.Shuffle;
 import org.apache.tez.engine.common.sort.impl.TezRawKeyValueIterator;
 import org.apache.tez.engine.newapi.Event;
-import org.apache.tez.engine.newapi.KVReader;
 import org.apache.tez.engine.newapi.LogicalInput;
 import org.apache.tez.engine.newapi.TezInputContext;
 

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/lib/oldinput/LocalMergedInput.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/lib/oldinput/LocalMergedInput.java b/tez-engine/src/main/java/org/apache/tez/engine/lib/oldinput/LocalMergedInput.java
deleted file mode 100644
index 269fe81..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/lib/oldinput/LocalMergedInput.java
+++ /dev/null
@@ -1,67 +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.engine.lib.oldinput;
-
-import java.io.IOException;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.tez.common.TezEngineTaskContext;
-import org.apache.tez.engine.api.Input;
-import org.apache.tez.engine.api.Master;
-import org.apache.tez.engine.common.sort.impl.TezRawKeyValueIterator;
-
-/**
- * {@link LocalMergedInput} in an {@link Input} which shuffles intermediate
- * sorted data, merges them and provides key/<values> to the consumer. 
- */
-public class LocalMergedInput extends OldShuffledMergedInput {
-
-  public LocalMergedInput(TezEngineTaskContext task, int index) {
-    super(task, index);
-  }
-
-  public void initialize(Configuration conf, Master master) throws IOException,
-      InterruptedException {
-  }
-
-  public boolean hasNext() throws IOException, InterruptedException {
-    return false;
-  }
-
-  public Object getNextKey() throws IOException, InterruptedException {
-    return null;
-  }
-
-  @SuppressWarnings({ "unchecked", "rawtypes" })
-  public Iterable getNextValues() 
-      throws IOException, InterruptedException {
-    return null;
-  }
-
-  public float getProgress() throws IOException, InterruptedException {
-    return 0f;
-  }
-
-  public void close() throws IOException {
-  }
-
-  public TezRawKeyValueIterator getIterator() {
-    return null;
-  }
-  
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/lib/oldinput/OldShuffledMergedInput.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/lib/oldinput/OldShuffledMergedInput.java b/tez-engine/src/main/java/org/apache/tez/engine/lib/oldinput/OldShuffledMergedInput.java
deleted file mode 100644
index c046a27..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/lib/oldinput/OldShuffledMergedInput.java
+++ /dev/null
@@ -1,74 +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.engine.lib.oldinput;
-
-import java.io.IOException;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.tez.common.RunningTaskContext;
-import org.apache.tez.common.TezEngineTaskContext;
-import org.apache.tez.engine.api.Input;
-import org.apache.tez.engine.api.Master;
-import org.apache.tez.engine.common.sort.impl.TezRawKeyValueIterator;
-
-/**
- * {@link OldShuffledMergedInput} in an {@link Input} which shuffles intermediate
- * sorted data, merges them and provides key/<values> to the consumer. 
- */
-public class OldShuffledMergedInput implements Input {
-
-
-  public OldShuffledMergedInput(TezEngineTaskContext task, int index) {
-  }
-
-  public void mergeWith(OldShuffledMergedInput other) {
-  }
-  
-  public void setTask(RunningTaskContext runningTaskContext) {
-  }
-  
-  public void initialize(Configuration conf, Master master) throws IOException,
-      InterruptedException {
-  }
-
-  public boolean hasNext() throws IOException, InterruptedException {
-    return false;
-  }
-
-  public Object getNextKey() throws IOException, InterruptedException {
-    return null;
-  }
-
-  @SuppressWarnings({ "unchecked", "rawtypes" })
-  public Iterable getNextValues() 
-      throws IOException, InterruptedException {
-    return null;
-  }
-
-  public float getProgress() throws IOException, InterruptedException {
-    return 0f;
-  }
-
-  public void close() throws IOException {
-  }
-
-  public TezRawKeyValueIterator getIterator() {
-    return null;
-  }
-  
-}


[39/50] [abbrv] Rename tez-engine-api to tez-runtime-api and tez-engine is split into 2: - tez-engine-library for user-visible Input/Output/Processor implementations - tez-engine-internals for framework internals

Posted by ss...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/dflt/DefaultSorter.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/dflt/DefaultSorter.java b/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/dflt/DefaultSorter.java
deleted file mode 100644
index 6b48270..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/dflt/DefaultSorter.java
+++ /dev/null
@@ -1,1108 +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.engine.common.sort.impl.dflt;
-
-import java.io.DataOutputStream;
-import java.io.IOException;
-import java.io.OutputStream;
-import java.nio.ByteBuffer;
-import java.nio.ByteOrder;
-import java.nio.IntBuffer;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.concurrent.locks.Condition;
-import java.util.concurrent.locks.ReentrantLock;
-
-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.Path;
-import org.apache.hadoop.io.DataInputBuffer;
-import org.apache.hadoop.io.RawComparator;
-import org.apache.hadoop.util.IndexedSortable;
-import org.apache.hadoop.util.Progress;
-import org.apache.hadoop.util.StringUtils;
-import org.apache.tez.common.TezJobConfig;
-import org.apache.tez.engine.api.TezOutputContext;
-import org.apache.tez.engine.common.ConfigUtils;
-import org.apache.tez.engine.common.sort.impl.ExternalSorter;
-import org.apache.tez.engine.common.sort.impl.IFile;
-import org.apache.tez.engine.common.sort.impl.TezIndexRecord;
-import org.apache.tez.engine.common.sort.impl.TezMerger;
-import org.apache.tez.engine.common.sort.impl.TezRawKeyValueIterator;
-import org.apache.tez.engine.common.sort.impl.TezSpillRecord;
-import org.apache.tez.engine.common.sort.impl.IFile.Writer;
-import org.apache.tez.engine.common.sort.impl.TezMerger.Segment;
-
-@SuppressWarnings({"unchecked", "rawtypes"})
-public class DefaultSorter extends ExternalSorter implements IndexedSortable {
-  
-  private static final Log LOG = LogFactory.getLog(DefaultSorter.class);
-
-  // TODO NEWTEZ Progress reporting to Tez framework. (making progress vs %complete)
-  
-  /**
-   * The size of each record in the index file for the map-outputs.
-   */
-  public static final int MAP_OUTPUT_INDEX_RECORD_LENGTH = 24;
-
-  private final static int APPROX_HEADER_LENGTH = 150;
-
-  // k/v accounting
-  IntBuffer kvmeta; // metadata overlay on backing store
-  int kvstart;            // marks origin of spill metadata
-  int kvend;              // marks end of spill metadata
-  int kvindex;            // marks end of fully serialized records
-
-  int equator;            // marks origin of meta/serialization
-  int bufstart;           // marks beginning of spill
-  int bufend;             // marks beginning of collectable
-  int bufmark;            // marks end of record
-  int bufindex;           // marks end of collected
-  int bufvoid;            // marks the point where we should stop
-                          // reading at the end of the buffer
-
-  byte[] kvbuffer;        // main output buffer
-  private final byte[] b0 = new byte[0];
-
-  protected static final int INDEX = 0;            // index offset in acct
-  protected static final int VALSTART = 1;         // val offset in acct
-  protected static final int KEYSTART = 2;         // key offset in acct
-  protected static final int PARTITION = 3;        // partition offset in acct
-  protected static final int NMETA = 4;            // num meta ints
-  protected static final int METASIZE = NMETA * 4; // size in bytes
-
-  // spill accounting
-  int maxRec;
-  int softLimit;
-  boolean spillInProgress;
-  int bufferRemaining;
-  volatile Throwable sortSpillException = null;
-
-  int numSpills = 0;
-  int minSpillsForCombine;
-  final ReentrantLock spillLock = new ReentrantLock();
-  final Condition spillDone = spillLock.newCondition();
-  final Condition spillReady = spillLock.newCondition();
-  final BlockingBuffer bb = new BlockingBuffer();
-  volatile boolean spillThreadRunning = false;
-  final SpillThread spillThread = new SpillThread();
-
-  final ArrayList<TezSpillRecord> indexCacheList =
-    new ArrayList<TezSpillRecord>();
-  private int totalIndexCacheMemory;
-  private int indexCacheMemoryLimit;
-
-  @Override
-  public void initialize(TezOutputContext outputContext, Configuration conf, int numOutputs) throws IOException { 
-    super.initialize(outputContext, conf, numOutputs);
-
-    // sanity checks
-    final float spillper = this.conf.getFloat(
-        TezJobConfig.TEZ_ENGINE_SORT_SPILL_PERCENT,
-        TezJobConfig.DEFAULT_TEZ_ENGINE_SORT_SPILL_PERCENT);
-    final int sortmb = this.conf.getInt(TezJobConfig.TEZ_ENGINE_IO_SORT_MB,
-        TezJobConfig.DEFAULT_TEZ_ENGINE_IO_SORT_MB);
-    if (spillper > (float) 1.0 || spillper <= (float) 0.0) {
-      throw new IOException("Invalid \""
-          + TezJobConfig.TEZ_ENGINE_SORT_SPILL_PERCENT + "\": " + spillper);
-    }
-    if ((sortmb & 0x7FF) != sortmb) {
-      throw new IOException("Invalid \"" + TezJobConfig.TEZ_ENGINE_IO_SORT_MB
-          + "\": " + sortmb);
-    }
-
-    indexCacheMemoryLimit = this.conf.getInt(TezJobConfig.TEZ_ENGINE_INDEX_CACHE_MEMORY_LIMIT_BYTES,
-                                       TezJobConfig.DEFAULT_TEZ_ENGINE_INDEX_CACHE_MEMORY_LIMIT_BYTES);
-
-    // buffers and accounting
-    int maxMemUsage = sortmb << 20;
-    maxMemUsage -= maxMemUsage % METASIZE;
-    kvbuffer = new byte[maxMemUsage];
-    bufvoid = kvbuffer.length;
-    kvmeta = ByteBuffer.wrap(kvbuffer)
-       .order(ByteOrder.nativeOrder())
-       .asIntBuffer();
-    setEquator(0);
-    bufstart = bufend = bufindex = equator;
-    kvstart = kvend = kvindex;
-
-    maxRec = kvmeta.capacity() / NMETA;
-    softLimit = (int)(kvbuffer.length * spillper);
-    bufferRemaining = softLimit;
-    if (LOG.isInfoEnabled()) {
-      LOG.info(TezJobConfig.TEZ_ENGINE_IO_SORT_MB + ": " + sortmb);
-      LOG.info("soft limit at " + softLimit);
-      LOG.info("bufstart = " + bufstart + "; bufvoid = " + bufvoid);
-      LOG.info("kvstart = " + kvstart + "; length = " + maxRec);
-    }
-
-    // k/v serialization
-    valSerializer.open(bb);
-    keySerializer.open(bb);
-
-    spillInProgress = false;
-    minSpillsForCombine = this.conf.getInt(TezJobConfig.TEZ_ENGINE_COMBINE_MIN_SPILLS, 3);
-    spillThread.setDaemon(true);
-    spillThread.setName("SpillThread");
-    spillLock.lock();
-    try {
-      spillThread.start();
-      while (!spillThreadRunning) {
-        spillDone.await();
-      }
-    } catch (InterruptedException e) {
-      throw new IOException("Spill thread failed to initialize", e);
-    } finally {
-      spillLock.unlock();
-    }
-    if (sortSpillException != null) {
-      throw new IOException("Spill thread failed to initialize",
-          sortSpillException);
-    }
-  }
-
-  @Override
-  public void write(Object key, Object value)
-      throws IOException {
-    collect(
-        key, value, partitioner.getPartition(key, value, partitions));
-  }
-
-  /**
-   * Serialize the key, value to intermediate storage.
-   * When this method returns, kvindex must refer to sufficient unused
-   * storage to store one METADATA.
-   */
-  synchronized void collect(Object key, Object value, final int partition
-                                   ) throws IOException {
-
-    if (key.getClass() != keyClass) {
-      throw new IOException("Type mismatch in key from map: expected "
-                            + keyClass.getName() + ", received "
-                            + key.getClass().getName());
-    }
-    if (value.getClass() != valClass) {
-      throw new IOException("Type mismatch in value from map: expected "
-                            + valClass.getName() + ", received "
-                            + value.getClass().getName());
-    }
-    if (partition < 0 || partition >= partitions) {
-      throw new IOException("Illegal partition for " + key + " (" +
-          partition + ")" + ", TotalPartitions: " + partitions);
-    }
-    checkSpillException();
-    bufferRemaining -= METASIZE;
-    if (bufferRemaining <= 0) {
-      // start spill if the thread is not running and the soft limit has been
-      // reached
-      spillLock.lock();
-      try {
-        do {
-          if (!spillInProgress) {
-            final int kvbidx = 4 * kvindex;
-            final int kvbend = 4 * kvend;
-            // serialized, unspilled bytes always lie between kvindex and
-            // bufindex, crossing the equator. Note that any void space
-            // created by a reset must be included in "used" bytes
-            final int bUsed = distanceTo(kvbidx, bufindex);
-            final boolean bufsoftlimit = bUsed >= softLimit;
-            if ((kvbend + METASIZE) % kvbuffer.length !=
-                equator - (equator % METASIZE)) {
-              // spill finished, reclaim space
-              resetSpill();
-              bufferRemaining = Math.min(
-                  distanceTo(bufindex, kvbidx) - 2 * METASIZE,
-                  softLimit - bUsed) - METASIZE;
-              continue;
-            } else if (bufsoftlimit && kvindex != kvend) {
-              // spill records, if any collected; check latter, as it may
-              // be possible for metadata alignment to hit spill pcnt
-              startSpill();
-              final int avgRec = (int)
-                (mapOutputByteCounter.getValue() /
-                mapOutputRecordCounter.getValue());
-              // leave at least half the split buffer for serialization data
-              // ensure that kvindex >= bufindex
-              final int distkvi = distanceTo(bufindex, kvbidx);
-              final int newPos = (bufindex +
-                Math.max(2 * METASIZE - 1,
-                        Math.min(distkvi / 2,
-                                 distkvi / (METASIZE + avgRec) * METASIZE)))
-                % kvbuffer.length;
-              setEquator(newPos);
-              bufmark = bufindex = newPos;
-              final int serBound = 4 * kvend;
-              // bytes remaining before the lock must be held and limits
-              // checked is the minimum of three arcs: the metadata space, the
-              // serialization space, and the soft limit
-              bufferRemaining = Math.min(
-                  // metadata max
-                  distanceTo(bufend, newPos),
-                  Math.min(
-                    // serialization max
-                    distanceTo(newPos, serBound),
-                    // soft limit
-                    softLimit)) - 2 * METASIZE;
-            }
-          }
-        } while (false);
-      } finally {
-        spillLock.unlock();
-      }
-    }
-
-    try {
-      // serialize key bytes into buffer
-      int keystart = bufindex;
-      keySerializer.serialize(key);
-      if (bufindex < keystart) {
-        // wrapped the key; must make contiguous
-        bb.shiftBufferedKey();
-        keystart = 0;
-      }
-      // serialize value bytes into buffer
-      final int valstart = bufindex;
-      valSerializer.serialize(value);
-      // It's possible for records to have zero length, i.e. the serializer
-      // will perform no writes. To ensure that the boundary conditions are
-      // checked and that the kvindex invariant is maintained, perform a
-      // zero-length write into the buffer. The logic monitoring this could be
-      // moved into collect, but this is cleaner and inexpensive. For now, it
-      // is acceptable.
-      bb.write(b0, 0, 0);
-
-      // the record must be marked after the preceding write, as the metadata
-      // for this record are not yet written
-      int valend = bb.markRecord();
-
-      mapOutputRecordCounter.increment(1);
-      mapOutputByteCounter.increment(
-          distanceTo(keystart, valend, bufvoid));
-
-      // write accounting info
-      kvmeta.put(kvindex + INDEX, kvindex);
-      kvmeta.put(kvindex + PARTITION, partition);
-      kvmeta.put(kvindex + KEYSTART, keystart);
-      kvmeta.put(kvindex + VALSTART, valstart);
-      // advance kvindex
-      kvindex = (kvindex - NMETA + kvmeta.capacity()) % kvmeta.capacity();
-    } catch (MapBufferTooSmallException e) {
-      LOG.info("Record too large for in-memory buffer: " + e.getMessage());
-      spillSingleRecord(key, value, partition);
-      mapOutputRecordCounter.increment(1);
-      return;
-    }
-  }
-
-  /**
-   * Set the point from which meta and serialization data expand. The meta
-   * indices are aligned with the buffer, so metadata never spans the ends of
-   * the circular buffer.
-   */
-  private void setEquator(int pos) {
-    equator = pos;
-    // set index prior to first entry, aligned at meta boundary
-    final int aligned = pos - (pos % METASIZE);
-    kvindex =
-      ((aligned - METASIZE + kvbuffer.length) % kvbuffer.length) / 4;
-    if (LOG.isInfoEnabled()) {
-      LOG.info("(EQUATOR) " + pos + " kvi " + kvindex +
-          "(" + (kvindex * 4) + ")");
-    }
-  }
-
-  /**
-   * The spill is complete, so set the buffer and meta indices to be equal to
-   * the new equator to free space for continuing collection. Note that when
-   * kvindex == kvend == kvstart, the buffer is empty.
-   */
-  private void resetSpill() {
-    final int e = equator;
-    bufstart = bufend = e;
-    final int aligned = e - (e % METASIZE);
-    // set start/end to point to first meta record
-    kvstart = kvend =
-      ((aligned - METASIZE + kvbuffer.length) % kvbuffer.length) / 4;
-    if (LOG.isInfoEnabled()) {
-      LOG.info("(RESET) equator " + e + " kv " + kvstart + "(" +
-        (kvstart * 4) + ")" + " kvi " + kvindex + "(" + (kvindex * 4) + ")");
-    }
-  }
-
-  /**
-   * Compute the distance in bytes between two indices in the serialization
-   * buffer.
-   * @see #distanceTo(int,int,int)
-   */
-  final int distanceTo(final int i, final int j) {
-    return distanceTo(i, j, kvbuffer.length);
-  }
-
-  /**
-   * Compute the distance between two indices in the circular buffer given the
-   * max distance.
-   */
-  int distanceTo(final int i, final int j, final int mod) {
-    return i <= j
-      ? j - i
-      : mod - i + j;
-  }
-
-  /**
-   * For the given meta position, return the dereferenced position in the
-   * integer array. Each meta block contains several integers describing
-   * record data in its serialized form, but the INDEX is not necessarily
-   * related to the proximate metadata. The index value at the referenced int
-   * position is the start offset of the associated metadata block. So the
-   * metadata INDEX at metapos may point to the metadata described by the
-   * metadata block at metapos + k, which contains information about that
-   * serialized record.
-   */
-  int offsetFor(int metapos) {
-    return kvmeta.get((metapos % maxRec) * NMETA + INDEX);
-  }
-
-  /**
-   * Compare logical range, st i, j MOD offset capacity.
-   * Compare by partition, then by key.
-   * @see IndexedSortable#compare
-   */
-  public int compare(final int mi, final int mj) {
-    final int kvi = offsetFor(mi);
-    final int kvj = offsetFor(mj);
-    final int kvip = kvmeta.get(kvi + PARTITION);
-    final int kvjp = kvmeta.get(kvj + PARTITION);
-    // sort by partition
-    if (kvip != kvjp) {
-      return kvip - kvjp;
-    }
-    // sort by key
-    return comparator.compare(kvbuffer,
-        kvmeta.get(kvi + KEYSTART),
-        kvmeta.get(kvi + VALSTART) - kvmeta.get(kvi + KEYSTART),
-        kvbuffer,
-        kvmeta.get(kvj + KEYSTART),
-        kvmeta.get(kvj + VALSTART) - kvmeta.get(kvj + KEYSTART));
-  }
-
-  /**
-   * Swap logical indices st i, j MOD offset capacity.
-   * @see IndexedSortable#swap
-   */
-  public void swap(final int mi, final int mj) {
-    final int kvi = (mi % maxRec) * NMETA + INDEX;
-    final int kvj = (mj % maxRec) * NMETA + INDEX;
-    int tmp = kvmeta.get(kvi);
-    kvmeta.put(kvi, kvmeta.get(kvj));
-    kvmeta.put(kvj, tmp);
-  }
-
-  /**
-   * Inner class managing the spill of serialized records to disk.
-   */
-  protected class BlockingBuffer extends DataOutputStream {
-
-    public BlockingBuffer() {
-      super(new Buffer());
-    }
-
-    /**
-     * Mark end of record. Note that this is required if the buffer is to
-     * cut the spill in the proper place.
-     */
-    public int markRecord() {
-      bufmark = bufindex;
-      return bufindex;
-    }
-
-    /**
-     * Set position from last mark to end of writable buffer, then rewrite
-     * the data between last mark and kvindex.
-     * This handles a special case where the key wraps around the buffer.
-     * If the key is to be passed to a RawComparator, then it must be
-     * contiguous in the buffer. This recopies the data in the buffer back
-     * into itself, but starting at the beginning of the buffer. Note that
-     * this method should <b>only</b> be called immediately after detecting
-     * this condition. To call it at any other time is undefined and would
-     * likely result in data loss or corruption.
-     * @see #markRecord()
-     */
-    protected void shiftBufferedKey() throws IOException {
-      // spillLock unnecessary; both kvend and kvindex are current
-      int headbytelen = bufvoid - bufmark;
-      bufvoid = bufmark;
-      final int kvbidx = 4 * kvindex;
-      final int kvbend = 4 * kvend;
-      final int avail =
-        Math.min(distanceTo(0, kvbidx), distanceTo(0, kvbend));
-      if (bufindex + headbytelen < avail) {
-        System.arraycopy(kvbuffer, 0, kvbuffer, headbytelen, bufindex);
-        System.arraycopy(kvbuffer, bufvoid, kvbuffer, 0, headbytelen);
-        bufindex += headbytelen;
-        bufferRemaining -= kvbuffer.length - bufvoid;
-      } else {
-        byte[] keytmp = new byte[bufindex];
-        System.arraycopy(kvbuffer, 0, keytmp, 0, bufindex);
-        bufindex = 0;
-        out.write(kvbuffer, bufmark, headbytelen);
-        out.write(keytmp);
-      }
-    }
-  }
-
-  public class Buffer extends OutputStream {
-    private final byte[] scratch = new byte[1];
-
-    @Override
-    public void write(int v)
-        throws IOException {
-      scratch[0] = (byte)v;
-      write(scratch, 0, 1);
-    }
-
-    /**
-     * Attempt to write a sequence of bytes to the collection buffer.
-     * This method will block if the spill thread is running and it
-     * cannot write.
-     * @throws MapBufferTooSmallException if record is too large to
-     *    deserialize into the collection buffer.
-     */
-    @Override
-    public void write(byte b[], int off, int len)
-        throws IOException {
-      // must always verify the invariant that at least METASIZE bytes are
-      // available beyond kvindex, even when len == 0
-      bufferRemaining -= len;
-      if (bufferRemaining <= 0) {
-        // writing these bytes could exhaust available buffer space or fill
-        // the buffer to soft limit. check if spill or blocking are necessary
-        boolean blockwrite = false;
-        spillLock.lock();
-        try {
-          do {
-            checkSpillException();
-
-            final int kvbidx = 4 * kvindex;
-            final int kvbend = 4 * kvend;
-            // ser distance to key index
-            final int distkvi = distanceTo(bufindex, kvbidx);
-            // ser distance to spill end index
-            final int distkve = distanceTo(bufindex, kvbend);
-
-            // if kvindex is closer than kvend, then a spill is neither in
-            // progress nor complete and reset since the lock was held. The
-            // write should block only if there is insufficient space to
-            // complete the current write, write the metadata for this record,
-            // and write the metadata for the next record. If kvend is closer,
-            // then the write should block if there is too little space for
-            // either the metadata or the current write. Note that collect
-            // ensures its metadata requirement with a zero-length write
-            blockwrite = distkvi <= distkve
-              ? distkvi <= len + 2 * METASIZE
-              : distkve <= len || distanceTo(bufend, kvbidx) < 2 * METASIZE;
-
-            if (!spillInProgress) {
-              if (blockwrite) {
-                if ((kvbend + METASIZE) % kvbuffer.length !=
-                    equator - (equator % METASIZE)) {
-                  // spill finished, reclaim space
-                  // need to use meta exclusively; zero-len rec & 100% spill
-                  // pcnt would fail
-                  resetSpill(); // resetSpill doesn't move bufindex, kvindex
-                  bufferRemaining = Math.min(
-                      distkvi - 2 * METASIZE,
-                      softLimit - distanceTo(kvbidx, bufindex)) - len;
-                  continue;
-                }
-                // we have records we can spill; only spill if blocked
-                if (kvindex != kvend) {
-                  startSpill();
-                  // Blocked on this write, waiting for the spill just
-                  // initiated to finish. Instead of repositioning the marker
-                  // and copying the partial record, we set the record start
-                  // to be the new equator
-                  setEquator(bufmark);
-                } else {
-                  // We have no buffered records, and this record is too large
-                  // to write into kvbuffer. We must spill it directly from
-                  // collect
-                  final int size = distanceTo(bufstart, bufindex) + len;
-                  setEquator(0);
-                  bufstart = bufend = bufindex = equator;
-                  kvstart = kvend = kvindex;
-                  bufvoid = kvbuffer.length;
-                  throw new MapBufferTooSmallException(size + " bytes");
-                }
-              }
-            }
-
-            if (blockwrite) {
-              // wait for spill
-              try {
-                while (spillInProgress) {
-                  spillDone.await();
-                }
-              } catch (InterruptedException e) {
-                  throw new IOException(
-                      "Buffer interrupted while waiting for the writer", e);
-              }
-            }
-          } while (blockwrite);
-        } finally {
-          spillLock.unlock();
-        }
-      }
-      // here, we know that we have sufficient space to write
-      if (bufindex + len > bufvoid) {
-        final int gaplen = bufvoid - bufindex;
-        System.arraycopy(b, off, kvbuffer, bufindex, gaplen);
-        len -= gaplen;
-        off += gaplen;
-        bufindex = 0;
-      }
-      System.arraycopy(b, off, kvbuffer, bufindex, len);
-      bufindex += len;
-    }
-  }
-
-  @Override
-  public void flush() throws IOException {
-    LOG.info("Starting flush of map output");
-    spillLock.lock();
-    try {
-      while (spillInProgress) {
-        spillDone.await();
-      }
-      checkSpillException();
-
-      final int kvbend = 4 * kvend;
-      if ((kvbend + METASIZE) % kvbuffer.length !=
-          equator - (equator % METASIZE)) {
-        // spill finished
-        resetSpill();
-      }
-      if (kvindex != kvend) {
-        kvend = (kvindex + NMETA) % kvmeta.capacity();
-        bufend = bufmark;
-        if (LOG.isInfoEnabled()) {
-          LOG.info("Sorting & Spilling map output");
-          LOG.info("bufstart = " + bufstart + "; bufend = " + bufmark +
-                   "; bufvoid = " + bufvoid);
-          LOG.info("kvstart = " + kvstart + "(" + (kvstart * 4) +
-                   "); kvend = " + kvend + "(" + (kvend * 4) +
-                   "); length = " + (distanceTo(kvend, kvstart,
-                         kvmeta.capacity()) + 1) + "/" + maxRec);
-        }
-        sortAndSpill();
-      }
-    } catch (InterruptedException e) {
-      throw new IOException("Interrupted while waiting for the writer", e);
-    } finally {
-      spillLock.unlock();
-    }
-    assert !spillLock.isHeldByCurrentThread();
-    // shut down spill thread and wait for it to exit. Since the preceding
-    // ensures that it is finished with its work (and sortAndSpill did not
-    // throw), we elect to use an interrupt instead of setting a flag.
-    // Spilling simultaneously from this thread while the spill thread
-    // finishes its work might be both a useful way to extend this and also
-    // sufficient motivation for the latter approach.
-    try {
-      spillThread.interrupt();
-      spillThread.join();
-    } catch (InterruptedException e) {
-      throw new IOException("Spill failed", e);
-    }
-    // release sort buffer before the merge
-    //FIXME
-    //kvbuffer = null;
-    mergeParts();
-    Path outputPath = mapOutputFile.getOutputFile();
-    fileOutputByteCounter.increment(rfs.getFileStatus(outputPath).getLen());
-  }
-
-  @Override
-  public void close() throws IOException { }
-
-  protected class SpillThread extends Thread {
-
-    @Override
-    public void run() {
-      spillLock.lock();
-      spillThreadRunning = true;
-      try {
-        while (true) {
-          spillDone.signal();
-          while (!spillInProgress) {
-            spillReady.await();
-          }
-          try {
-            spillLock.unlock();
-            sortAndSpill();
-          } catch (Throwable t) {
-            LOG.warn("Got an exception in sortAndSpill", t);
-            sortSpillException = t;
-          } finally {
-            spillLock.lock();
-            if (bufend < bufstart) {
-              bufvoid = kvbuffer.length;
-            }
-            kvstart = kvend;
-            bufstart = bufend;
-            spillInProgress = false;
-          }
-        }
-      } catch (InterruptedException e) {
-        Thread.currentThread().interrupt();
-      } finally {
-        spillLock.unlock();
-        spillThreadRunning = false;
-      }
-    }
-  }
-
-  private void checkSpillException() throws IOException {
-    final Throwable lspillException = sortSpillException;
-    if (lspillException != null) {
-      if (lspillException instanceof Error) {
-        final String logMsg = "Task " + outputContext.getUniqueIdentifier()
-            + " failed : " + StringUtils.stringifyException(lspillException);
-        outputContext.fatalError(lspillException, logMsg);
-      }
-      throw new IOException("Spill failed", lspillException);
-    }
-  }
-
-  private void startSpill() {
-    assert !spillInProgress;
-    kvend = (kvindex + NMETA) % kvmeta.capacity();
-    bufend = bufmark;
-    spillInProgress = true;
-    if (LOG.isInfoEnabled()) {
-      LOG.info("Spilling map output");
-      LOG.info("bufstart = " + bufstart + "; bufend = " + bufmark +
-               "; bufvoid = " + bufvoid);
-      LOG.info("kvstart = " + kvstart + "(" + (kvstart * 4) +
-               "); kvend = " + kvend + "(" + (kvend * 4) +
-               "); length = " + (distanceTo(kvend, kvstart,
-                     kvmeta.capacity()) + 1) + "/" + maxRec);
-    }
-    spillReady.signal();
-  }
-
-  int getMetaStart() {
-    return kvend / NMETA;
-  }
-
-  int getMetaEnd() {
-    return 1 + // kvend is a valid record
-        (kvstart >= kvend
-        ? kvstart
-        : kvmeta.capacity() + kvstart) / NMETA;
-  }
-
-  protected void sortAndSpill()
-      throws IOException, InterruptedException {
-    final int mstart = getMetaStart();
-    final int mend = getMetaEnd();
-    sorter.sort(this, mstart, mend, nullProgressable);
-    spill(mstart, mend);
-  }
-
-  protected void spill(int mstart, int mend)
-      throws IOException, InterruptedException {
-
-    //approximate the length of the output file to be the length of the
-    //buffer + header lengths for the partitions
-    final long size = (bufend >= bufstart
-        ? bufend - bufstart
-        : (bufvoid - bufend) + bufstart) +
-                partitions * APPROX_HEADER_LENGTH;
-    FSDataOutputStream out = null;
-    try {
-      // create spill file
-      final TezSpillRecord spillRec = new TezSpillRecord(partitions);
-      final Path filename =
-          mapOutputFile.getSpillFileForWrite(numSpills, size);
-      out = rfs.create(filename);
-
-      int spindex = mstart;
-      final InMemValBytes value = createInMemValBytes();
-      for (int i = 0; i < partitions; ++i) {
-        IFile.Writer writer = null;
-        try {
-          long segmentStart = out.getPos();
-          writer = new Writer(conf, out, keyClass, valClass, codec,
-                                    spilledRecordsCounter);
-          if (combiner == null) {
-            // spill directly
-            DataInputBuffer key = new DataInputBuffer();
-            while (spindex < mend &&
-                kvmeta.get(offsetFor(spindex) + PARTITION) == i) {
-              final int kvoff = offsetFor(spindex);
-              key.reset(
-                  kvbuffer,
-                  kvmeta.get(kvoff + KEYSTART),
-                  (kvmeta.get(kvoff + VALSTART) - kvmeta.get(kvoff + KEYSTART))
-                  );
-              getVBytesForOffset(kvoff, value);
-              writer.append(key, value);
-              ++spindex;
-            }
-          } else {
-            int spstart = spindex;
-            while (spindex < mend &&
-                kvmeta.get(offsetFor(spindex)
-                          + PARTITION) == i) {
-              ++spindex;
-            }
-            // Note: we would like to avoid the combiner if we've fewer
-            // than some threshold of records for a partition
-            if (spstart != spindex) {
-              TezRawKeyValueIterator kvIter =
-                new MRResultIterator(spstart, spindex);
-              if (LOG.isDebugEnabled()) {
-                LOG.debug("Running combine processor");
-              }
-              runCombineProcessor(kvIter, writer);
-            }
-          }
-
-          // close the writer
-          writer.close();
-
-          // record offsets
-          final TezIndexRecord rec =
-              new TezIndexRecord(
-                  segmentStart,
-                  writer.getRawLength(),
-                  writer.getCompressedLength());
-          spillRec.putIndex(rec, i);
-
-          writer = null;
-        } finally {
-          if (null != writer) writer.close();
-        }
-      }
-
-      if (totalIndexCacheMemory >= indexCacheMemoryLimit) {
-        // create spill index file
-        Path indexFilename =
-            mapOutputFile.getSpillIndexFileForWrite(numSpills, partitions
-                * MAP_OUTPUT_INDEX_RECORD_LENGTH);
-        spillRec.writeToFile(indexFilename, conf);
-      } else {
-        indexCacheList.add(spillRec);
-        totalIndexCacheMemory +=
-          spillRec.size() * MAP_OUTPUT_INDEX_RECORD_LENGTH;
-      }
-      LOG.info("Finished spill " + numSpills);
-      ++numSpills;
-    } finally {
-      if (out != null) out.close();
-    }
-  }
-
-  /**
-   * Handles the degenerate case where serialization fails to fit in
-   * the in-memory buffer, so we must spill the record from collect
-   * directly to a spill file. Consider this "losing".
-   */
-  private void spillSingleRecord(final Object key, final Object value,
-                                 int partition) throws IOException {
-    long size = kvbuffer.length + partitions * APPROX_HEADER_LENGTH;
-    FSDataOutputStream out = null;
-    try {
-      // create spill file
-      final TezSpillRecord spillRec = new TezSpillRecord(partitions);
-      final Path filename =
-          mapOutputFile.getSpillFileForWrite(numSpills, size);
-      out = rfs.create(filename);
-
-      // we don't run the combiner for a single record
-      for (int i = 0; i < partitions; ++i) {
-        IFile.Writer writer = null;
-        try {
-          long segmentStart = out.getPos();
-          // Create a new codec, don't care!
-          writer = new IFile.Writer(conf, out, keyClass, valClass, codec,
-                                          spilledRecordsCounter);
-
-          if (i == partition) {
-            final long recordStart = out.getPos();
-            writer.append(key, value);
-            // Note that our map byte count will not be accurate with
-            // compression
-            mapOutputByteCounter.increment(out.getPos() - recordStart);
-          }
-          writer.close();
-
-          // record offsets
-          TezIndexRecord rec =
-              new TezIndexRecord(
-                  segmentStart,
-                  writer.getRawLength(),
-                  writer.getCompressedLength());
-          spillRec.putIndex(rec, i);
-
-          writer = null;
-        } catch (IOException e) {
-          if (null != writer) writer.close();
-          throw e;
-        }
-      }
-      if (totalIndexCacheMemory >= indexCacheMemoryLimit) {
-        // create spill index file
-        Path indexFilename =
-            mapOutputFile.getSpillIndexFileForWrite(numSpills, partitions
-                * MAP_OUTPUT_INDEX_RECORD_LENGTH);
-        spillRec.writeToFile(indexFilename, conf);
-      } else {
-        indexCacheList.add(spillRec);
-        totalIndexCacheMemory +=
-          spillRec.size() * MAP_OUTPUT_INDEX_RECORD_LENGTH;
-      }
-      ++numSpills;
-    } finally {
-      if (out != null) out.close();
-    }
-  }
-
-  protected int getInMemVBytesLength(int kvoff) {
-    // get the keystart for the next serialized value to be the end
-    // of this value. If this is the last value in the buffer, use bufend
-    final int nextindex = kvoff == kvend
-      ? bufend
-      : kvmeta.get(
-          (kvoff - NMETA + kvmeta.capacity() + KEYSTART) % kvmeta.capacity());
-    // calculate the length of the value
-    int vallen = (nextindex >= kvmeta.get(kvoff + VALSTART))
-      ? nextindex - kvmeta.get(kvoff + VALSTART)
-      : (bufvoid - kvmeta.get(kvoff + VALSTART)) + nextindex;
-      return vallen;
-  }
-
-  /**
-   * Given an offset, populate vbytes with the associated set of
-   * deserialized value bytes. Should only be called during a spill.
-   */
-  int getVBytesForOffset(int kvoff, InMemValBytes vbytes) {
-    int vallen = getInMemVBytesLength(kvoff);
-    vbytes.reset(kvbuffer, kvmeta.get(kvoff + VALSTART), vallen);
-    return vallen;
-  }
-
-  /**
-   * Inner class wrapping valuebytes, used for appendRaw.
-   */
-  static class InMemValBytes extends DataInputBuffer {
-    private byte[] buffer;
-    private int start;
-    private int length;
-    private final int bufvoid;
-
-    public InMemValBytes(int bufvoid) {
-      this.bufvoid = bufvoid;
-    }
-
-    public void reset(byte[] buffer, int start, int length) {
-      this.buffer = buffer;
-      this.start = start;
-      this.length = length;
-
-      if (start + length > bufvoid) {
-        this.buffer = new byte[this.length];
-        final int taillen = bufvoid - start;
-        System.arraycopy(buffer, start, this.buffer, 0, taillen);
-        System.arraycopy(buffer, 0, this.buffer, taillen, length-taillen);
-        this.start = 0;
-      }
-
-      super.reset(this.buffer, this.start, this.length);
-    }
-  }
-
-  InMemValBytes createInMemValBytes() {
-    return new InMemValBytes(bufvoid);
-  }
-
-  protected class MRResultIterator implements TezRawKeyValueIterator {
-    private final DataInputBuffer keybuf = new DataInputBuffer();
-    private final InMemValBytes vbytes = createInMemValBytes();
-    private final int end;
-    private int current;
-    public MRResultIterator(int start, int end) {
-      this.end = end;
-      current = start - 1;
-    }
-    public boolean next() throws IOException {
-      return ++current < end;
-    }
-    public DataInputBuffer getKey() throws IOException {
-      final int kvoff = offsetFor(current);
-      keybuf.reset(kvbuffer, kvmeta.get(kvoff + KEYSTART),
-          kvmeta.get(kvoff + VALSTART) - kvmeta.get(kvoff + KEYSTART));
-      return keybuf;
-    }
-    public DataInputBuffer getValue() throws IOException {
-      getVBytesForOffset(offsetFor(current), vbytes);
-      return vbytes;
-    }
-    public Progress getProgress() {
-      return null;
-    }
-    public void close() { }
-  }
-
-  private void mergeParts() throws IOException {
-    // get the approximate size of the final output/index files
-    long finalOutFileSize = 0;
-    long finalIndexFileSize = 0;
-    final Path[] filename = new Path[numSpills];
-    final String taskIdentifier = outputContext.getUniqueIdentifier();
-
-    for(int i = 0; i < numSpills; i++) {
-      filename[i] = mapOutputFile.getSpillFile(i);
-      finalOutFileSize += rfs.getFileStatus(filename[i]).getLen();
-    }
-    if (numSpills == 1) { //the spill is the final output
-      sameVolRename(filename[0],
-          mapOutputFile.getOutputFileForWriteInVolume(filename[0]));
-      if (indexCacheList.size() == 0) {
-        sameVolRename(mapOutputFile.getSpillIndexFile(0),
-          mapOutputFile.getOutputIndexFileForWriteInVolume(filename[0]));
-      } else {
-        indexCacheList.get(0).writeToFile(
-          mapOutputFile.getOutputIndexFileForWriteInVolume(filename[0]), conf);
-      }
-      return;
-    }
-
-    // read in paged indices
-    for (int i = indexCacheList.size(); i < numSpills; ++i) {
-      Path indexFileName = mapOutputFile.getSpillIndexFile(i);
-      indexCacheList.add(new TezSpillRecord(indexFileName, conf));
-    }
-
-    //make correction in the length to include the sequence file header
-    //lengths for each partition
-    finalOutFileSize += partitions * APPROX_HEADER_LENGTH;
-    finalIndexFileSize = partitions * MAP_OUTPUT_INDEX_RECORD_LENGTH;
-    Path finalOutputFile =
-        mapOutputFile.getOutputFileForWrite(finalOutFileSize);
-    Path finalIndexFile =
-        mapOutputFile.getOutputIndexFileForWrite(finalIndexFileSize);
-
-    //The output stream for the final single output file
-    FSDataOutputStream finalOut = rfs.create(finalOutputFile, true, 4096);
-
-    if (numSpills == 0) {
-      //create dummy files
-
-      TezSpillRecord sr = new TezSpillRecord(partitions);
-      try {
-        for (int i = 0; i < partitions; i++) {
-          long segmentStart = finalOut.getPos();
-          Writer writer =
-            new Writer(conf, finalOut, keyClass, valClass, codec, null);
-          writer.close();
-
-          TezIndexRecord rec =
-              new TezIndexRecord(
-                  segmentStart,
-                  writer.getRawLength(),
-                  writer.getCompressedLength());
-          sr.putIndex(rec, i);
-        }
-        sr.writeToFile(finalIndexFile, conf);
-      } finally {
-        finalOut.close();
-      }
-      return;
-    }
-    else {
-      TezMerger.considerFinalMergeForProgress();
-
-      final TezSpillRecord spillRec = new TezSpillRecord(partitions);
-      for (int parts = 0; parts < partitions; parts++) {
-        //create the segments to be merged
-        List<Segment> segmentList =
-          new ArrayList<Segment>(numSpills);
-        for(int i = 0; i < numSpills; i++) {
-          TezIndexRecord indexRecord = indexCacheList.get(i).getIndex(parts);
-
-          Segment s =
-            new Segment(conf, rfs, filename[i], indexRecord.getStartOffset(),
-                             indexRecord.getPartLength(), codec, true);
-          segmentList.add(i, s);
-
-          if (LOG.isDebugEnabled()) {
-            LOG.debug("TaskIdentifier=" + taskIdentifier + " Partition=" + parts +
-                "Spill =" + i + "(" + indexRecord.getStartOffset() + "," +
-                indexRecord.getRawLength() + ", " +
-                indexRecord.getPartLength() + ")");
-          }
-        }
-
-        int mergeFactor =
-            this.conf.getInt(TezJobConfig.TEZ_ENGINE_IO_SORT_FACTOR,
-                TezJobConfig.DEFAULT_TEZ_ENGINE_IO_SORT_FACTOR);
-        // sort the segments only if there are intermediate merges
-        boolean sortSegments = segmentList.size() > mergeFactor;
-        //merge
-        TezRawKeyValueIterator kvIter = TezMerger.merge(conf, rfs,
-                       keyClass, valClass, codec,
-                       segmentList, mergeFactor,
-                       new Path(taskIdentifier),
-                       (RawComparator)ConfigUtils.getIntermediateOutputKeyComparator(conf),
-                       nullProgressable, sortSegments,
-                       null, spilledRecordsCounter,
-                       null); // Not using any Progress in TezMerger. Should just work.
-
-        //write merged output to disk
-        long segmentStart = finalOut.getPos();
-        Writer writer =
-            new Writer(conf, finalOut, keyClass, valClass, codec,
-                spilledRecordsCounter);
-        if (combiner == null || numSpills < minSpillsForCombine) {
-          TezMerger.writeFile(kvIter, writer,
-              nullProgressable, conf);
-        } else {
-          runCombineProcessor(kvIter, writer);
-        }
-        writer.close();
-
-        // record offsets
-        final TezIndexRecord rec =
-            new TezIndexRecord(
-                segmentStart,
-                writer.getRawLength(),
-                writer.getCompressedLength());
-        spillRec.putIndex(rec, parts);
-      }
-      spillRec.writeToFile(finalIndexFile, conf);
-      finalOut.close();
-      for(int i = 0; i < numSpills; i++) {
-        rfs.delete(filename[i],true);
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/dflt/InMemoryShuffleSorter.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/dflt/InMemoryShuffleSorter.java b/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/dflt/InMemoryShuffleSorter.java
deleted file mode 100644
index e2b3315..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/dflt/InMemoryShuffleSorter.java
+++ /dev/null
@@ -1,126 +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.engine.common.sort.impl.dflt;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.nio.IntBuffer;
-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.io.WritableUtils;
-import org.apache.hadoop.util.DataChecksum;
-import org.apache.tez.engine.api.TezOutputContext;
-import org.apache.tez.engine.common.shuffle.impl.ShuffleHeader;
-import org.apache.tez.engine.common.shuffle.server.ShuffleHandler;
-import org.apache.tez.engine.common.sort.impl.IFile;
-
-public class InMemoryShuffleSorter extends DefaultSorter {
-
-  private static final Log LOG = LogFactory.getLog(InMemoryShuffleSorter.class);
-  
-  static final int IFILE_EOF_LENGTH = 
-      2 * WritableUtils.getVIntSize(IFile.EOF_MARKER);
-  static final int IFILE_CHECKSUM_LENGTH = DataChecksum.Type.CRC32.size;
-  
-  private List<Integer> spillIndices = new ArrayList<Integer>();
-  private List<ShuffleHeader> shuffleHeaders = new ArrayList<ShuffleHeader>();
-
-  ShuffleHandler shuffleHandler = new ShuffleHandler(this);
-  
-  byte[] kvbuffer;
-  IntBuffer kvmeta;
-
-  @Override
-  public void initialize(TezOutputContext outputContext, Configuration conf, int numOutputs) throws IOException {
-    super.initialize(outputContext, conf, numOutputs);
-    shuffleHandler.initialize(outputContext, conf);
-  }
-
-  @Override
-  protected void spill(int mstart, int mend) 
-      throws IOException, InterruptedException {
-    // Start the shuffleHandler
-    shuffleHandler.start();
-
-    // Don't spill!
-    
-    // Make a copy
-    this.kvbuffer = super.kvbuffer;
-    this.kvmeta = super.kvmeta;
-
-    // Just save spill-indices for serving later
-    int spindex = mstart;
-    for (int i = 0; i < partitions; ++i) {
-      spillIndices.add(spindex);
-      
-      int length = 0;
-      while (spindex < mend &&
-          kvmeta.get(offsetFor(spindex) + PARTITION) == i) {
-
-        final int kvoff = offsetFor(spindex);
-        int keyLen = 
-            kvmeta.get(kvoff + VALSTART) - kvmeta.get(kvoff + KEYSTART);
-        int valLen = getInMemVBytesLength(kvoff);
-        length += 
-            (keyLen + WritableUtils.getVIntSize(keyLen)) + 
-            (valLen + WritableUtils.getVIntSize(valLen));
-
-        ++spindex;
-      }
-      length += IFILE_EOF_LENGTH;
-      
-      shuffleHeaders.add( 
-          new ShuffleHeader(
-              outputContext.getUniqueIdentifier(), // TODO Verify that this is correct. 
-              length + IFILE_CHECKSUM_LENGTH, length, i)
-          );
-      LOG.info("shuffleHeader[" + i + "]:" +
-      		" rawLen=" + length + " partLen=" + (length + IFILE_CHECKSUM_LENGTH) + 
-          " spillIndex=" + spillIndices.get(i));
-    }
-    
-    LOG.info("Saved " + spillIndices.size() + " spill-indices and " + 
-        shuffleHeaders.size() + " shuffle headers");
-  }
-
-  @Override
-  public InputStream getSortedStream(int partition) {
-    return new SortBufferInputStream(this, partition);
-  }
-
-  @Override
-  public void close() throws IOException {
-    // FIXME
-    //shuffleHandler.stop();
-  }
-
-  @Override
-  public ShuffleHeader getShuffleHeader(int reduce) {
-    return shuffleHeaders.get(reduce);
-  }
-
-  public int getSpillIndex(int partition) {
-    return spillIndices.get(partition);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/dflt/SortBufferInputStream.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/dflt/SortBufferInputStream.java b/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/dflt/SortBufferInputStream.java
deleted file mode 100644
index d74e159..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/dflt/SortBufferInputStream.java
+++ /dev/null
@@ -1,271 +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.engine.common.sort.impl.dflt;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.nio.IntBuffer;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.io.BoundedByteArrayOutputStream;
-import org.apache.hadoop.io.DataInputBuffer;
-import org.apache.hadoop.io.WritableUtils;
-import org.apache.tez.engine.common.shuffle.impl.InMemoryWriter;
-import org.apache.tez.engine.common.sort.impl.dflt.DefaultSorter.InMemValBytes;
-
-  public class SortBufferInputStream extends InputStream {
-
-  private static final Log LOG = LogFactory.getLog(SortBufferInputStream.class);
-  
-  private final InMemoryShuffleSorter sorter;
-  private InMemoryWriter sortOutput;
-  
-  private int mend;
-  private int recIndex;
-  private final byte[] kvbuffer;       
-  private final IntBuffer kvmeta;
-  private final int partitionBytes;
-  private final int partition;
-  
-  byte[] dualBuf = new byte[8192];
-  DualBufferOutputStream out;
-  private int readBytes = 0;
-  
-  public SortBufferInputStream(
-      InMemoryShuffleSorter sorter, int partition) {
-    this.sorter = sorter;
-    this.partitionBytes = 
-        (int)sorter.getShuffleHeader(partition).getCompressedLength();
-    this.partition = partition;
-    this.mend = sorter.getMetaEnd();
-    this.recIndex = sorter.getSpillIndex(partition);
-    this.kvbuffer = sorter.kvbuffer;
-    this.kvmeta = sorter.kvmeta;
-    out = new DualBufferOutputStream(null, 0, 0, dualBuf);
-    sortOutput = new InMemoryWriter(out);
-  }
-  
-  byte[] one = new byte[1];
-  
-  @Override
-  public int read() throws IOException {
-    int b = read(one, 0, 1);
-    return (b == -1) ? b : one[0]; 
-  }
-
-  @Override
-  public int read(byte[] b) throws IOException {
-    return read(b, 0, b.length);
-  }
-
-  @Override
-  public int read(byte[] b, int off, int len) throws IOException {
-    if (available() == 0) {
-      return -1;
-    }
-    
-    int currentOffset = off;
-    int currentLength = len;
-    int currentReadBytes = 0;
-    
-    // Check if there is residual data in the dualBuf
-    int residualLen = out.getCurrent();
-    if (residualLen > 0) {
-      int readable = Math.min(currentLength, residualLen);
-      System.arraycopy(dualBuf, 0, b, currentOffset, readable);
-      currentOffset += readable;
-      currentReadBytes += readable;
-      out.setCurrentPointer(-readable);
-      
-      // buffer has less capacity
-      currentLength -= readable;
-      
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("XXX read_residual:" +
-            " readable=" + readable +
-            " readBytes=" + readBytes);
-      }
-    }
-    
-    // Now, use the provided buffer
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("XXX read: out.reset" +
-          " b=" + b + 
-          " currentOffset=" + currentOffset + 
-          " currentLength=" + currentLength +
-          " recIndex=" + recIndex);
-    }
-    out.reset(b, currentOffset, currentLength);
-    
-    // Read from sort-buffer into the provided buffer, space permitting
-    DataInputBuffer key = new DataInputBuffer();
-    final InMemValBytes value = sorter.createInMemValBytes();
-    
-    int kvPartition = 0;
-    int numRec = 0;
-    for (;
-         currentLength > 0 && recIndex < mend && 
-             (kvPartition = getKVPartition(recIndex)) == partition;
-        ++recIndex) {
-      
-      final int kvoff = sorter.offsetFor(recIndex);
-      
-      int keyLen = 
-          (kvmeta.get(kvoff + InMemoryShuffleSorter.VALSTART) - 
-              kvmeta.get(kvoff + InMemoryShuffleSorter.KEYSTART));
-      key.reset(
-          kvbuffer, 
-          kvmeta.get(kvoff + InMemoryShuffleSorter.KEYSTART),
-          keyLen
-          );
-      
-      int valLen = sorter.getVBytesForOffset(kvoff, value);
-
-      int recLen = 
-          (keyLen + WritableUtils.getVIntSize(keyLen)) + 
-          (valLen + WritableUtils.getVIntSize(valLen));
-      
-      currentReadBytes += recLen;
-      currentOffset += recLen;
-      currentLength -= recLen;
-
-      // Write out key/value into the in-mem ifile
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("XXX read: sortOutput.append" +
-            " #rec=" + ++numRec +
-            " recIndex=" + recIndex + " kvoff=" + kvoff + 
-            " keyLen=" + keyLen + " valLen=" + valLen + " recLen=" + recLen +
-            " readBytes=" + readBytes +
-            " currentReadBytes="  + currentReadBytes +
-            " currentLength=" + currentLength);
-      }
-      sortOutput.append(key, value);
-    }
-
-    // If we are at the end of the segment, close the ifile
-    if (currentLength > 0 && 
-        (recIndex == mend || kvPartition != partition)) {
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("XXX About to call close:" +
-            " currentLength=" + currentLength + 
-            " recIndex=" + recIndex + " mend=" + mend + 
-            " kvPartition=" + kvPartition + " partitino=" + partition);
-      }
-      sortOutput.close();
-      currentReadBytes += 
-          (InMemoryShuffleSorter.IFILE_EOF_LENGTH + 
-              InMemoryShuffleSorter.IFILE_CHECKSUM_LENGTH);
-    } else {
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("XXX Hmm..." +
-            " currentLength=" + currentLength + 
-            " recIndex=" + recIndex + " mend=" + mend + 
-            " kvPartition=" + kvPartition + " partitino=" + partition);
-      }
-    }
-    
-    int retVal = Math.min(currentReadBytes, len);
-    readBytes += retVal;
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("XXX read: done" +
-          " retVal=" + retVal + 
-          " currentReadBytes=" + currentReadBytes +
-          " len=" + len + 
-          " readBytes=" + readBytes +
-          " partitionBytes=" + partitionBytes +
-          " residualBytes=" + out.getCurrent());
-    }
-    return retVal;
-  }
-
-  private int getKVPartition(int recIndex) {
-    return kvmeta.get(
-        sorter.offsetFor(recIndex) + InMemoryShuffleSorter.PARTITION);
-  }
-  
-  @Override
-  public int available() throws IOException {
-    return (partitionBytes - readBytes);
-  }
-
-  @Override
-  public void close() throws IOException {
-    super.close();
-  }
-
-  @Override
-  public boolean markSupported() {
-    return false;
-  }
-  
-  static class DualBufferOutputStream extends BoundedByteArrayOutputStream {
-
-    byte[] dualBuf;
-    int currentPointer = 0;
-    byte[] one = new byte[1];
-    
-    public DualBufferOutputStream(
-        byte[] buf, int offset, int length, 
-        byte[] altBuf) {
-      super(buf, offset, length);
-      this.dualBuf = altBuf;
-    }
-    
-    public void reset(byte[] b, int off, int len) {
-      super.resetBuffer(b, off, len);
-    }
-
-    @Override
-    public void write(int b) throws IOException {
-      one[0] = (byte)b;
-      write(one, 0, 1);
-    }
-
-    @Override
-    public void write(byte[] b) throws IOException {
-      write(b, 0, b.length);
-    }
-
-    @Override
-    public void write(byte[] b, int off, int len) throws IOException {
-      int available = super.available();
-      if (available >= len) {
-        super.write(b, off, len);
-      } else {
-        super.write(b, off, available);
-        System.arraycopy(b, off+available, dualBuf, currentPointer, len-available);
-        currentPointer += (len - available);
-      }
-    }
-    
-    int getCurrent() {
-      return currentPointer;
-    }
-    
-    void setCurrentPointer(int delta) {
-      if ((currentPointer + delta) > dualBuf.length) {
-        throw new IndexOutOfBoundsException("Trying to set dualBuf 'current'" +
-        		" marker to " + (currentPointer+delta) + " when " +
-        		" dualBuf.length is " + dualBuf.length);
-      }
-      currentPointer = (currentPointer + delta) % dualBuf.length;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/engine/common/task/impl/ValuesIterator.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/task/impl/ValuesIterator.java b/tez-engine/src/main/java/org/apache/tez/engine/common/task/impl/ValuesIterator.java
deleted file mode 100644
index 841e54d..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/task/impl/ValuesIterator.java
+++ /dev/null
@@ -1,149 +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.engine.common.task.impl;
-
-import java.io.IOException;
-import java.util.Iterator;
-import java.util.NoSuchElementException;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.io.DataInputBuffer;
-import org.apache.hadoop.io.RawComparator;
-import org.apache.hadoop.io.serializer.Deserializer;
-import org.apache.hadoop.io.serializer.SerializationFactory;
-import org.apache.hadoop.util.Progressable;
-import org.apache.tez.engine.common.sort.impl.TezRawKeyValueIterator;
-
-
-/**
- * Iterates values while keys match in sorted input.
- *
- * Usage: Call moveToNext to move to the next k, v pair. This returns true if another exists,
- * followed by getKey() and getValues() to get the current key and list of values.
- * 
- */
-public class ValuesIterator<KEY,VALUE> implements Iterator<VALUE> {
-  protected TezRawKeyValueIterator in; //input iterator
-  private KEY key;               // current key
-  private KEY nextKey;
-  private VALUE value;             // current value
-  private boolean hasNext;                      // more w/ this key
-  private boolean more;                         // more in file
-  private RawComparator<KEY> comparator;
-  protected Progressable reporter;
-  private Deserializer<KEY> keyDeserializer;
-  private Deserializer<VALUE> valDeserializer;
-  private DataInputBuffer keyIn = new DataInputBuffer();
-  private DataInputBuffer valueIn = new DataInputBuffer();
-  
-  public ValuesIterator (TezRawKeyValueIterator in, 
-                         RawComparator<KEY> comparator, 
-                         Class<KEY> keyClass,
-                         Class<VALUE> valClass, Configuration conf, 
-                         Progressable reporter)
-    throws IOException {
-    this.in = in;
-    this.comparator = comparator;
-    this.reporter = reporter;
-    SerializationFactory serializationFactory = new SerializationFactory(conf);
-    this.keyDeserializer = serializationFactory.getDeserializer(keyClass);
-    this.keyDeserializer.open(keyIn);
-    this.valDeserializer = serializationFactory.getDeserializer(valClass);
-    this.valDeserializer.open(this.valueIn);
-    readNextKey();
-    key = nextKey;
-    nextKey = null; // force new instance creation
-    hasNext = more;
-  }
-
-  TezRawKeyValueIterator getRawIterator() { return in; }
-  
-  /// Iterator methods
-
-  public boolean hasNext() { return hasNext; }
-
-  private int ctr = 0;
-  public VALUE next() {
-    if (!hasNext) {
-      throw new NoSuchElementException("iterate past last value");
-    }
-    try {
-      readNextValue();
-      readNextKey();
-    } catch (IOException ie) {
-      throw new RuntimeException("problem advancing post rec#"+ctr, ie);
-    }
-    reporter.progress();
-    return value;
-  }
-
-  public void remove() { throw new RuntimeException("not implemented"); }
-
-  /// Auxiliary methods
-
-  /** Start processing next unique key. */
-  public void nextKey() throws IOException {
-    // read until we find a new key
-    while (hasNext) { 
-      readNextKey();
-    }
-    ++ctr;
-    
-    // move the next key to the current one
-    KEY tmpKey = key;
-    key = nextKey;
-    nextKey = tmpKey;
-    hasNext = more;
-  }
-
-  /** True iff more keys remain. */
-  public boolean more() { 
-    return more; 
-  }
-
-  /** The current key. */
-  public KEY getKey() { 
-    return key; 
-  }
-
-  /** 
-   * read the next key 
-   */
-  private void readNextKey() throws IOException {
-    more = in.next();
-    if (more) {
-      DataInputBuffer nextKeyBytes = in.getKey();
-      keyIn.reset(nextKeyBytes.getData(), nextKeyBytes.getPosition(), nextKeyBytes.getLength());
-      nextKey = keyDeserializer.deserialize(nextKey);
-      hasNext = key != null && (comparator.compare(key, nextKey) == 0);
-    } else {
-      hasNext = false;
-    }
-  }
-
-  /**
-   * Read the next value
-   * @throws IOException
-   */
-  private void readNextValue() throws IOException {
-    DataInputBuffer nextValueBytes = in.getValue();
-    valueIn.reset(nextValueBytes.getData(), nextValueBytes.getPosition(), nextValueBytes.getLength());
-    value = valDeserializer.deserialize(value);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/engine/common/task/local/output/TezLocalTaskOutputFiles.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/task/local/output/TezLocalTaskOutputFiles.java b/tez-engine/src/main/java/org/apache/tez/engine/common/task/local/output/TezLocalTaskOutputFiles.java
deleted file mode 100644
index 40e6b1a..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/task/local/output/TezLocalTaskOutputFiles.java
+++ /dev/null
@@ -1,249 +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.engine.common.task.local.output;
-
-import java.io.IOException;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-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.tez.common.Constants;
-import org.apache.tez.common.TezJobConfig;
-import org.apache.tez.engine.common.InputAttemptIdentifier;
-
-/**
- * Manipulate the working area for the transient store for maps and reduces.
- *
- * This class is used by map and reduce tasks to identify the directories that
- * they need to write to/read from for intermediate files. The callers of
- * these methods are from the Child running the Task.
- */
-@InterfaceAudience.Private
-@InterfaceStability.Unstable
-public class TezLocalTaskOutputFiles extends TezTaskOutput {
-
-  public TezLocalTaskOutputFiles(Configuration conf, String uniqueId) {
-    super(conf, uniqueId);
-  }
-
-  private LocalDirAllocator lDirAlloc =
-    new LocalDirAllocator(TezJobConfig.LOCAL_DIRS);
-
-
-  /**
-   * Return the path to local map output file created earlier
-   *
-   * @return path
-   * @throws IOException
-   */
-  @Override
-  public Path getOutputFile()
-      throws IOException {
-    return lDirAlloc.getLocalPathToRead(Constants.TASK_OUTPUT_DIR + Path.SEPARATOR
-        + Constants.TEZ_ENGINE_TASK_OUTPUT_FILENAME_STRING, conf);
-  }
-
-  /**
-   * Create a local map output file name.
-   *
-   * @param size the size of the file
-   * @return path
-   * @throws IOException
-   */
-  @Override
-  public Path getOutputFileForWrite(long size)
-      throws IOException {
-    return lDirAlloc.getLocalPathForWrite(Constants.TASK_OUTPUT_DIR + Path.SEPARATOR
-        + Constants.TEZ_ENGINE_TASK_OUTPUT_FILENAME_STRING, size, conf);
-  }
-  
-  /**
-   * Create a local map output file name. This should *only* be used if the size
-   * of the file is not known. Otherwise use the equivalent which accepts a size
-   * parameter.
-   * 
-   * @return path
-   * @throws IOException
-   */
-  @Override
-  public Path getOutputFileForWrite() throws IOException {
-    return lDirAlloc.getLocalPathForWrite(Constants.TASK_OUTPUT_DIR
-        + Path.SEPARATOR + Constants.TEZ_ENGINE_TASK_OUTPUT_FILENAME_STRING,
-        conf);
-  }
-
-  /**
-   * Create a local map output file name on the same volume.
-   */
-  @Override
-  public Path getOutputFileForWriteInVolume(Path existing) {
-    return new Path(existing.getParent(), Constants.TEZ_ENGINE_TASK_OUTPUT_FILENAME_STRING);
-  }
-
-  /**
-   * Return the path to a local map output index file created earlier
-   *
-   * @return path
-   * @throws IOException
-   */
-  @Override
-  public Path getOutputIndexFile()
-      throws IOException {
-    return lDirAlloc.getLocalPathToRead(Constants.TASK_OUTPUT_DIR + Path.SEPARATOR
-        + Constants.TEZ_ENGINE_TASK_OUTPUT_FILENAME_STRING + Constants.TEZ_ENGINE_TASK_OUTPUT_INDEX_SUFFIX_STRING,
-        conf);
-  }
-
-  /**
-   * Create a local map output index file name.
-   *
-   * @param size the size of the file
-   * @return path
-   * @throws IOException
-   */
-  @Override
-  public Path getOutputIndexFileForWrite(long size)
-      throws IOException {
-    return lDirAlloc.getLocalPathForWrite(Constants.TASK_OUTPUT_DIR + Path.SEPARATOR
-        + Constants.TEZ_ENGINE_TASK_OUTPUT_FILENAME_STRING + Constants.TEZ_ENGINE_TASK_OUTPUT_INDEX_SUFFIX_STRING,
-        size, conf);
-  }
-
-  /**
-   * Create a local map output index file name on the same volume.
-   */
-  @Override
-  public Path getOutputIndexFileForWriteInVolume(Path existing) {
-    return new Path(existing.getParent(),
-        Constants.TEZ_ENGINE_TASK_OUTPUT_FILENAME_STRING + Constants.TEZ_ENGINE_TASK_OUTPUT_INDEX_SUFFIX_STRING);
-  }
-
-  /**
-   * Return a local map spill file created earlier.
-   *
-   * @param spillNumber the number
-   * @return path
-   * @throws IOException
-   */
-  @Override
-  public Path getSpillFile(int spillNumber)
-      throws IOException {
-    return lDirAlloc.getLocalPathToRead(Constants.TASK_OUTPUT_DIR + "/spill"
-        + spillNumber + ".out", conf);
-  }
-
-  /**
-   * Create a local map spill file name.
-   *
-   * @param spillNumber the number
-   * @param size the size of the file
-   * @return path
-   * @throws IOException
-   */
-  @Override
-  public Path getSpillFileForWrite(int spillNumber, long size)
-      throws IOException {
-    return lDirAlloc.getLocalPathForWrite(Constants.TASK_OUTPUT_DIR + "/spill"
-        + spillNumber + ".out", size, conf);
-  }
-
-  /**
-   * Return a local map spill index file created earlier
-   *
-   * @param spillNumber the number
-   * @return path
-   * @throws IOException
-   */
-  @Override
-  public Path getSpillIndexFile(int spillNumber)
-      throws IOException {
-    return lDirAlloc.getLocalPathToRead(Constants.TASK_OUTPUT_DIR + "/spill"
-        + spillNumber + ".out.index", conf);
-  }
-
-  /**
-   * Create a local map spill index file name.
-   *
-   * @param spillNumber the number
-   * @param size the size of the file
-   * @return path
-   * @throws IOException
-   */
-  @Override
-  public Path getSpillIndexFileForWrite(int spillNumber, long size)
-      throws IOException {
-    return lDirAlloc.getLocalPathForWrite(Constants.TASK_OUTPUT_DIR + "/spill"
-        + spillNumber + ".out.index", size, conf);
-  }
-
-  /**
-   * Return a local reduce input file created earlier
-   *
-   * @param mapId a map task id
-   * @return path
-   * @throws IOException
-   */
-  @Override
-  public Path getInputFile(InputAttemptIdentifier mapId)
-      throws IOException {
-    return lDirAlloc.getLocalPathToRead(String.format(
-        Constants.TEZ_ENGINE_TASK_INPUT_FILE_FORMAT_STRING, 
-        Constants.TASK_OUTPUT_DIR, Integer.valueOf(mapId.getInputIdentifier().getSrcTaskIndex())), conf);
-  }
-
-  /**
-   * Create a local reduce input file name.
-   *
-   * @param mapId a map task id
-   * @param size the size of the file
-   * @return path
-   * @throws IOException
-   */
-  @Override
-  public Path getInputFileForWrite(int taskId,
-                                   long size)
-      throws IOException {
-    return lDirAlloc.getLocalPathForWrite(String.format(
-        Constants.TEZ_ENGINE_TASK_INPUT_FILE_FORMAT_STRING, Constants.TASK_OUTPUT_DIR, taskId),
-        size, conf);
-  }
-
-  /** Removes all of the files related to a task. */
-  @Override
-  public void removeAll()
-      throws IOException {
-    deleteLocalFiles(Constants.TASK_OUTPUT_DIR);
-  }
-
-  private String[] getLocalDirs() throws IOException {
-    return conf.getStrings(TezJobConfig.LOCAL_DIRS);
-  }
-
-  @SuppressWarnings("deprecation")
-  private void deleteLocalFiles(String subdir) throws IOException {
-    String[] localDirs = getLocalDirs();
-    for (int i = 0; i < localDirs.length; i++) {
-      FileSystem.getLocal(conf).delete(new Path(localDirs[i], subdir));
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/engine/common/task/local/output/TezTaskOutput.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/task/local/output/TezTaskOutput.java b/tez-engine/src/main/java/org/apache/tez/engine/common/task/local/output/TezTaskOutput.java
deleted file mode 100644
index e1d83ad..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/task/local/output/TezTaskOutput.java
+++ /dev/null
@@ -1,165 +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.engine.common.task.local.output;
-
-import java.io.IOException;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.Path;
-import org.apache.tez.engine.common.InputAttemptIdentifier;
-
-/**
- * Manipulate the working area for the transient store for maps and reduces.
- *
- * This class is used by map and reduce tasks to identify the directories that
- * they need to write to/read from for intermediate files. The callers of
- * these methods are from child space and see mapreduce.cluster.local.dir as
- * taskTracker/jobCache/jobId/attemptId
- * This class should not be used from TaskTracker space.
- */
-@InterfaceAudience.Private
-@InterfaceStability.Unstable
-public abstract class TezTaskOutput {
-
-  protected Configuration conf;
-  protected String uniqueId;
-
-  public TezTaskOutput(Configuration conf, String uniqueId) {
-    this.conf = conf;
-    this.uniqueId = uniqueId;
-  }
-
-  /**
-   * Return the path to local map output file created earlier
-   *
-   * @return path
-   * @throws IOException
-   */
-  public abstract Path getOutputFile() throws IOException;
-
-  /**
-   * Create a local map output file name.
-   *
-   * @param size the size of the file
-   * @return path
-   * @throws IOException
-   */
-  public abstract Path getOutputFileForWrite(long size) throws IOException;
-
-  /**
-   * Create a local output file name. This method is meant to be used *only* if
-   * the size of the file is not know up front.
-   * 
-   * @return path
-   * @throws IOException
-   */
-  public abstract Path getOutputFileForWrite() throws IOException;
-  
-  /**
-   * Create a local map output file name on the same volume.
-   */
-  public abstract Path getOutputFileForWriteInVolume(Path existing);
-
-  /**
-   * Return the path to a local map output index file created earlier
-   *
-   * @return path
-   * @throws IOException
-   */
-  public abstract Path getOutputIndexFile() throws IOException;
-
-  /**
-   * Create a local map output index file name.
-   *
-   * @param size the size of the file
-   * @return path
-   * @throws IOException
-   */
-  public abstract Path getOutputIndexFileForWrite(long size) throws IOException;
-
-  /**
-   * Create a local map output index file name on the same volume.
-   */
-  public abstract Path getOutputIndexFileForWriteInVolume(Path existing);
-
-  /**
-   * Return a local map spill file created earlier.
-   *
-   * @param spillNumber the number
-   * @return path
-   * @throws IOException
-   */
-  public abstract Path getSpillFile(int spillNumber) throws IOException;
-
-  /**
-   * Create a local map spill file name.
-   *
-   * @param spillNumber the number
-   * @param size the size of the file
-   * @return path
-   * @throws IOException
-   */
-  public abstract Path getSpillFileForWrite(int spillNumber, long size)
-      throws IOException;
-
-  /**
-   * Return a local map spill index file created earlier
-   *
-   * @param spillNumber the number
-   * @return path
-   * @throws IOException
-   */
-  public abstract Path getSpillIndexFile(int spillNumber) throws IOException;
-
-  /**
-   * Create a local map spill index file name.
-   *
-   * @param spillNumber the number
-   * @param size the size of the file
-   * @return path
-   * @throws IOException
-   */
-  public abstract Path getSpillIndexFileForWrite(int spillNumber, long size)
-      throws IOException;
-
-  /**
-   * Return a local reduce input file created earlier
-   *
-   * @param attemptIdentifier The identifier for the source task
-   * @return path
-   * @throws IOException
-   */
-  public abstract Path getInputFile(InputAttemptIdentifier attemptIdentifier) throws IOException;
-
-  /**
-   * Create a local reduce input file name.
-   *
-   * @param taskIdentifier The identifier for the source task
-   * @param size the size of the file
-   * @return path
-   * @throws IOException
-   */
-  public abstract Path getInputFileForWrite(
-      int taskIdentifier, long size) throws IOException;
-
-  /** Removes all of the files related to a task. */
-  public abstract void removeAll() throws IOException;
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/engine/common/task/local/output/TezTaskOutputFiles.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/task/local/output/TezTaskOutputFiles.java b/tez-engine/src/main/java/org/apache/tez/engine/common/task/local/output/TezTaskOutputFiles.java
deleted file mode 100644
index b8f051b..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/task/local/output/TezTaskOutputFiles.java
+++ /dev/null
@@ -1,246 +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.engine.common.task.local.output;
-
-import java.io.IOException;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.LocalDirAllocator;
-import org.apache.hadoop.fs.Path;
-import org.apache.tez.common.Constants;
-import org.apache.tez.common.TezJobConfig;
-import org.apache.tez.engine.common.InputAttemptIdentifier;
-
-/**
- * Manipulate the working area for the transient store for maps and reduces.
- *
- * This class is used by map and reduce tasks to identify the directories that
- * they need to write to/read from for intermediate files. The callers of
- * these methods are from child space and see mapreduce.cluster.local.dir as
- * taskTracker/jobCache/jobId/attemptId
- * This class should not be used from TaskTracker space.
- */
-
-@InterfaceAudience.Private
-@InterfaceStability.Unstable
-public class TezTaskOutputFiles extends TezTaskOutput {
-  
-  public TezTaskOutputFiles(Configuration conf, String uniqueId) {
-    super(conf, uniqueId);
-  }
-
-  private static final Log LOG = LogFactory.getLog(TezTaskOutputFiles.class);
-
-  private static final String SPILL_FILE_PATTERN = "%s_spill_%d.out";
-  private static final String SPILL_INDEX_FILE_PATTERN = SPILL_FILE_PATTERN
-      + ".index";
-
-  
-
-  // assume configured to $localdir/usercache/$user/appcache/$appId
-  private LocalDirAllocator lDirAlloc =
-    new LocalDirAllocator(TezJobConfig.LOCAL_DIRS);
-  
-
-  private Path getAttemptOutputDir() {
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("getAttemptOutputDir: "
-          + Constants.TASK_OUTPUT_DIR + "/"
-          + uniqueId);
-    }
-    return new Path(Constants.TASK_OUTPUT_DIR, uniqueId);
-  }
-
-  /**
-   * Return the path to local map output file created earlier
-   *
-   * @return path
-   * @throws IOException
-   */
-  public Path getOutputFile() throws IOException {
-    Path attemptOutput =
-      new Path(getAttemptOutputDir(), Constants.TEZ_ENGINE_TASK_OUTPUT_FILENAME_STRING);
-    return lDirAlloc.getLocalPathToRead(attemptOutput.toString(), conf);
-  }
-
-  /**
-   * Create a local map output file name.
-   *
-   * @param size the size of the file
-   * @return path
-   * @throws IOException
-   */
-  public Path getOutputFileForWrite(long size) throws IOException {
-    Path attemptOutput =
-      new Path(getAttemptOutputDir(), Constants.TEZ_ENGINE_TASK_OUTPUT_FILENAME_STRING);
-    return lDirAlloc.getLocalPathForWrite(attemptOutput.toString(), size, conf);
-  }
-
-  /**
-   * Create a local map output file name. This should *only* be used if the size
-   * of the file is not known. Otherwise use the equivalent which accepts a size
-   * parameter.
-   * 
-   * @return path
-   * @throws IOException
-   */
-  public Path getOutputFileForWrite() throws IOException {
-    Path attemptOutput =
-      new Path(getAttemptOutputDir(), Constants.TEZ_ENGINE_TASK_OUTPUT_FILENAME_STRING);
-    return lDirAlloc.getLocalPathForWrite(attemptOutput.toString(), conf);
-  }
-
-  /**
-   * Create a local map output file name on the same volume.
-   */
-  public Path getOutputFileForWriteInVolume(Path existing) {
-    Path outputDir = new Path(existing.getParent(), Constants.TASK_OUTPUT_DIR);
-    Path attemptOutputDir = new Path(outputDir, uniqueId);
-    return new Path(attemptOutputDir, Constants.TEZ_ENGINE_TASK_OUTPUT_FILENAME_STRING);
-  }
-
-  /**
-   * Return the path to a local map output index file created earlier
-   *
-   * @return path
-   * @throws IOException
-   */
-  public Path getOutputIndexFile() throws IOException {
-    Path attemptIndexOutput =
-      new Path(getAttemptOutputDir(), Constants.TEZ_ENGINE_TASK_OUTPUT_FILENAME_STRING +
-                                      Constants.TEZ_ENGINE_TASK_OUTPUT_INDEX_SUFFIX_STRING);
-    return lDirAlloc.getLocalPathToRead(attemptIndexOutput.toString(), conf);
-  }
-
-  /**
-   * Create a local map output index file name.
-   *
-   * @param size the size of the file
-   * @return path
-   * @throws IOException
-   */
-  public Path getOutputIndexFileForWrite(long size) throws IOException {
-    Path attemptIndexOutput =
-      new Path(getAttemptOutputDir(), Constants.TEZ_ENGINE_TASK_OUTPUT_FILENAME_STRING +
-                                      Constants.TEZ_ENGINE_TASK_OUTPUT_INDEX_SUFFIX_STRING);
-    return lDirAlloc.getLocalPathForWrite(attemptIndexOutput.toString(),
-        size, conf);
-  }
-
-  /**
-   * Create a local map output index file name on the same volume.
-   */
-  public Path getOutputIndexFileForWriteInVolume(Path existing) {
-    Path outputDir = new Path(existing.getParent(), Constants.TASK_OUTPUT_DIR);
-    Path attemptOutputDir = new Path(outputDir, uniqueId);
-    return new Path(attemptOutputDir, Constants.TEZ_ENGINE_TASK_OUTPUT_FILENAME_STRING +
-                                      Constants.TEZ_ENGINE_TASK_OUTPUT_INDEX_SUFFIX_STRING);
-  }
-
-  /**
-   * Return a local map spill file created earlier.
-   *
-   * @param spillNumber the number
-   * @return path
-   * @throws IOException
-   */
-  public Path getSpillFile(int spillNumber) throws IOException {
-    return lDirAlloc.getLocalPathToRead(
-        String.format(SPILL_FILE_PATTERN,
-            uniqueId, spillNumber), conf);
-  }
-
-  /**
-   * Create a local map spill file name.
-   *
-   * @param spillNumber the number
-   * @param size the size of the file
-   * @return path
-   * @throws IOException
-   */
-  public Path getSpillFileForWrite(int spillNumber, long size)
-      throws IOException {
-    return lDirAlloc.getLocalPathForWrite(
-        String.format(String.format(SPILL_FILE_PATTERN,
-            uniqueId, spillNumber)), size, conf);
-  }
-
-  /**
-   * Return a local map spill index file created earlier
-   *
-   * @param spillNumber the number
-   * @return path
-   * @throws IOException
-   */
-  public Path getSpillIndexFile(int spillNumber) throws IOException {
-    return lDirAlloc.getLocalPathToRead(
-        String.format(SPILL_INDEX_FILE_PATTERN,
-            uniqueId, spillNumber), conf);
-  }
-
-  /**
-   * Create a local map spill index file name.
-   *
-   * @param spillNumber the number
-   * @param size the size of the file
-   * @return path
-   * @throws IOException
-   */
-  public Path getSpillIndexFileForWrite(int spillNumber, long size)
-      throws IOException {
-    return lDirAlloc.getLocalPathForWrite(
-        String.format(SPILL_INDEX_FILE_PATTERN,
-            uniqueId, spillNumber), size, conf);
-  }
-
-  /**
-   * Return a local reduce input file created earlier
-   *
-   * @param attemptIdentifier an identifier for a task. The attempt information is ignored.
-   * @return path
-   * @throws IOException
-   */
-  public Path getInputFile(InputAttemptIdentifier attemptIdentifier) throws IOException {
-    throw new UnsupportedOperationException("Incompatible with LocalRunner");
-  }
-
-  /**
-   * Create a local reduce input file name.
-   *
-   * @param attemptIdentifier an identifier for a task. The attempt information is ignored.
-   * @param size the size of the file
-   * @return path
-   * @throws IOException
-   */
-  public Path getInputFileForWrite(int srcTaskId,
-      long size) throws IOException {
-    return lDirAlloc.getLocalPathForWrite(String.format(
-        uniqueId, getAttemptOutputDir().toString(), srcTaskId),
-        size, conf);
-  }
-
-  /** Removes all of the files related to a task. */
-  public void removeAll() throws IOException {
-    throw new UnsupportedOperationException("Incompatible with LocalRunner");
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/engine/hadoop/compat/NullProgressable.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/hadoop/compat/NullProgressable.java b/tez-engine/src/main/java/org/apache/tez/engine/hadoop/compat/NullProgressable.java
deleted file mode 100644
index 5071dd2..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/hadoop/compat/NullProgressable.java
+++ /dev/null
@@ -1,33 +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.engine.hadoop.compat;
-
-import org.apache.hadoop.util.Progressable;
-
-public class NullProgressable implements Progressable {
-
-  public NullProgressable() {
-    // TODO Auto-generated constructor stub
-  }
-
-  @Override
-  public void progress() {
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/engine/lib/input/LocalMergedInput.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/lib/input/LocalMergedInput.java b/tez-engine/src/main/java/org/apache/tez/engine/lib/input/LocalMergedInput.java
deleted file mode 100644
index 6371787..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/lib/input/LocalMergedInput.java
+++ /dev/null
@@ -1,52 +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.engine.lib.input;
-
-import java.io.IOException;
-import java.util.Collections;
-import java.util.List;
-
-import org.apache.tez.common.TezUtils;
-import org.apache.tez.engine.api.Event;
-import org.apache.tez.engine.api.LogicalInput;
-import org.apache.tez.engine.api.TezInputContext;
-import org.apache.tez.engine.common.localshuffle.LocalShuffle;
-
-/**
- * <code>LocalMergedInput</code> in an {@link LogicalInput} which shuffles intermediate
- * sorted data, merges them and provides key/<values> to the consumer. 
- */
-public class LocalMergedInput extends ShuffledMergedInputLegacy {
-
-  @Override
-  public List<Event> initialize(TezInputContext inputContext) throws IOException {
-    this.inputContext = inputContext;
-    this.conf = TezUtils.createConfFromUserPayload(inputContext.getUserPayload());
-
-    LocalShuffle localShuffle = new LocalShuffle(inputContext, conf, numInputs);
-    rawIter = localShuffle.run();
-    createValuesIterator();
-    return Collections.emptyList();
-  }
-
-  @Override
-  public List<Event> close() throws IOException {
-    rawIter.close();
-    return Collections.emptyList();
-  }
-}


[34/50] [abbrv] Rename tez-engine-api to tez-runtime-api and tez-engine is split into 2: - tez-engine-library for user-visible Input/Output/Processor implementations - tez-engine-internals for framework internals

Posted by ss...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/broadcast/output/FileBasedKVWriter.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/broadcast/output/FileBasedKVWriter.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/broadcast/output/FileBasedKVWriter.java
new file mode 100644
index 0000000..8b19ce0
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/broadcast/output/FileBasedKVWriter.java
@@ -0,0 +1,125 @@
+/**
+* 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.library.broadcast.output;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.LocalFileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.compress.CompressionCodec;
+import org.apache.hadoop.io.compress.DefaultCodec;
+import org.apache.hadoop.util.ReflectionUtils;
+import org.apache.tez.common.TezJobConfig;
+import org.apache.tez.common.TezUtils;
+import org.apache.tez.runtime.api.TezOutputContext;
+import org.apache.tez.runtime.library.api.KVWriter;
+import org.apache.tez.runtime.library.common.ConfigUtils;
+import org.apache.tez.runtime.library.common.TezRuntimeUtils;
+import org.apache.tez.runtime.library.common.sort.impl.IFile;
+import org.apache.tez.runtime.library.common.sort.impl.TezIndexRecord;
+import org.apache.tez.runtime.library.common.sort.impl.TezSpillRecord;
+import org.apache.tez.runtime.library.common.task.local.output.TezTaskOutput;
+
+public class FileBasedKVWriter implements KVWriter {
+
+  public static final int INDEX_RECORD_LENGTH = 24;
+
+  private final Configuration conf;
+  private int numRecords = 0;
+
+  @SuppressWarnings("rawtypes")
+  private Class keyClass;
+  @SuppressWarnings("rawtypes")
+  private Class valClass;
+  private CompressionCodec codec;
+  private FileSystem rfs;
+  private IFile.Writer writer;
+
+  private TezTaskOutput ouputFileManager;
+
+  // TODO NEWTEZ Define Counters
+  // Number of records
+  // Time waiting for a write to complete, if that's possible.
+  // Size of key-value pairs written.
+
+  public FileBasedKVWriter(TezOutputContext outputContext) throws IOException {
+    this.conf = TezUtils.createConfFromUserPayload(outputContext
+        .getUserPayload());
+    this.conf.setStrings(TezJobConfig.LOCAL_DIRS,
+        outputContext.getWorkDirs());
+
+    this.rfs = ((LocalFileSystem) FileSystem.getLocal(this.conf)).getRaw();
+
+    // Setup serialization
+    keyClass = ConfigUtils.getIntermediateOutputKeyClass(this.conf);
+    valClass = ConfigUtils.getIntermediateOutputValueClass(this.conf);
+
+    // Setup compression
+    if (ConfigUtils.shouldCompressIntermediateOutput(this.conf)) {
+      Class<? extends CompressionCodec> codecClass = ConfigUtils
+          .getIntermediateOutputCompressorClass(this.conf, DefaultCodec.class);
+      codec = ReflectionUtils.newInstance(codecClass, this.conf);
+    } else {
+      codec = null;
+    }
+
+    this.ouputFileManager = TezRuntimeUtils.instantiateTaskOutputManager(conf,
+        outputContext);
+
+    initWriter();
+  }
+
+  /**
+   * @return true if any output was generated. false otherwise
+   * @throws IOException
+   */
+  public boolean close() throws IOException {
+    this.writer.close();
+    TezIndexRecord rec = new TezIndexRecord(0, writer.getRawLength(),
+        writer.getCompressedLength());
+    TezSpillRecord sr = new TezSpillRecord(1);
+    sr.putIndex(rec, 0);
+
+    Path indexFile = ouputFileManager
+        .getOutputIndexFileForWrite(INDEX_RECORD_LENGTH);
+    sr.writeToFile(indexFile, conf);
+    return numRecords > 0;
+  }
+
+  @Override
+  public void write(Object key, Object value) throws IOException {
+    this.writer.append(key, value);
+    numRecords++;
+  }
+
+  public void initWriter() throws IOException {
+    Path outputFile = ouputFileManager.getOutputFileForWrite();
+
+    // TODO NEWTEZ Consider making the buffer size configurable. Also consider
+    // setting up an in-memory buffer which is occasionally flushed to disk so
+    // that the output does not block.
+
+    // TODO NEWTEZ maybe use appropriate counter
+    this.writer = new IFile.Writer(conf, rfs, outputFile, keyClass, valClass,
+        codec, null);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/ConfigUtils.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/ConfigUtils.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/ConfigUtils.java
new file mode 100644
index 0000000..d1b7ced
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/ConfigUtils.java
@@ -0,0 +1,148 @@
+/**
+* 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.library.common;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.RawComparator;
+import org.apache.hadoop.io.WritableComparable;
+import org.apache.hadoop.io.WritableComparator;
+import org.apache.hadoop.io.compress.CompressionCodec;
+import org.apache.hadoop.io.compress.DefaultCodec;
+import org.apache.hadoop.util.ReflectionUtils;
+import org.apache.tez.common.TezJobConfig;
+
+@SuppressWarnings({"unchecked", "rawtypes"})
+public class ConfigUtils {
+
+  public static Class<? extends CompressionCodec> getIntermediateOutputCompressorClass(
+      Configuration conf, Class<DefaultCodec> defaultValue) {
+    Class<? extends CompressionCodec> codecClass = defaultValue;
+    String name = conf
+        .get(TezJobConfig.TEZ_RUNTIME_INTERMEDIATE_OUTPUT_COMPRESS_CODEC);
+    if (name != null) {
+      try {
+        codecClass = conf.getClassByName(name).asSubclass(
+            CompressionCodec.class);
+      } catch (ClassNotFoundException e) {
+        throw new IllegalArgumentException("Compression codec " + name
+            + " was not found.", e);
+      }
+    }
+    return codecClass;
+  }
+  
+  public static Class<? extends CompressionCodec> getIntermediateInputCompressorClass(
+      Configuration conf, Class<DefaultCodec> defaultValue) {
+    Class<? extends CompressionCodec> codecClass = defaultValue;
+    String name = conf
+        .get(TezJobConfig.TEZ_RUNTIME_INTERMEDIATE_INPUT_COMPRESS_CODEC);
+    if (name != null) {
+      try {
+        codecClass = conf.getClassByName(name).asSubclass(
+            CompressionCodec.class);
+      } catch (ClassNotFoundException e) {
+        throw new IllegalArgumentException("Compression codec " + name
+            + " was not found.", e);
+      }
+    }
+    return codecClass;
+  }
+
+
+  // TODO Move defaults over to a constants file.
+  
+  public static boolean shouldCompressIntermediateOutput(Configuration conf) {
+    return conf.getBoolean(
+        TezJobConfig.TEZ_RUNTIME_INTERMEDIATE_OUTPUT_SHOULD_COMPRESS, false);
+  }
+
+  public static boolean isIntermediateInputCompressed(Configuration conf) {
+    return conf.getBoolean(
+        TezJobConfig.TEZ_RUNTIME_INTERMEDIATE_INPUT_IS_COMPRESSED, false);
+  }
+
+  public static <V> Class<V> getIntermediateOutputValueClass(Configuration conf) {
+    Class<V> retv = (Class<V>) conf.getClass(
+        TezJobConfig.TEZ_RUNTIME_INTERMEDIATE_OUTPUT_VALUE_CLASS, null,
+        Object.class);
+    return retv;
+  }
+  
+  public static <V> Class<V> getIntermediateInputValueClass(Configuration conf) {
+    Class<V> retv = (Class<V>) conf.getClass(
+        TezJobConfig.TEZ_RUNTIME_INTERMEDIATE_INPUT_VALUE_CLASS, null,
+        Object.class);
+    return retv;
+  }
+
+  public static <K> Class<K> getIntermediateOutputKeyClass(Configuration conf) {
+    Class<K> retv = (Class<K>) conf.getClass(
+        TezJobConfig.TEZ_RUNTIME_INTERMEDIATE_OUTPUT_KEY_CLASS, null,
+        Object.class);
+    return retv;
+  }
+
+  public static <K> Class<K> getIntermediateInputKeyClass(Configuration conf) {
+    Class<K> retv = (Class<K>) conf.getClass(
+        TezJobConfig.TEZ_RUNTIME_INTERMEDIATE_INPUT_KEY_CLASS, null,
+        Object.class);
+    return retv;
+  }
+
+  public static <K> RawComparator<K> getIntermediateOutputKeyComparator(Configuration conf) {
+    Class<? extends RawComparator> theClass = conf.getClass(
+        TezJobConfig.TEZ_RUNTIME_INTERMEDIATE_OUTPUT_KEY_COMPARATOR_CLASS, null,
+        RawComparator.class);
+    if (theClass != null)
+      return ReflectionUtils.newInstance(theClass, conf);
+    return WritableComparator.get(getIntermediateOutputKeyClass(conf).asSubclass(
+        WritableComparable.class));
+  }
+
+  public static <K> RawComparator<K> getIntermediateInputKeyComparator(Configuration conf) {
+    Class<? extends RawComparator> theClass = conf.getClass(
+        TezJobConfig.TEZ_RUNTIME_INTERMEDIATE_INPUT_KEY_COMPARATOR_CLASS, null,
+        RawComparator.class);
+    if (theClass != null)
+      return ReflectionUtils.newInstance(theClass, conf);
+    return WritableComparator.get(getIntermediateInputKeyClass(conf).asSubclass(
+        WritableComparable.class));
+  }
+
+  
+  
+  // TODO Fix name
+  public static <V> RawComparator<V> getInputKeySecondaryGroupingComparator(
+      Configuration conf) {
+    Class<? extends RawComparator> theClass = conf
+        .getClass(
+            TezJobConfig.TEZ_RUNTIME_INTERMEDIATE_INPUT_KEY_SECONDARY_COMPARATOR_CLASS,
+            null, RawComparator.class);
+    if (theClass == null) {
+      return getIntermediateInputKeyComparator(conf);
+    }
+
+    return ReflectionUtils.newInstance(theClass, conf);
+  }
+  
+  public static boolean useNewApi(Configuration conf) {
+    return conf.getBoolean("mapred.mapper.new-api", false);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/Constants.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/Constants.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/Constants.java
new file mode 100644
index 0000000..33cd0f6
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/Constants.java
@@ -0,0 +1,61 @@
+/**
+ * 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.library.common;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+
+
+public class Constants {
+
+  // TODO NEWTEZ Check which of these constants are expecting specific pieces of information which are being removed - like taskAttemptId
+  
+  public static final String TEZ = "tez";
+
+  public static final String MAP_OUTPUT_FILENAME_STRING = "file.out";
+  public static final String MAP_OUTPUT_INDEX_SUFFIX_STRING = ".index";
+  public static final String REDUCE_INPUT_FILE_FORMAT_STRING = "%s/map_%d.out";
+
+  public static final int MAP_OUTPUT_INDEX_RECORD_LENGTH = 24;
+  public static String MERGED_OUTPUT_PREFIX = ".merged";
+  
+  // TODO NEWTEZ Remove this constant once the old code is removed.
+  public static final String TEZ_RUNTIME_TASK_ATTEMPT_ID = 
+      "tez.runtime.task.attempt.id";
+
+  public static final String TEZ_RUNTIME_TASK_OUTPUT_FILENAME_STRING =
+      "file.out";
+
+  public static final String TEZ_RUNTIME_TASK_OUTPUT_INDEX_SUFFIX_STRING =
+      ".index";
+
+  public static final String TEZ_RUNTIME_TASK_INPUT_FILE_FORMAT_STRING =
+      "%s/task_%d.out"; 
+
+  public static final String TEZ_RUNTIME_JOB_CREDENTIALS =
+      "tez.runtime.job.credentials";
+  
+  @Private
+  public static final String TEZ_RUNTIME_TASK_MEMORY =
+      "tez.runtime.task.memory";
+  
+  public static final String TEZ_RUNTIME_TASK_OUTPUT_DIR = "output";
+  
+  public static final String TEZ_RUNTIME_TASK_OUTPUT_MANAGER = 
+      "tez.runtime.task.local.output.manager";
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/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
new file mode 100644
index 0000000..a13f3f1
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/InputAttemptIdentifier.java
@@ -0,0 +1,95 @@
+/**
+ * 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.library.common;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+
+/**
+ * Container for a task number and an attempt number for the task.
+ */
+@Private
+public class InputAttemptIdentifier {
+
+  private final InputIdentifier inputIdentifier;
+  private final int attemptNumber;
+  private String pathComponent;
+  
+  public InputAttemptIdentifier(int taskIndex, int attemptNumber) {
+    this(new InputIdentifier(taskIndex), attemptNumber, null);
+  }
+  
+  public InputAttemptIdentifier(InputIdentifier inputIdentifier, int attemptNumber, String pathComponent) {
+    this.inputIdentifier = inputIdentifier;
+    this.attemptNumber = attemptNumber;
+    this.pathComponent = pathComponent;
+  }
+  
+  public InputAttemptIdentifier(int taskIndex, int attemptNumber, String pathComponent) {
+    this(new InputIdentifier(taskIndex), attemptNumber, pathComponent);
+  }
+
+  public InputIdentifier getInputIdentifier() {
+    return this.inputIdentifier;
+  }
+
+  public int getAttemptNumber() {
+    return attemptNumber;
+  }
+  
+  public String getPathComponent() {
+    return pathComponent;
+  }
+
+  // PathComponent does not need to be part of the hashCode and equals computation.
+  @Override
+  public int hashCode() {
+    final int prime = 31;
+    int result = 1;
+    result = prime * result + attemptNumber;
+    result = prime * result
+        + ((inputIdentifier == null) ? 0 : inputIdentifier.hashCode());
+    return result;
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (this == obj)
+      return true;
+    if (obj == null)
+      return false;
+    if (getClass() != obj.getClass())
+      return false;
+    InputAttemptIdentifier other = (InputAttemptIdentifier) obj;
+    if (attemptNumber != other.attemptNumber)
+      return false;
+    if (inputIdentifier == null) {
+      if (other.inputIdentifier != null)
+        return false;
+    } else if (!inputIdentifier.equals(other.inputIdentifier))
+      return false;
+    return true;
+  }
+
+  @Override
+  public String toString() {
+    return "InputAttemptIdentifier [inputIdentifier=" + inputIdentifier
+        + ", attemptNumber=" + attemptNumber + ", pathComponent="
+        + pathComponent + "]";
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/InputIdentifier.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/InputIdentifier.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/InputIdentifier.java
new file mode 100644
index 0000000..f4ce190
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/InputIdentifier.java
@@ -0,0 +1,56 @@
+/**
+ * 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.library.common;
+
+public class InputIdentifier {
+
+  private final int srcTaskIndex;
+  
+  public InputIdentifier(int srcTaskIndex) {
+    this.srcTaskIndex = srcTaskIndex;
+  }
+
+  public int getSrcTaskIndex() {
+    return this.srcTaskIndex;
+  }
+
+  @Override
+  public int hashCode() {
+    return srcTaskIndex;
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (this == obj)
+      return true;
+    if (obj == null)
+      return false;
+    if (getClass() != obj.getClass())
+      return false;
+    InputIdentifier other = (InputIdentifier) obj;
+    if (srcTaskIndex != other.srcTaskIndex)
+      return false;
+    return true;
+  }
+
+  @Override
+  public String toString() {
+    return "InputIdentifier [srcTaskIndex=" + srcTaskIndex + "]";
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/TezRuntimeUtils.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/TezRuntimeUtils.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/TezRuntimeUtils.java
new file mode 100644
index 0000000..2381780
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/TezRuntimeUtils.java
@@ -0,0 +1,152 @@
+/**
+* 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.library.common;
+
+import java.io.IOException;
+import java.lang.reflect.Constructor;
+import java.lang.reflect.InvocationTargetException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.tez.common.TezJobConfig;
+import org.apache.tez.dag.api.TezUncheckedException;
+import org.apache.tez.runtime.api.TezOutputContext;
+import org.apache.tez.runtime.api.TezTaskContext;
+import org.apache.tez.runtime.library.api.Partitioner;
+import org.apache.tez.runtime.library.common.combine.Combiner;
+import org.apache.tez.runtime.library.common.task.local.output.TezTaskOutput;
+import org.apache.tez.runtime.library.common.task.local.output.TezTaskOutputFiles;
+
+public class TezRuntimeUtils {
+
+  private static final Log LOG = LogFactory
+      .getLog(TezRuntimeUtils.class);
+  
+  public static String getTaskIdentifier(String vertexName, int taskIndex) {
+    return String.format("%s_%06d", vertexName, taskIndex);
+  }
+
+  public static String getTaskAttemptIdentifier(int taskIndex,
+      int taskAttemptNumber) {
+    return String.format("%d_%d", taskIndex, taskAttemptNumber);
+  }
+
+  // TODO Maybe include a dag name in this.
+  public static String getTaskAttemptIdentifier(String vertexName,
+      int taskIndex, int taskAttemptNumber) {
+    return String.format("%s_%06d_%02d", vertexName, taskIndex,
+        taskAttemptNumber);
+  }
+
+  @SuppressWarnings("unchecked")
+  public static Combiner instantiateCombiner(Configuration conf, TezTaskContext taskContext) throws IOException {
+    Class<? extends Combiner> clazz;
+    String className = conf.get(TezJobConfig.TEZ_RUNTIME_COMBINER_CLASS);
+    if (className == null) {
+      LOG.info("No combiner specified via " + TezJobConfig.TEZ_RUNTIME_COMBINER_CLASS + ". Combiner will not be used");
+      return null;
+    }
+    LOG.info("Using Combiner class: " + className);
+    try {
+      clazz = (Class<? extends Combiner>) conf.getClassByName(className);
+    } catch (ClassNotFoundException e) {
+      throw new IOException("Unable to load combiner class: " + className);
+    }
+    
+    Combiner combiner = null;
+    
+      Constructor<? extends Combiner> ctor;
+      try {
+        ctor = clazz.getConstructor(TezTaskContext.class);
+        combiner = ctor.newInstance(taskContext);
+      } catch (SecurityException e) {
+        throw new IOException(e);
+      } catch (NoSuchMethodException e) {
+        throw new IOException(e);
+      } catch (IllegalArgumentException e) {
+        throw new IOException(e);
+      } catch (InstantiationException e) {
+        throw new IOException(e);
+      } catch (IllegalAccessException e) {
+        throw new IOException(e);
+      } catch (InvocationTargetException e) {
+        throw new IOException(e);
+      }
+      return combiner;
+  }
+  
+  @SuppressWarnings("unchecked")
+  public static Partitioner instantiatePartitioner(Configuration conf)
+      throws IOException {
+    Class<? extends Partitioner> clazz;
+    try {
+      clazz = (Class<? extends Partitioner>) conf
+          .getClassByName(conf.get(TezJobConfig.TEZ_RUNTIME_PARTITIONER_CLASS));
+    } catch (ClassNotFoundException e) {
+      throw new IOException("Unable to find Partitioner class in config", e);
+    }
+
+    LOG.info("Using partitioner class: " + clazz.getName());
+
+    Partitioner partitioner = null;
+
+    try {
+      Constructor<? extends Partitioner> ctorWithConf = clazz
+          .getConstructor(Configuration.class);
+      partitioner = ctorWithConf.newInstance(conf);
+    } catch (SecurityException e) {
+      throw new IOException(e);
+    } catch (NoSuchMethodException e) {
+      try {
+        // Try a 0 argument constructor.
+        partitioner = clazz.newInstance();
+      } catch (InstantiationException e1) {
+        throw new IOException(e1);
+      } catch (IllegalAccessException e1) {
+        throw new IOException(e1);
+      }
+    } catch (IllegalArgumentException e) {
+      throw new IOException(e);
+    } catch (InstantiationException e) {
+      throw new IOException(e);
+    } catch (IllegalAccessException e) {
+      throw new IOException(e);
+    } catch (InvocationTargetException e) {
+      throw new IOException(e);
+    }
+    return partitioner;
+  }
+  
+  public static TezTaskOutput instantiateTaskOutputManager(Configuration conf, TezOutputContext outputContext) {
+    Class<?> clazz = conf.getClass(Constants.TEZ_RUNTIME_TASK_OUTPUT_MANAGER,
+        TezTaskOutputFiles.class);
+    try {
+      Constructor<?> ctor = clazz.getConstructor(Configuration.class, String.class);
+      ctor.setAccessible(true);
+      TezTaskOutput instance = (TezTaskOutput) ctor.newInstance(conf, outputContext.getUniqueIdentifier());
+      return instance;
+    } catch (Exception e) {
+      throw new TezUncheckedException(
+          "Unable to instantiate configured TezOutputFileManager: "
+              + conf.get(Constants.TEZ_RUNTIME_TASK_OUTPUT_MANAGER,
+                  TezTaskOutputFiles.class.getName()), e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/ValuesIterator.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/ValuesIterator.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/ValuesIterator.java
new file mode 100644
index 0000000..fef3356
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/ValuesIterator.java
@@ -0,0 +1,194 @@
+/**
+* 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.library.common;
+
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.NoSuchElementException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.DataInputBuffer;
+import org.apache.hadoop.io.RawComparator;
+import org.apache.hadoop.io.serializer.Deserializer;
+import org.apache.hadoop.io.serializer.SerializationFactory;
+import org.apache.tez.common.counters.TezCounter;
+import org.apache.tez.runtime.library.common.sort.impl.TezRawKeyValueIterator;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * Iterates values while keys match in sorted input.
+ * 
+ * This class is not thread safe. Accessing methods from multiple threads will
+ * lead to corrupt data.
+ * 
+ */
+public class ValuesIterator<KEY,VALUE> {
+  protected TezRawKeyValueIterator in; //input iterator
+  private KEY key;               // current key
+  private KEY nextKey;
+  private VALUE value;             // current value
+  //private boolean hasNext;                      // more w/ this key
+  private boolean more;                         // more in file
+  private RawComparator<KEY> comparator;
+  private Deserializer<KEY> keyDeserializer;
+  private Deserializer<VALUE> valDeserializer;
+  private DataInputBuffer keyIn = new DataInputBuffer();
+  private DataInputBuffer valueIn = new DataInputBuffer();
+  private TezCounter inputKeyCounter;
+  private TezCounter inputValueCounter;
+  
+  private int keyCtr = 0;
+  private boolean hasMoreValues; // For the current key.
+  private boolean isFirstRecord = true;
+  
+  public ValuesIterator (TezRawKeyValueIterator in, 
+                         RawComparator<KEY> comparator, 
+                         Class<KEY> keyClass,
+                         Class<VALUE> valClass, Configuration conf,
+                         TezCounter inputKeyCounter,
+                         TezCounter inputValueCounter)
+    throws IOException {
+    this.in = in;
+    this.comparator = comparator;
+    this.inputKeyCounter = inputKeyCounter;
+    this.inputValueCounter = inputValueCounter;
+    SerializationFactory serializationFactory = new SerializationFactory(conf);
+    this.keyDeserializer = serializationFactory.getDeserializer(keyClass);
+    this.keyDeserializer.open(keyIn);
+    this.valDeserializer = serializationFactory.getDeserializer(valClass);
+    this.valDeserializer.open(this.valueIn);
+  }
+
+  TezRawKeyValueIterator getRawIterator() { return in; }
+
+  /**
+   * Move to the next K-Vs pair
+   * @return true if another pair exists, otherwise false.
+   * @throws IOException 
+   */
+  public boolean moveToNext() throws IOException {
+    if (isFirstRecord) {
+      readNextKey();
+      key = nextKey;
+      nextKey = null;
+      hasMoreValues = more;
+      isFirstRecord = false;
+    } else {
+      nextKey();
+    }
+    return more;
+  }
+  
+  /** The current key. */
+  public KEY getKey() { 
+    return key; 
+  }
+  
+  // TODO NEWTEZ Maybe add another method which returns an iterator instead of iterable
+  
+  public Iterable<VALUE> getValues() {
+    return new Iterable<VALUE>() {
+
+      @Override
+      public Iterator<VALUE> iterator() {
+        
+        return new Iterator<VALUE>() {
+
+          private final int keyNumber = keyCtr;
+          
+          @Override
+          public boolean hasNext() {
+            return hasMoreValues;
+          }
+
+          @Override
+          public VALUE next() {
+            if (!hasMoreValues) {
+              throw new NoSuchElementException("iterate past last value");
+            }
+            Preconditions
+                .checkState(
+                    keyNumber == keyCtr,
+                    "Cannot use values iterator on the previous K-V pair after moveToNext has been invoked to move to the next K-V pair");
+            
+            try {
+              readNextValue();
+              readNextKey();
+            } catch (IOException ie) {
+              throw new RuntimeException("problem advancing post rec#"+keyCtr, ie);
+            }
+            inputValueCounter.increment(1);
+            return value;
+          }
+
+          @Override
+          public void remove() {
+            throw new UnsupportedOperationException("Cannot remove elements");
+          }
+        };
+      }
+    };
+  }
+  
+  
+
+  /** Start processing next unique key. */
+  private void nextKey() throws IOException {
+    // read until we find a new key
+    while (hasMoreValues) { 
+      readNextKey();
+    }
+    if (more) {
+      inputKeyCounter.increment(1);
+      ++keyCtr;
+    }
+    
+    // move the next key to the current one
+    KEY tmpKey = key;
+    key = nextKey;
+    nextKey = tmpKey;
+    hasMoreValues = more;
+  }
+
+  /** 
+   * read the next key - which may be the same as the current key.
+   */
+  private void readNextKey() throws IOException {
+    more = in.next();
+    if (more) {      
+      DataInputBuffer nextKeyBytes = in.getKey();
+      keyIn.reset(nextKeyBytes.getData(), nextKeyBytes.getPosition(), nextKeyBytes.getLength());
+      nextKey = keyDeserializer.deserialize(nextKey);
+      hasMoreValues = key != null && (comparator.compare(key, nextKey) == 0);
+    } else {
+      hasMoreValues = false;
+    }
+  }
+
+  /**
+   * Read the next value
+   * @throws IOException
+   */
+  private void readNextValue() throws IOException {
+    DataInputBuffer nextValueBytes = in.getValue();
+    valueIn.reset(nextValueBytes.getData(), nextValueBytes.getPosition(), nextValueBytes.getLength());
+    value = valDeserializer.deserialize(value);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/YARNMaster.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/YARNMaster.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/YARNMaster.java
new file mode 100644
index 0000000..8709e05
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/YARNMaster.java
@@ -0,0 +1,57 @@
+/**
+* Licensed to the Apache Software Foundation (ASF) under one
+* or more contributor license agreements.  See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership.  The ASF licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License.  You may obtain a copy of the License at
+*
+*     http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package org.apache.tez.runtime.library.common;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.SecurityUtil;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+
+@Private
+@Unstable
+public class YARNMaster {
+  
+  public enum State {
+    INITIALIZING, RUNNING;
+  }
+
+  public static String getMasterUserName(Configuration conf) {
+    return conf.get(YarnConfiguration.RM_PRINCIPAL);
+  }
+  
+  public static InetSocketAddress getMasterAddress(Configuration conf) {
+    return conf.getSocketAddr(
+        YarnConfiguration.RM_ADDRESS,
+        YarnConfiguration.DEFAULT_RM_ADDRESS,
+        YarnConfiguration.DEFAULT_RM_PORT);
+  }
+
+  public static String getMasterPrincipal(Configuration conf) 
+  throws IOException {
+    String masterHostname = getMasterAddress(conf).getHostName();
+    // get kerberos principal for use as delegation token renewer
+    return SecurityUtil.getServerPrincipal(
+        getMasterUserName(conf), masterHostname);
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/combine/Combiner.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/combine/Combiner.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/combine/Combiner.java
new file mode 100644
index 0000000..5b10590
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/combine/Combiner.java
@@ -0,0 +1,42 @@
+/**
+ * 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.library.common.combine;
+
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience.LimitedPrivate;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.tez.common.TezJobConfig;
+import org.apache.tez.runtime.library.common.sort.impl.TezRawKeyValueIterator;
+import org.apache.tez.runtime.library.common.sort.impl.IFile.Writer;
+
+/**
+ *<b>Combiner Initialization</b></p> The Combiner class is picked up
+ * using the TEZ_RUNTIME_COMBINER_CLASS attribute in {@link TezJobConfig}
+ * 
+ * 
+ * Partitioners need to provide a single argument ({@link TezTaskContext})
+ * constructor.
+ */
+@Unstable
+@LimitedPrivate("mapreduce")
+public interface Combiner {
+  public void combine(TezRawKeyValueIterator rawIter, Writer writer)
+      throws InterruptedException, IOException;
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/localshuffle/LocalShuffle.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/localshuffle/LocalShuffle.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/localshuffle/LocalShuffle.java
new file mode 100644
index 0000000..b40df6f
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/localshuffle/LocalShuffle.java
@@ -0,0 +1,120 @@
+/**
+* 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.library.common.localshuffle;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.RawComparator;
+import org.apache.hadoop.io.compress.CompressionCodec;
+import org.apache.hadoop.io.compress.DefaultCodec;
+import org.apache.hadoop.util.ReflectionUtils;
+import org.apache.tez.common.TezJobConfig;
+import org.apache.tez.common.counters.TaskCounter;
+import org.apache.tez.common.counters.TezCounter;
+import org.apache.tez.runtime.api.TezInputContext;
+import org.apache.tez.runtime.library.common.ConfigUtils;
+import org.apache.tez.runtime.library.common.sort.impl.TezMerger;
+import org.apache.tez.runtime.library.common.sort.impl.TezRawKeyValueIterator;
+import org.apache.tez.runtime.library.common.task.local.output.TezLocalTaskOutputFiles;
+import org.apache.tez.runtime.library.common.task.local.output.TezTaskOutput;
+
+@SuppressWarnings({"rawtypes"})
+public class LocalShuffle {
+
+  // TODO NEWTEZ This is broken.
+
+  private final TezInputContext inputContext;
+  private final Configuration conf;
+  private final int numInputs;
+
+  private final Class keyClass;
+  private final Class valClass;
+  private final RawComparator comparator;
+
+  private final FileSystem rfs;
+  private final int sortFactor;
+  
+  private final TezCounter spilledRecordsCounter;
+  private final CompressionCodec codec;
+  private final TezTaskOutput mapOutputFile;
+
+  public LocalShuffle(TezInputContext inputContext, Configuration conf, int numInputs) throws IOException {
+    this.inputContext = inputContext;
+    this.conf = conf;
+    this.numInputs = numInputs;
+    
+    this.keyClass = ConfigUtils.getIntermediateInputKeyClass(conf);
+    this.valClass = ConfigUtils.getIntermediateInputValueClass(conf);
+    this.comparator = ConfigUtils.getIntermediateInputKeyComparator(conf);
+    
+    this.sortFactor =
+        conf.getInt(
+            TezJobConfig.TEZ_RUNTIME_IO_SORT_FACTOR, 
+            TezJobConfig.DEFAULT_TEZ_RUNTIME_IO_SORT_FACTOR);
+    
+    this.rfs = FileSystem.getLocal(conf).getRaw();
+
+    this.spilledRecordsCounter = inputContext.getCounters().findCounter(TaskCounter.SPILLED_RECORDS);
+    
+ // compression
+    if (ConfigUtils.isIntermediateInputCompressed(conf)) {
+      Class<? extends CompressionCodec> codecClass =
+          ConfigUtils.getIntermediateInputCompressorClass(conf, DefaultCodec.class);
+      this.codec = ReflectionUtils.newInstance(codecClass, conf);
+    } else {
+      this.codec = null;
+    }
+    
+    // Always local
+    this.mapOutputFile = new TezLocalTaskOutputFiles(conf, inputContext.getUniqueIdentifier());
+  }
+ 
+  
+  public TezRawKeyValueIterator run() throws IOException {
+    // Copy is complete, obviously! 
+
+    
+    // Merge
+    return TezMerger.merge(conf, rfs, 
+        keyClass, valClass,
+        codec, 
+        getMapFiles(),
+        false, 
+        sortFactor,
+        new Path(inputContext.getUniqueIdentifier()), // TODO NEWTEZ This is likely broken 
+        comparator,
+        null, spilledRecordsCounter, null, null);
+  }
+  
+  private Path[] getMapFiles() 
+  throws IOException {
+    List<Path> fileList = new ArrayList<Path>();
+      // for local jobs
+      for(int i = 0; i < numInputs; ++i) {
+        //fileList.add(mapOutputFile.getInputFile(i));
+      }
+      
+    return fileList.toArray(new Path[0]);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/security/JobTokenIdentifier.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/security/JobTokenIdentifier.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/security/JobTokenIdentifier.java
new file mode 100644
index 0000000..4b916fa
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/security/JobTokenIdentifier.java
@@ -0,0 +1,98 @@
+/**
+ * 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.library.common.security;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.TokenIdentifier;
+
+/**
+ * The token identifier for job token
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class JobTokenIdentifier extends TokenIdentifier {
+  private Text jobid;
+  public final static Text KIND_NAME = new Text("mapreduce.job");
+  
+  /**
+   * Default constructor
+   */
+  public JobTokenIdentifier() {
+    this.jobid = new Text();
+  }
+
+  /**
+   * Create a job token identifier from a jobid
+   * @param jobid the jobid to use
+   */
+  public JobTokenIdentifier(Text jobid) {
+    this.jobid = jobid;
+  }
+
+  /** {@inheritDoc} */
+  @Override
+  public Text getKind() {
+    return KIND_NAME;
+  }
+  
+  /** {@inheritDoc} */
+  @Override
+  public UserGroupInformation getUser() {
+    if (jobid == null || "".equals(jobid.toString())) {
+      return null;
+    }
+    return UserGroupInformation.createRemoteUser(jobid.toString());
+  }
+  
+  /**
+   * Get the jobid
+   * @return the jobid
+   */
+  public Text getJobId() {
+    return jobid;
+  }
+
+  /** {@inheritDoc} */
+  @Override
+  public void readFields(DataInput in) throws IOException {
+    jobid.readFields(in);
+  }
+
+  /** {@inheritDoc} */
+  @Override
+  public void write(DataOutput out) throws IOException {
+    jobid.write(out);
+  }
+
+  @InterfaceAudience.Private
+  public static class Renewer extends Token.TrivialRenewer {
+    @Override
+    protected Text getKind() {
+      return KIND_NAME;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/security/JobTokenSecretManager.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/security/JobTokenSecretManager.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/security/JobTokenSecretManager.java
new file mode 100644
index 0000000..a03ee94
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/security/JobTokenSecretManager.java
@@ -0,0 +1,137 @@
+/**
+ * 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.library.common.security;
+
+import java.util.Map;
+import java.util.TreeMap;
+
+import javax.crypto.SecretKey;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.security.token.SecretManager;
+import org.apache.hadoop.security.token.Token;
+
+/**
+ * SecretManager for job token. It can be used to cache generated job tokens.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class JobTokenSecretManager extends SecretManager<JobTokenIdentifier> {
+  private final SecretKey masterKey;
+  private final Map<String, SecretKey> currentJobTokens;
+
+  /**
+   * Convert the byte[] to a secret key
+   * @param key the byte[] to create the secret key from
+   * @return the secret key
+   */
+  public static SecretKey createSecretKey(byte[] key) {
+    return SecretManager.createSecretKey(key);
+  }
+  
+  /**
+   * Compute the HMAC hash of the message using the key
+   * @param msg the message to hash
+   * @param key the key to use
+   * @return the computed hash
+   */
+  public static byte[] computeHash(byte[] msg, SecretKey key) {
+    return createPassword(msg, key);
+  }
+  
+  /**
+   * Default constructor
+   */
+  public JobTokenSecretManager() {
+    this.masterKey = generateSecret();
+    this.currentJobTokens = new TreeMap<String, SecretKey>();
+  }
+  
+  /**
+   * Create a new password/secret for the given job token identifier.
+   * @param identifier the job token identifier
+   * @return token password/secret
+   */
+  @Override
+  public byte[] createPassword(JobTokenIdentifier identifier) {
+    byte[] result = createPassword(identifier.getBytes(), masterKey);
+    return result;
+  }
+
+  /**
+   * Add the job token of a job to cache
+   * @param jobId the job that owns the token
+   * @param token the job token
+   */
+  public void addTokenForJob(String jobId, Token<JobTokenIdentifier> token) {
+    SecretKey tokenSecret = createSecretKey(token.getPassword());
+    synchronized (currentJobTokens) {
+      currentJobTokens.put(jobId, tokenSecret);
+    }
+  }
+
+  /**
+   * Remove the cached job token of a job from cache
+   * @param jobId the job whose token is to be removed
+   */
+  public void removeTokenForJob(String jobId) {
+    synchronized (currentJobTokens) {
+      currentJobTokens.remove(jobId);
+    }
+  }
+  
+  /**
+   * Look up the token password/secret for the given jobId.
+   * @param jobId the jobId to look up
+   * @return token password/secret as SecretKey
+   * @throws InvalidToken
+   */
+  public SecretKey retrieveTokenSecret(String jobId) throws InvalidToken {
+    SecretKey tokenSecret = null;
+    synchronized (currentJobTokens) {
+      tokenSecret = currentJobTokens.get(jobId);
+    }
+    if (tokenSecret == null) {
+      throw new InvalidToken("Can't find job token for job " + jobId + " !!");
+    }
+    return tokenSecret;
+  }
+  
+  /**
+   * Look up the token password/secret for the given job token identifier.
+   * @param identifier the job token identifier to look up
+   * @return token password/secret as byte[]
+   * @throws InvalidToken
+   */
+  @Override
+  public byte[] retrievePassword(JobTokenIdentifier identifier)
+      throws InvalidToken {
+    return retrieveTokenSecret(identifier.getJobId().toString()).getEncoded();
+  }
+
+  /**
+   * Create an empty job token identifier
+   * @return a newly created empty job token identifier
+   */
+  @Override
+  public JobTokenIdentifier createIdentifier() {
+    return new JobTokenIdentifier();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/security/JobTokenSelector.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/security/JobTokenSelector.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/security/JobTokenSelector.java
new file mode 100644
index 0000000..b8227ab
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/security/JobTokenSelector.java
@@ -0,0 +1,53 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.runtime.library.common.security;
+
+import java.util.Collection;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.TokenIdentifier;
+import org.apache.hadoop.security.token.TokenSelector;
+
+/**
+ * Look through tokens to find the first job token that matches the service
+ * and return it.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class JobTokenSelector implements TokenSelector<JobTokenIdentifier> {
+
+  @SuppressWarnings("unchecked")
+  @Override
+  public Token<JobTokenIdentifier> selectToken(Text service,
+      Collection<Token<? extends TokenIdentifier>> tokens) {
+    if (service == null) {
+      return null;
+    }
+    for (Token<? extends TokenIdentifier> token : tokens) {
+      if (JobTokenIdentifier.KIND_NAME.equals(token.getKind())
+          && service.equals(token.getService())) {
+        return (Token<JobTokenIdentifier>) token;
+      }
+    }
+    return null;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/security/Master.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/security/Master.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/security/Master.java
new file mode 100644
index 0000000..5b91e0f
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/security/Master.java
@@ -0,0 +1,57 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.runtime.library.common.security;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.SecurityUtil;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+
+@Private
+@Unstable
+public class Master {
+
+  public enum State {
+    INITIALIZING, RUNNING;
+  }
+
+  public static String getMasterUserName(Configuration conf) {
+    return conf.get(YarnConfiguration.RM_PRINCIPAL);
+  }
+
+  public static InetSocketAddress getMasterAddress(Configuration conf) {
+    return conf
+        .getSocketAddr(YarnConfiguration.RM_ADDRESS,
+            YarnConfiguration.DEFAULT_RM_ADDRESS,
+            YarnConfiguration.DEFAULT_RM_PORT);
+  }
+
+  public static String getMasterPrincipal(Configuration conf)
+      throws IOException {
+    String masterHostname = getMasterAddress(conf).getHostName();
+    // get kerberos principal for use as delegation token renewer
+    return SecurityUtil.getServerPrincipal(getMasterUserName(conf),
+        masterHostname);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/security/SecureShuffleUtils.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/security/SecureShuffleUtils.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/security/SecureShuffleUtils.java
new file mode 100644
index 0000000..b2d382c
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/security/SecureShuffleUtils.java
@@ -0,0 +1,142 @@
+/**
+ * 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.library.common.security;
+
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.PrintStream;
+import java.net.URL;
+
+import javax.crypto.SecretKey;
+import javax.servlet.http.HttpServletRequest;
+
+import org.apache.commons.codec.binary.Base64;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.io.WritableComparator;
+
+/**
+ * 
+ * utilities for generating kyes, hashes and verifying them for shuffle
+ *
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class SecureShuffleUtils {
+  public static final String HTTP_HEADER_URL_HASH = "UrlHash";
+  public static final String HTTP_HEADER_REPLY_URL_HASH = "ReplyHash";
+  
+  /**
+   * Base64 encoded hash of msg
+   * @param msg
+   */
+  public static String generateHash(byte[] msg, SecretKey key) {
+    return new String(Base64.encodeBase64(generateByteHash(msg, key)));
+  }
+  
+  /**
+   * calculate hash of msg
+   * @param msg
+   * @return
+   */
+  private static byte[] generateByteHash(byte[] msg, SecretKey key) {
+    return JobTokenSecretManager.computeHash(msg, key);
+  }
+  
+  /**
+   * verify that hash equals to HMacHash(msg)
+   * @param newHash
+   * @return true if is the same
+   */
+  private static boolean verifyHash(byte[] hash, byte[] msg, SecretKey key) {
+    byte[] msg_hash = generateByteHash(msg, key);
+    return WritableComparator.compareBytes(msg_hash, 0, msg_hash.length, hash, 0, hash.length) == 0;
+  }
+  
+  /**
+   * Aux util to calculate hash of a String
+   * @param enc_str
+   * @param key
+   * @return Base64 encodedHash
+   * @throws IOException
+   */
+  public static String hashFromString(String enc_str, SecretKey key) 
+  throws IOException {
+    return generateHash(enc_str.getBytes(), key); 
+  }
+  
+  /**
+   * verify that base64Hash is same as HMacHash(msg)  
+   * @param base64Hash (Base64 encoded hash)
+   * @param msg
+   * @throws IOException if not the same
+   */
+  public static void verifyReply(String base64Hash, String msg, SecretKey key)
+  throws IOException {
+    byte[] hash = Base64.decodeBase64(base64Hash.getBytes());
+    
+    boolean res = verifyHash(hash, msg.getBytes(), key);
+    
+    if(res != true) {
+      throw new IOException("Verification of the hashReply failed");
+    }
+  }
+  
+  /**
+   * Shuffle specific utils - build string for encoding from URL
+   * @param url
+   * @return string for encoding
+   */
+  public static String buildMsgFrom(URL url) {
+    return buildMsgFrom(url.getPath(), url.getQuery(), url.getPort());
+  }
+  /**
+   * Shuffle specific utils - build string for encoding from URL
+   * @param request
+   * @return string for encoding
+   */
+  public static String buildMsgFrom(HttpServletRequest request ) {
+    return buildMsgFrom(request.getRequestURI(), request.getQueryString(),
+        request.getLocalPort());
+  }
+  /**
+   * Shuffle specific utils - build string for encoding from URL
+   * @param uri_path
+   * @param uri_query
+   * @return string for encoding
+   */
+  private static String buildMsgFrom(String uri_path, String uri_query, int port) {
+    return String.valueOf(port) + uri_path + "?" + uri_query;
+  }
+  
+  
+  /**
+   * byte array to Hex String
+   * @param ba
+   * @return string with HEX value of the key
+   */
+  public static String toHex(byte[] ba) {
+    ByteArrayOutputStream baos = new ByteArrayOutputStream();
+    PrintStream ps = new PrintStream(baos);
+    for(byte b: ba) {
+      ps.printf("%x", b);
+    }
+    return baos.toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/security/TokenCache.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/security/TokenCache.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/security/TokenCache.java
new file mode 100644
index 0000000..33373b7
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/security/TokenCache.java
@@ -0,0 +1,205 @@
+/**
+ * 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.library.common.security;
+
+import java.io.IOException;
+import java.util.HashSet;
+import java.util.Set;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.security.Credentials;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.TokenIdentifier;
+import org.apache.tez.common.TezJobConfig;
+
+
+/**
+ * This class provides user facing APIs for transferring secrets from
+ * the job client to the tasks.
+ * The secrets can be stored just before submission of jobs and read during
+ * the task execution.  
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class TokenCache {
+  
+  private static final Log LOG = LogFactory.getLog(TokenCache.class);
+
+  
+  /**
+   * auxiliary method to get user's secret keys..
+   * @param alias
+   * @return secret key from the storage
+   */
+  public static byte[] getSecretKey(Credentials credentials, Text alias) {
+    if(credentials == null)
+      return null;
+    return credentials.getSecretKey(alias);
+  }
+  
+  /**
+   * Convenience method to obtain delegation tokens from namenodes 
+   * corresponding to the paths passed.
+   * @param credentials
+   * @param ps array of paths
+   * @param conf configuration
+   * @throws IOException
+   */
+  public static void obtainTokensForNamenodes(Credentials credentials,
+      Path[] ps, Configuration conf) throws IOException {
+    if (!UserGroupInformation.isSecurityEnabled()) {
+      return;
+    }
+    obtainTokensForNamenodesInternal(credentials, ps, conf);
+  }
+
+  /**
+   * Remove jobtoken referrals which don't make sense in the context
+   * of the task execution.
+   *
+   * @param conf
+   */
+  public static void cleanUpTokenReferral(Configuration conf) {
+    conf.unset(TezJobConfig.DAG_CREDENTIALS_BINARY);
+  }
+
+  static void obtainTokensForNamenodesInternal(Credentials credentials,
+      Path[] ps, Configuration conf) throws IOException {
+    Set<FileSystem> fsSet = new HashSet<FileSystem>();
+    for(Path p: ps) {
+      fsSet.add(p.getFileSystem(conf));
+    }
+    for (FileSystem fs : fsSet) {
+      obtainTokensForNamenodesInternal(fs, credentials, conf);
+    }
+  }
+
+  /**
+   * get delegation token for a specific FS
+   * @param fs
+   * @param credentials
+   * @param p
+   * @param conf
+   * @throws IOException
+   */
+  static void obtainTokensForNamenodesInternal(FileSystem fs, 
+      Credentials credentials, Configuration conf) throws IOException {
+    String delegTokenRenewer = Master.getMasterPrincipal(conf);
+    if (delegTokenRenewer == null || delegTokenRenewer.length() == 0) {
+      throw new IOException(
+          "Can't get Master Kerberos principal for use as renewer");
+    }
+    mergeBinaryTokens(credentials, conf);
+
+    final Token<?> tokens[] = fs.addDelegationTokens(delegTokenRenewer,
+                                                     credentials);
+    if (tokens != null) {
+      for (Token<?> token : tokens) {
+        LOG.info("Got dt for " + fs.getUri() + "; "+token);
+      }
+    }
+  }
+
+  private static void mergeBinaryTokens(Credentials creds, Configuration conf) {
+    String binaryTokenFilename =
+        conf.get(TezJobConfig.DAG_CREDENTIALS_BINARY);
+    if (binaryTokenFilename != null) {
+      Credentials binary;
+      try {
+        binary = Credentials.readTokenStorageFile(
+            new Path("file:///" +  binaryTokenFilename), conf);
+      } catch (IOException e) {
+        throw new RuntimeException(e);
+      }
+      // supplement existing tokens with the tokens in the binary file
+      creds.mergeAll(binary);
+    }
+  }
+  
+  /**
+   * file name used on HDFS for generated job token
+   */
+  @InterfaceAudience.Private
+  public static final String JOB_TOKEN_HDFS_FILE = "jobToken";
+
+  /**
+   * conf setting for job tokens cache file name
+   */
+  @InterfaceAudience.Private
+  public static final String JOB_TOKENS_FILENAME = "mapreduce.job.jobTokenFile";
+  private static final Text JOB_TOKEN = new Text("JobToken");
+  private static final Text SHUFFLE_TOKEN = new Text("MapReduceShuffleToken");
+  
+  /**
+   * load job token from a file
+   * @param conf
+   * @throws IOException
+   */
+  @InterfaceAudience.Private
+  public static Credentials loadTokens(String jobTokenFile, Configuration conf) 
+  throws IOException {
+    Path localJobTokenFile = new Path ("file:///" + jobTokenFile);
+
+    Credentials ts = Credentials.readTokenStorageFile(localJobTokenFile, conf);
+
+    if(LOG.isDebugEnabled()) {
+      LOG.debug("Task: Loaded jobTokenFile from: "+
+          localJobTokenFile.toUri().getPath() 
+          +"; num of sec keys  = " + ts.numberOfSecretKeys() +
+          " Number of tokens " +  ts.numberOfTokens());
+    }
+    return ts;
+  }
+  /**
+   * store job token
+   * @param t
+   */
+  @InterfaceAudience.Private
+  public static void setJobToken(Token<? extends TokenIdentifier> t, 
+      Credentials credentials) {
+    credentials.addToken(JOB_TOKEN, t);
+  }
+  /**
+   * 
+   * @return job token
+   */
+  @SuppressWarnings("unchecked")
+  @InterfaceAudience.Private
+  public static Token<JobTokenIdentifier> getJobToken(Credentials credentials) {
+    return (Token<JobTokenIdentifier>) credentials.getToken(JOB_TOKEN);
+  }
+
+  @InterfaceAudience.Private
+  public static void setShuffleSecretKey(byte[] key, Credentials credentials) {
+    credentials.addSecretKey(SHUFFLE_TOKEN, key);
+  }
+
+  @InterfaceAudience.Private
+  public static byte[] getShuffleSecretKey(Credentials credentials) {
+    return getSecretKey(credentials, SHUFFLE_TOKEN);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ExceptionReporter.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ExceptionReporter.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ExceptionReporter.java
new file mode 100644
index 0000000..a872ba1
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ExceptionReporter.java
@@ -0,0 +1,25 @@
+/**
+ * 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.library.common.shuffle.impl;
+
+/**
+ * An interface for reporting exceptions to other threads
+ */
+interface ExceptionReporter {
+  void reportException(Throwable t);
+}


[43/50] [abbrv] Rename tez-engine-api to tez-runtime-api and tez-engine is split into 2: - tez-engine-library for user-visible Input/Output/Processor implementations - tez-engine-internals for framework internals

Posted by ss...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/engine/common/ValuesIterator.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/ValuesIterator.java b/tez-engine/src/main/java/org/apache/tez/engine/common/ValuesIterator.java
deleted file mode 100644
index b7867aa..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/ValuesIterator.java
+++ /dev/null
@@ -1,194 +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.engine.common;
-
-import java.io.IOException;
-import java.util.Iterator;
-import java.util.NoSuchElementException;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.io.DataInputBuffer;
-import org.apache.hadoop.io.RawComparator;
-import org.apache.hadoop.io.serializer.Deserializer;
-import org.apache.hadoop.io.serializer.SerializationFactory;
-import org.apache.tez.common.counters.TezCounter;
-import org.apache.tez.engine.common.sort.impl.TezRawKeyValueIterator;
-
-import com.google.common.base.Preconditions;
-
-/**
- * Iterates values while keys match in sorted input.
- * 
- * This class is not thread safe. Accessing methods from multiple threads will
- * lead to corrupt data.
- * 
- */
-public class ValuesIterator<KEY,VALUE> {
-  protected TezRawKeyValueIterator in; //input iterator
-  private KEY key;               // current key
-  private KEY nextKey;
-  private VALUE value;             // current value
-  //private boolean hasNext;                      // more w/ this key
-  private boolean more;                         // more in file
-  private RawComparator<KEY> comparator;
-  private Deserializer<KEY> keyDeserializer;
-  private Deserializer<VALUE> valDeserializer;
-  private DataInputBuffer keyIn = new DataInputBuffer();
-  private DataInputBuffer valueIn = new DataInputBuffer();
-  private TezCounter inputKeyCounter;
-  private TezCounter inputValueCounter;
-  
-  private int keyCtr = 0;
-  private boolean hasMoreValues; // For the current key.
-  private boolean isFirstRecord = true;
-  
-  public ValuesIterator (TezRawKeyValueIterator in, 
-                         RawComparator<KEY> comparator, 
-                         Class<KEY> keyClass,
-                         Class<VALUE> valClass, Configuration conf,
-                         TezCounter inputKeyCounter,
-                         TezCounter inputValueCounter)
-    throws IOException {
-    this.in = in;
-    this.comparator = comparator;
-    this.inputKeyCounter = inputKeyCounter;
-    this.inputValueCounter = inputValueCounter;
-    SerializationFactory serializationFactory = new SerializationFactory(conf);
-    this.keyDeserializer = serializationFactory.getDeserializer(keyClass);
-    this.keyDeserializer.open(keyIn);
-    this.valDeserializer = serializationFactory.getDeserializer(valClass);
-    this.valDeserializer.open(this.valueIn);
-  }
-
-  TezRawKeyValueIterator getRawIterator() { return in; }
-
-  /**
-   * Move to the next K-Vs pair
-   * @return true if another pair exists, otherwise false.
-   * @throws IOException 
-   */
-  public boolean moveToNext() throws IOException {
-    if (isFirstRecord) {
-      readNextKey();
-      key = nextKey;
-      nextKey = null;
-      hasMoreValues = more;
-      isFirstRecord = false;
-    } else {
-      nextKey();
-    }
-    return more;
-  }
-  
-  /** The current key. */
-  public KEY getKey() { 
-    return key; 
-  }
-  
-  // TODO NEWTEZ Maybe add another method which returns an iterator instead of iterable
-  
-  public Iterable<VALUE> getValues() {
-    return new Iterable<VALUE>() {
-
-      @Override
-      public Iterator<VALUE> iterator() {
-        
-        return new Iterator<VALUE>() {
-
-          private final int keyNumber = keyCtr;
-          
-          @Override
-          public boolean hasNext() {
-            return hasMoreValues;
-          }
-
-          @Override
-          public VALUE next() {
-            if (!hasMoreValues) {
-              throw new NoSuchElementException("iterate past last value");
-            }
-            Preconditions
-                .checkState(
-                    keyNumber == keyCtr,
-                    "Cannot use values iterator on the previous K-V pair after moveToNext has been invoked to move to the next K-V pair");
-            
-            try {
-              readNextValue();
-              readNextKey();
-            } catch (IOException ie) {
-              throw new RuntimeException("problem advancing post rec#"+keyCtr, ie);
-            }
-            inputValueCounter.increment(1);
-            return value;
-          }
-
-          @Override
-          public void remove() {
-            throw new UnsupportedOperationException("Cannot remove elements");
-          }
-        };
-      }
-    };
-  }
-  
-  
-
-  /** Start processing next unique key. */
-  private void nextKey() throws IOException {
-    // read until we find a new key
-    while (hasMoreValues) { 
-      readNextKey();
-    }
-    if (more) {
-      inputKeyCounter.increment(1);
-      ++keyCtr;
-    }
-    
-    // move the next key to the current one
-    KEY tmpKey = key;
-    key = nextKey;
-    nextKey = tmpKey;
-    hasMoreValues = more;
-  }
-
-  /** 
-   * read the next key - which may be the same as the current key.
-   */
-  private void readNextKey() throws IOException {
-    more = in.next();
-    if (more) {      
-      DataInputBuffer nextKeyBytes = in.getKey();
-      keyIn.reset(nextKeyBytes.getData(), nextKeyBytes.getPosition(), nextKeyBytes.getLength());
-      nextKey = keyDeserializer.deserialize(nextKey);
-      hasMoreValues = key != null && (comparator.compare(key, nextKey) == 0);
-    } else {
-      hasMoreValues = false;
-    }
-  }
-
-  /**
-   * Read the next value
-   * @throws IOException
-   */
-  private void readNextValue() throws IOException {
-    DataInputBuffer nextValueBytes = in.getValue();
-    valueIn.reset(nextValueBytes.getData(), nextValueBytes.getPosition(), nextValueBytes.getLength());
-    value = valDeserializer.deserialize(value);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/engine/common/YARNMaster.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/YARNMaster.java b/tez-engine/src/main/java/org/apache/tez/engine/common/YARNMaster.java
deleted file mode 100644
index 48ad639..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/YARNMaster.java
+++ /dev/null
@@ -1,57 +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.engine.common;
-
-import java.io.IOException;
-import java.net.InetSocketAddress;
-
-import org.apache.hadoop.classification.InterfaceAudience.Private;
-import org.apache.hadoop.classification.InterfaceStability.Unstable;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.security.SecurityUtil;
-import org.apache.hadoop.yarn.conf.YarnConfiguration;
-
-@Private
-@Unstable
-public class YARNMaster {
-  
-  public enum State {
-    INITIALIZING, RUNNING;
-  }
-
-  public static String getMasterUserName(Configuration conf) {
-    return conf.get(YarnConfiguration.RM_PRINCIPAL);
-  }
-  
-  public static InetSocketAddress getMasterAddress(Configuration conf) {
-    return conf.getSocketAddr(
-        YarnConfiguration.RM_ADDRESS,
-        YarnConfiguration.DEFAULT_RM_ADDRESS,
-        YarnConfiguration.DEFAULT_RM_PORT);
-  }
-
-  public static String getMasterPrincipal(Configuration conf) 
-  throws IOException {
-    String masterHostname = getMasterAddress(conf).getHostName();
-    // get kerberos principal for use as delegation token renewer
-    return SecurityUtil.getServerPrincipal(
-        getMasterUserName(conf), masterHostname);
-  }
-  
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/engine/common/combine/Combiner.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/combine/Combiner.java b/tez-engine/src/main/java/org/apache/tez/engine/common/combine/Combiner.java
deleted file mode 100644
index b387b36..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/combine/Combiner.java
+++ /dev/null
@@ -1,42 +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.engine.common.combine;
-
-import java.io.IOException;
-
-import org.apache.hadoop.classification.InterfaceAudience.LimitedPrivate;
-import org.apache.hadoop.classification.InterfaceStability.Unstable;
-import org.apache.tez.common.TezJobConfig;
-import org.apache.tez.engine.common.sort.impl.IFile.Writer;
-import org.apache.tez.engine.common.sort.impl.TezRawKeyValueIterator;
-
-/**
- *<b>Combiner Initialization</b></p> The Combiner class is picked up
- * using the TEZ_ENGINE_COMBINER_CLASS attribute in {@link TezJobConfig}
- * 
- * 
- * Partitioners need to provide a single argument ({@link TezTaskContext})
- * constructor.
- */
-@Unstable
-@LimitedPrivate("mapreduce")
-public interface Combiner {
-  public void combine(TezRawKeyValueIterator rawIter, Writer writer)
-      throws InterruptedException, IOException;
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/engine/common/localshuffle/LocalShuffle.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/localshuffle/LocalShuffle.java b/tez-engine/src/main/java/org/apache/tez/engine/common/localshuffle/LocalShuffle.java
deleted file mode 100644
index 546151f..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/localshuffle/LocalShuffle.java
+++ /dev/null
@@ -1,120 +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.engine.common.localshuffle;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.RawComparator;
-import org.apache.hadoop.io.compress.CompressionCodec;
-import org.apache.hadoop.io.compress.DefaultCodec;
-import org.apache.hadoop.util.ReflectionUtils;
-import org.apache.tez.common.TezJobConfig;
-import org.apache.tez.common.counters.TaskCounter;
-import org.apache.tez.common.counters.TezCounter;
-import org.apache.tez.engine.api.TezInputContext;
-import org.apache.tez.engine.common.ConfigUtils;
-import org.apache.tez.engine.common.sort.impl.TezMerger;
-import org.apache.tez.engine.common.sort.impl.TezRawKeyValueIterator;
-import org.apache.tez.engine.common.task.local.output.TezLocalTaskOutputFiles;
-import org.apache.tez.engine.common.task.local.output.TezTaskOutput;
-
-@SuppressWarnings({"rawtypes"})
-public class LocalShuffle {
-
-  // TODO NEWTEZ This is broken.
-
-  private final TezInputContext inputContext;
-  private final Configuration conf;
-  private final int numInputs;
-
-  private final Class keyClass;
-  private final Class valClass;
-  private final RawComparator comparator;
-
-  private final FileSystem rfs;
-  private final int sortFactor;
-  
-  private final TezCounter spilledRecordsCounter;
-  private final CompressionCodec codec;
-  private final TezTaskOutput mapOutputFile;
-
-  public LocalShuffle(TezInputContext inputContext, Configuration conf, int numInputs) throws IOException {
-    this.inputContext = inputContext;
-    this.conf = conf;
-    this.numInputs = numInputs;
-    
-    this.keyClass = ConfigUtils.getIntermediateInputKeyClass(conf);
-    this.valClass = ConfigUtils.getIntermediateInputValueClass(conf);
-    this.comparator = ConfigUtils.getIntermediateInputKeyComparator(conf);
-    
-    this.sortFactor =
-        conf.getInt(
-            TezJobConfig.TEZ_ENGINE_IO_SORT_FACTOR, 
-            TezJobConfig.DEFAULT_TEZ_ENGINE_IO_SORT_FACTOR);
-    
-    this.rfs = FileSystem.getLocal(conf).getRaw();
-
-    this.spilledRecordsCounter = inputContext.getCounters().findCounter(TaskCounter.SPILLED_RECORDS);
-    
- // compression
-    if (ConfigUtils.isIntermediateInputCompressed(conf)) {
-      Class<? extends CompressionCodec> codecClass =
-          ConfigUtils.getIntermediateInputCompressorClass(conf, DefaultCodec.class);
-      this.codec = ReflectionUtils.newInstance(codecClass, conf);
-    } else {
-      this.codec = null;
-    }
-    
-    // Always local
-    this.mapOutputFile = new TezLocalTaskOutputFiles(conf, inputContext.getUniqueIdentifier());
-  }
- 
-  
-  public TezRawKeyValueIterator run() throws IOException {
-    // Copy is complete, obviously! 
-
-    
-    // Merge
-    return TezMerger.merge(conf, rfs, 
-        keyClass, valClass,
-        codec, 
-        getMapFiles(),
-        false, 
-        sortFactor,
-        new Path(inputContext.getUniqueIdentifier()), // TODO NEWTEZ This is likely broken 
-        comparator,
-        null, spilledRecordsCounter, null, null);
-  }
-  
-  private Path[] getMapFiles() 
-  throws IOException {
-    List<Path> fileList = new ArrayList<Path>();
-      // for local jobs
-      for(int i = 0; i < numInputs; ++i) {
-        //fileList.add(mapOutputFile.getInputFile(i));
-      }
-      
-    return fileList.toArray(new Path[0]);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/engine/common/objectregistry/ObjectRegistryImpl.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/objectregistry/ObjectRegistryImpl.java b/tez-engine/src/main/java/org/apache/tez/engine/common/objectregistry/ObjectRegistryImpl.java
deleted file mode 100644
index 351e01c..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/objectregistry/ObjectRegistryImpl.java
+++ /dev/null
@@ -1,65 +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.engine.common.objectregistry;
-
-import java.util.AbstractMap;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Map.Entry;
-
-import com.google.inject.Singleton;
-
-@Singleton
-public class ObjectRegistryImpl implements ObjectRegistry {
-
-  private Map<String, Map.Entry<Object, ObjectLifeCycle>> objectCache =
-      new HashMap<String, Map.Entry<Object, ObjectLifeCycle>>();
-
-  @Override
-  public synchronized Object add(ObjectLifeCycle lifeCycle,
-      String key, Object value) {
-    Map.Entry<Object, ObjectLifeCycle> oldEntry =
-        objectCache.put(key,
-            new AbstractMap.SimpleImmutableEntry<Object, ObjectLifeCycle>(
-                value, lifeCycle));
-    return oldEntry != null ? oldEntry.getKey() : null;
-  }
-
-  @Override
-  public synchronized Object get(String key) {
-    Map.Entry<Object, ObjectLifeCycle> entry =
-        objectCache.get(key);
-    return entry != null ? entry.getKey() : null;
-  }
-
-  @Override
-  public synchronized boolean delete(String key) {
-    return (null != objectCache.remove(key));
-  }
-
-  public synchronized void clearCache(ObjectLifeCycle lifeCycle) {
-    for (Entry<String, Entry<Object, ObjectLifeCycle>> entry :
-      objectCache.entrySet()) {
-      if (entry.getValue().getValue().equals(lifeCycle)) {
-        objectCache.remove(entry.getKey());
-      }
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/engine/common/objectregistry/ObjectRegistryModule.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/objectregistry/ObjectRegistryModule.java b/tez-engine/src/main/java/org/apache/tez/engine/common/objectregistry/ObjectRegistryModule.java
deleted file mode 100644
index ab346fd..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/objectregistry/ObjectRegistryModule.java
+++ /dev/null
@@ -1,43 +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.engine.common.objectregistry;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.inject.AbstractModule;
-
-public class ObjectRegistryModule extends AbstractModule {
-
-  private final ObjectRegistry objectRegistry;
-
-  public ObjectRegistryModule(ObjectRegistry objectRegistry) {
-    this.objectRegistry = objectRegistry;
-  }
-
-  @VisibleForTesting
-  public ObjectRegistryModule() {
-    objectRegistry = new ObjectRegistryImpl();
-  }
-
-  @Override
-  protected void configure() {
-    bind(ObjectRegistry.class).toInstance(this.objectRegistry);
-    requestStaticInjection(ObjectRegistryFactory.class);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/engine/common/security/JobTokenIdentifier.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/security/JobTokenIdentifier.java b/tez-engine/src/main/java/org/apache/tez/engine/common/security/JobTokenIdentifier.java
deleted file mode 100644
index 827001b..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/security/JobTokenIdentifier.java
+++ /dev/null
@@ -1,98 +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.engine.common.security;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.security.token.Token;
-import org.apache.hadoop.security.token.TokenIdentifier;
-
-/**
- * The token identifier for job token
- */
-@InterfaceAudience.Private
-@InterfaceStability.Unstable
-public class JobTokenIdentifier extends TokenIdentifier {
-  private Text jobid;
-  public final static Text KIND_NAME = new Text("mapreduce.job");
-  
-  /**
-   * Default constructor
-   */
-  public JobTokenIdentifier() {
-    this.jobid = new Text();
-  }
-
-  /**
-   * Create a job token identifier from a jobid
-   * @param jobid the jobid to use
-   */
-  public JobTokenIdentifier(Text jobid) {
-    this.jobid = jobid;
-  }
-
-  /** {@inheritDoc} */
-  @Override
-  public Text getKind() {
-    return KIND_NAME;
-  }
-  
-  /** {@inheritDoc} */
-  @Override
-  public UserGroupInformation getUser() {
-    if (jobid == null || "".equals(jobid.toString())) {
-      return null;
-    }
-    return UserGroupInformation.createRemoteUser(jobid.toString());
-  }
-  
-  /**
-   * Get the jobid
-   * @return the jobid
-   */
-  public Text getJobId() {
-    return jobid;
-  }
-
-  /** {@inheritDoc} */
-  @Override
-  public void readFields(DataInput in) throws IOException {
-    jobid.readFields(in);
-  }
-
-  /** {@inheritDoc} */
-  @Override
-  public void write(DataOutput out) throws IOException {
-    jobid.write(out);
-  }
-
-  @InterfaceAudience.Private
-  public static class Renewer extends Token.TrivialRenewer {
-    @Override
-    protected Text getKind() {
-      return KIND_NAME;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/engine/common/security/JobTokenSecretManager.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/security/JobTokenSecretManager.java b/tez-engine/src/main/java/org/apache/tez/engine/common/security/JobTokenSecretManager.java
deleted file mode 100644
index d957b8b..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/security/JobTokenSecretManager.java
+++ /dev/null
@@ -1,137 +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.engine.common.security;
-
-import java.util.Map;
-import java.util.TreeMap;
-
-import javax.crypto.SecretKey;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.security.token.SecretManager;
-import org.apache.hadoop.security.token.Token;
-
-/**
- * SecretManager for job token. It can be used to cache generated job tokens.
- */
-@InterfaceAudience.Private
-@InterfaceStability.Unstable
-public class JobTokenSecretManager extends SecretManager<JobTokenIdentifier> {
-  private final SecretKey masterKey;
-  private final Map<String, SecretKey> currentJobTokens;
-
-  /**
-   * Convert the byte[] to a secret key
-   * @param key the byte[] to create the secret key from
-   * @return the secret key
-   */
-  public static SecretKey createSecretKey(byte[] key) {
-    return SecretManager.createSecretKey(key);
-  }
-  
-  /**
-   * Compute the HMAC hash of the message using the key
-   * @param msg the message to hash
-   * @param key the key to use
-   * @return the computed hash
-   */
-  public static byte[] computeHash(byte[] msg, SecretKey key) {
-    return createPassword(msg, key);
-  }
-  
-  /**
-   * Default constructor
-   */
-  public JobTokenSecretManager() {
-    this.masterKey = generateSecret();
-    this.currentJobTokens = new TreeMap<String, SecretKey>();
-  }
-  
-  /**
-   * Create a new password/secret for the given job token identifier.
-   * @param identifier the job token identifier
-   * @return token password/secret
-   */
-  @Override
-  public byte[] createPassword(JobTokenIdentifier identifier) {
-    byte[] result = createPassword(identifier.getBytes(), masterKey);
-    return result;
-  }
-
-  /**
-   * Add the job token of a job to cache
-   * @param jobId the job that owns the token
-   * @param token the job token
-   */
-  public void addTokenForJob(String jobId, Token<JobTokenIdentifier> token) {
-    SecretKey tokenSecret = createSecretKey(token.getPassword());
-    synchronized (currentJobTokens) {
-      currentJobTokens.put(jobId, tokenSecret);
-    }
-  }
-
-  /**
-   * Remove the cached job token of a job from cache
-   * @param jobId the job whose token is to be removed
-   */
-  public void removeTokenForJob(String jobId) {
-    synchronized (currentJobTokens) {
-      currentJobTokens.remove(jobId);
-    }
-  }
-  
-  /**
-   * Look up the token password/secret for the given jobId.
-   * @param jobId the jobId to look up
-   * @return token password/secret as SecretKey
-   * @throws InvalidToken
-   */
-  public SecretKey retrieveTokenSecret(String jobId) throws InvalidToken {
-    SecretKey tokenSecret = null;
-    synchronized (currentJobTokens) {
-      tokenSecret = currentJobTokens.get(jobId);
-    }
-    if (tokenSecret == null) {
-      throw new InvalidToken("Can't find job token for job " + jobId + " !!");
-    }
-    return tokenSecret;
-  }
-  
-  /**
-   * Look up the token password/secret for the given job token identifier.
-   * @param identifier the job token identifier to look up
-   * @return token password/secret as byte[]
-   * @throws InvalidToken
-   */
-  @Override
-  public byte[] retrievePassword(JobTokenIdentifier identifier)
-      throws InvalidToken {
-    return retrieveTokenSecret(identifier.getJobId().toString()).getEncoded();
-  }
-
-  /**
-   * Create an empty job token identifier
-   * @return a newly created empty job token identifier
-   */
-  @Override
-  public JobTokenIdentifier createIdentifier() {
-    return new JobTokenIdentifier();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/engine/common/security/JobTokenSelector.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/security/JobTokenSelector.java b/tez-engine/src/main/java/org/apache/tez/engine/common/security/JobTokenSelector.java
deleted file mode 100644
index 080b9e2..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/security/JobTokenSelector.java
+++ /dev/null
@@ -1,53 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tez.engine.common.security;
-
-import java.util.Collection;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.security.token.Token;
-import org.apache.hadoop.security.token.TokenIdentifier;
-import org.apache.hadoop.security.token.TokenSelector;
-
-/**
- * Look through tokens to find the first job token that matches the service
- * and return it.
- */
-@InterfaceAudience.Private
-@InterfaceStability.Unstable
-public class JobTokenSelector implements TokenSelector<JobTokenIdentifier> {
-
-  @SuppressWarnings("unchecked")
-  @Override
-  public Token<JobTokenIdentifier> selectToken(Text service,
-      Collection<Token<? extends TokenIdentifier>> tokens) {
-    if (service == null) {
-      return null;
-    }
-    for (Token<? extends TokenIdentifier> token : tokens) {
-      if (JobTokenIdentifier.KIND_NAME.equals(token.getKind())
-          && service.equals(token.getService())) {
-        return (Token<JobTokenIdentifier>) token;
-      }
-    }
-    return null;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/engine/common/security/Master.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/security/Master.java b/tez-engine/src/main/java/org/apache/tez/engine/common/security/Master.java
deleted file mode 100644
index 20cad0a..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/security/Master.java
+++ /dev/null
@@ -1,57 +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.engine.common.security;
-
-import java.io.IOException;
-import java.net.InetSocketAddress;
-
-import org.apache.hadoop.classification.InterfaceAudience.Private;
-import org.apache.hadoop.classification.InterfaceStability.Unstable;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.security.SecurityUtil;
-import org.apache.hadoop.yarn.conf.YarnConfiguration;
-
-@Private
-@Unstable
-public class Master {
-
-  public enum State {
-    INITIALIZING, RUNNING;
-  }
-
-  public static String getMasterUserName(Configuration conf) {
-    return conf.get(YarnConfiguration.RM_PRINCIPAL);
-  }
-
-  public static InetSocketAddress getMasterAddress(Configuration conf) {
-    return conf
-        .getSocketAddr(YarnConfiguration.RM_ADDRESS,
-            YarnConfiguration.DEFAULT_RM_ADDRESS,
-            YarnConfiguration.DEFAULT_RM_PORT);
-  }
-
-  public static String getMasterPrincipal(Configuration conf)
-      throws IOException {
-    String masterHostname = getMasterAddress(conf).getHostName();
-    // get kerberos principal for use as delegation token renewer
-    return SecurityUtil.getServerPrincipal(getMasterUserName(conf),
-        masterHostname);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/engine/common/security/SecureShuffleUtils.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/security/SecureShuffleUtils.java b/tez-engine/src/main/java/org/apache/tez/engine/common/security/SecureShuffleUtils.java
deleted file mode 100644
index 71d990a..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/security/SecureShuffleUtils.java
+++ /dev/null
@@ -1,142 +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.engine.common.security;
-
-
-import java.io.ByteArrayOutputStream;
-import java.io.IOException;
-import java.io.PrintStream;
-import java.net.URL;
-
-import javax.crypto.SecretKey;
-import javax.servlet.http.HttpServletRequest;
-
-import org.apache.commons.codec.binary.Base64;
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.io.WritableComparator;
-
-/**
- * 
- * utilities for generating kyes, hashes and verifying them for shuffle
- *
- */
-@InterfaceAudience.Private
-@InterfaceStability.Unstable
-public class SecureShuffleUtils {
-  public static final String HTTP_HEADER_URL_HASH = "UrlHash";
-  public static final String HTTP_HEADER_REPLY_URL_HASH = "ReplyHash";
-  
-  /**
-   * Base64 encoded hash of msg
-   * @param msg
-   */
-  public static String generateHash(byte[] msg, SecretKey key) {
-    return new String(Base64.encodeBase64(generateByteHash(msg, key)));
-  }
-  
-  /**
-   * calculate hash of msg
-   * @param msg
-   * @return
-   */
-  private static byte[] generateByteHash(byte[] msg, SecretKey key) {
-    return JobTokenSecretManager.computeHash(msg, key);
-  }
-  
-  /**
-   * verify that hash equals to HMacHash(msg)
-   * @param newHash
-   * @return true if is the same
-   */
-  private static boolean verifyHash(byte[] hash, byte[] msg, SecretKey key) {
-    byte[] msg_hash = generateByteHash(msg, key);
-    return WritableComparator.compareBytes(msg_hash, 0, msg_hash.length, hash, 0, hash.length) == 0;
-  }
-  
-  /**
-   * Aux util to calculate hash of a String
-   * @param enc_str
-   * @param key
-   * @return Base64 encodedHash
-   * @throws IOException
-   */
-  public static String hashFromString(String enc_str, SecretKey key) 
-  throws IOException {
-    return generateHash(enc_str.getBytes(), key); 
-  }
-  
-  /**
-   * verify that base64Hash is same as HMacHash(msg)  
-   * @param base64Hash (Base64 encoded hash)
-   * @param msg
-   * @throws IOException if not the same
-   */
-  public static void verifyReply(String base64Hash, String msg, SecretKey key)
-  throws IOException {
-    byte[] hash = Base64.decodeBase64(base64Hash.getBytes());
-    
-    boolean res = verifyHash(hash, msg.getBytes(), key);
-    
-    if(res != true) {
-      throw new IOException("Verification of the hashReply failed");
-    }
-  }
-  
-  /**
-   * Shuffle specific utils - build string for encoding from URL
-   * @param url
-   * @return string for encoding
-   */
-  public static String buildMsgFrom(URL url) {
-    return buildMsgFrom(url.getPath(), url.getQuery(), url.getPort());
-  }
-  /**
-   * Shuffle specific utils - build string for encoding from URL
-   * @param request
-   * @return string for encoding
-   */
-  public static String buildMsgFrom(HttpServletRequest request ) {
-    return buildMsgFrom(request.getRequestURI(), request.getQueryString(),
-        request.getLocalPort());
-  }
-  /**
-   * Shuffle specific utils - build string for encoding from URL
-   * @param uri_path
-   * @param uri_query
-   * @return string for encoding
-   */
-  private static String buildMsgFrom(String uri_path, String uri_query, int port) {
-    return String.valueOf(port) + uri_path + "?" + uri_query;
-  }
-  
-  
-  /**
-   * byte array to Hex String
-   * @param ba
-   * @return string with HEX value of the key
-   */
-  public static String toHex(byte[] ba) {
-    ByteArrayOutputStream baos = new ByteArrayOutputStream();
-    PrintStream ps = new PrintStream(baos);
-    for(byte b: ba) {
-      ps.printf("%x", b);
-    }
-    return baos.toString();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/engine/common/security/TokenCache.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/security/TokenCache.java b/tez-engine/src/main/java/org/apache/tez/engine/common/security/TokenCache.java
deleted file mode 100644
index 5c42d22..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/security/TokenCache.java
+++ /dev/null
@@ -1,205 +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.engine.common.security;
-
-import java.io.IOException;
-import java.util.HashSet;
-import java.util.Set;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.security.Credentials;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.security.token.Token;
-import org.apache.hadoop.security.token.TokenIdentifier;
-import org.apache.tez.common.TezJobConfig;
-
-
-/**
- * This class provides user facing APIs for transferring secrets from
- * the job client to the tasks.
- * The secrets can be stored just before submission of jobs and read during
- * the task execution.  
- */
-@InterfaceAudience.Public
-@InterfaceStability.Evolving
-public class TokenCache {
-  
-  private static final Log LOG = LogFactory.getLog(TokenCache.class);
-
-  
-  /**
-   * auxiliary method to get user's secret keys..
-   * @param alias
-   * @return secret key from the storage
-   */
-  public static byte[] getSecretKey(Credentials credentials, Text alias) {
-    if(credentials == null)
-      return null;
-    return credentials.getSecretKey(alias);
-  }
-  
-  /**
-   * Convenience method to obtain delegation tokens from namenodes 
-   * corresponding to the paths passed.
-   * @param credentials
-   * @param ps array of paths
-   * @param conf configuration
-   * @throws IOException
-   */
-  public static void obtainTokensForNamenodes(Credentials credentials,
-      Path[] ps, Configuration conf) throws IOException {
-    if (!UserGroupInformation.isSecurityEnabled()) {
-      return;
-    }
-    obtainTokensForNamenodesInternal(credentials, ps, conf);
-  }
-
-  /**
-   * Remove jobtoken referrals which don't make sense in the context
-   * of the task execution.
-   *
-   * @param conf
-   */
-  public static void cleanUpTokenReferral(Configuration conf) {
-    conf.unset(TezJobConfig.DAG_CREDENTIALS_BINARY);
-  }
-
-  static void obtainTokensForNamenodesInternal(Credentials credentials,
-      Path[] ps, Configuration conf) throws IOException {
-    Set<FileSystem> fsSet = new HashSet<FileSystem>();
-    for(Path p: ps) {
-      fsSet.add(p.getFileSystem(conf));
-    }
-    for (FileSystem fs : fsSet) {
-      obtainTokensForNamenodesInternal(fs, credentials, conf);
-    }
-  }
-
-  /**
-   * get delegation token for a specific FS
-   * @param fs
-   * @param credentials
-   * @param p
-   * @param conf
-   * @throws IOException
-   */
-  static void obtainTokensForNamenodesInternal(FileSystem fs, 
-      Credentials credentials, Configuration conf) throws IOException {
-    String delegTokenRenewer = Master.getMasterPrincipal(conf);
-    if (delegTokenRenewer == null || delegTokenRenewer.length() == 0) {
-      throw new IOException(
-          "Can't get Master Kerberos principal for use as renewer");
-    }
-    mergeBinaryTokens(credentials, conf);
-
-    final Token<?> tokens[] = fs.addDelegationTokens(delegTokenRenewer,
-                                                     credentials);
-    if (tokens != null) {
-      for (Token<?> token : tokens) {
-        LOG.info("Got dt for " + fs.getUri() + "; "+token);
-      }
-    }
-  }
-
-  private static void mergeBinaryTokens(Credentials creds, Configuration conf) {
-    String binaryTokenFilename =
-        conf.get(TezJobConfig.DAG_CREDENTIALS_BINARY);
-    if (binaryTokenFilename != null) {
-      Credentials binary;
-      try {
-        binary = Credentials.readTokenStorageFile(
-            new Path("file:///" +  binaryTokenFilename), conf);
-      } catch (IOException e) {
-        throw new RuntimeException(e);
-      }
-      // supplement existing tokens with the tokens in the binary file
-      creds.mergeAll(binary);
-    }
-  }
-  
-  /**
-   * file name used on HDFS for generated job token
-   */
-  @InterfaceAudience.Private
-  public static final String JOB_TOKEN_HDFS_FILE = "jobToken";
-
-  /**
-   * conf setting for job tokens cache file name
-   */
-  @InterfaceAudience.Private
-  public static final String JOB_TOKENS_FILENAME = "mapreduce.job.jobTokenFile";
-  private static final Text JOB_TOKEN = new Text("JobToken");
-  private static final Text SHUFFLE_TOKEN = new Text("MapReduceShuffleToken");
-  
-  /**
-   * load job token from a file
-   * @param conf
-   * @throws IOException
-   */
-  @InterfaceAudience.Private
-  public static Credentials loadTokens(String jobTokenFile, Configuration conf) 
-  throws IOException {
-    Path localJobTokenFile = new Path ("file:///" + jobTokenFile);
-
-    Credentials ts = Credentials.readTokenStorageFile(localJobTokenFile, conf);
-
-    if(LOG.isDebugEnabled()) {
-      LOG.debug("Task: Loaded jobTokenFile from: "+
-          localJobTokenFile.toUri().getPath() 
-          +"; num of sec keys  = " + ts.numberOfSecretKeys() +
-          " Number of tokens " +  ts.numberOfTokens());
-    }
-    return ts;
-  }
-  /**
-   * store job token
-   * @param t
-   */
-  @InterfaceAudience.Private
-  public static void setJobToken(Token<? extends TokenIdentifier> t, 
-      Credentials credentials) {
-    credentials.addToken(JOB_TOKEN, t);
-  }
-  /**
-   * 
-   * @return job token
-   */
-  @SuppressWarnings("unchecked")
-  @InterfaceAudience.Private
-  public static Token<JobTokenIdentifier> getJobToken(Credentials credentials) {
-    return (Token<JobTokenIdentifier>) credentials.getToken(JOB_TOKEN);
-  }
-
-  @InterfaceAudience.Private
-  public static void setShuffleSecretKey(byte[] key, Credentials credentials) {
-    credentials.addSecretKey(SHUFFLE_TOKEN, key);
-  }
-
-  @InterfaceAudience.Private
-  public static byte[] getShuffleSecretKey(Credentials credentials) {
-    return getSecretKey(credentials, SHUFFLE_TOKEN);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/ExceptionReporter.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/ExceptionReporter.java b/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/ExceptionReporter.java
deleted file mode 100644
index 89a5888..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/ExceptionReporter.java
+++ /dev/null
@@ -1,25 +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.engine.common.shuffle.impl;
-
-/**
- * An interface for reporting exceptions to other threads
- */
-interface ExceptionReporter {
-  void reportException(Throwable t);
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/Fetcher.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/Fetcher.java b/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/Fetcher.java
deleted file mode 100644
index b48bb0b..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/Fetcher.java
+++ /dev/null
@@ -1,624 +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.engine.common.shuffle.impl;
-
-import java.io.DataInputStream;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.net.HttpURLConnection;
-import java.net.MalformedURLException;
-import java.net.URL;
-import java.net.URLConnection;
-import java.security.GeneralSecurityException;
-import java.util.Arrays;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-
-import javax.crypto.SecretKey;
-import javax.net.ssl.HttpsURLConnection;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.io.IOUtils;
-import org.apache.hadoop.io.compress.CodecPool;
-import org.apache.hadoop.io.compress.CompressionCodec;
-import org.apache.hadoop.io.compress.Decompressor;
-import org.apache.hadoop.io.compress.DefaultCodec;
-import org.apache.hadoop.security.ssl.SSLFactory;
-import org.apache.hadoop.util.ReflectionUtils;
-import org.apache.tez.common.TezJobConfig;
-import org.apache.tez.common.counters.TezCounter;
-import org.apache.tez.engine.api.TezInputContext;
-import org.apache.tez.engine.common.ConfigUtils;
-import org.apache.tez.engine.common.InputAttemptIdentifier;
-import org.apache.tez.engine.common.security.SecureShuffleUtils;
-import org.apache.tez.engine.common.shuffle.impl.MapOutput.Type;
-import org.apache.tez.engine.common.sort.impl.IFileInputStream;
-
-import com.google.common.annotations.VisibleForTesting;
-
-class Fetcher extends Thread {
-  
-  private static final Log LOG = LogFactory.getLog(Fetcher.class);
-  
-  /** Basic/unit connection timeout (in milliseconds) */
-  private final static int UNIT_CONNECT_TIMEOUT = 60 * 1000;
-
-  private static enum ShuffleErrors{IO_ERROR, WRONG_LENGTH, BAD_ID, WRONG_MAP,
-                                    CONNECTION, WRONG_REDUCE}
-  
-  private final static String SHUFFLE_ERR_GRP_NAME = "Shuffle Errors";
-  private final TezCounter connectionErrs;
-  private final TezCounter ioErrs;
-  private final TezCounter wrongLengthErrs;
-  private final TezCounter badIdErrs;
-  private final TezCounter wrongMapErrs;
-  private final TezCounter wrongReduceErrs;
-  private final MergeManager merger;
-  private final ShuffleScheduler scheduler;
-  private final ShuffleClientMetrics metrics;
-  private final Shuffle shuffle;
-  private final int id;
-  private static int nextId = 0;
-  
-  private final int connectionTimeout;
-  private final int readTimeout;
-  
-  // Decompression of map-outputs
-  private final CompressionCodec codec;
-  private final Decompressor decompressor;
-  private final SecretKey jobTokenSecret;
-
-  private volatile boolean stopped = false;
-
-  private Configuration job;
-
-  private static boolean sslShuffle;
-  private static SSLFactory sslFactory;
-
-  public Fetcher(Configuration job, 
-      ShuffleScheduler scheduler, MergeManager merger,
-      ShuffleClientMetrics metrics,
-      Shuffle shuffle, SecretKey jobTokenSecret, TezInputContext inputContext) throws IOException {
-    this.job = job;
-    this.scheduler = scheduler;
-    this.merger = merger;
-    this.metrics = metrics;
-    this.shuffle = shuffle;
-    this.id = ++nextId;
-    this.jobTokenSecret = jobTokenSecret;
-    ioErrs = inputContext.getCounters().findCounter(SHUFFLE_ERR_GRP_NAME,
-        ShuffleErrors.IO_ERROR.toString());
-    wrongLengthErrs = inputContext.getCounters().findCounter(SHUFFLE_ERR_GRP_NAME,
-        ShuffleErrors.WRONG_LENGTH.toString());
-    badIdErrs = inputContext.getCounters().findCounter(SHUFFLE_ERR_GRP_NAME,
-        ShuffleErrors.BAD_ID.toString());
-    wrongMapErrs = inputContext.getCounters().findCounter(SHUFFLE_ERR_GRP_NAME,
-        ShuffleErrors.WRONG_MAP.toString());
-    connectionErrs = inputContext.getCounters().findCounter(SHUFFLE_ERR_GRP_NAME,
-        ShuffleErrors.CONNECTION.toString());
-    wrongReduceErrs = inputContext.getCounters().findCounter(SHUFFLE_ERR_GRP_NAME,
-        ShuffleErrors.WRONG_REDUCE.toString());
-
-    if (ConfigUtils.isIntermediateInputCompressed(job)) {
-      Class<? extends CompressionCodec> codecClass =
-          ConfigUtils.getIntermediateInputCompressorClass(job, DefaultCodec.class);
-      codec = ReflectionUtils.newInstance(codecClass, job);
-      decompressor = CodecPool.getDecompressor(codec);
-    } else {
-      codec = null;
-      decompressor = null;
-    }
-
-    this.connectionTimeout = 
-        job.getInt(TezJobConfig.TEZ_ENGINE_SHUFFLE_CONNECT_TIMEOUT,
-            TezJobConfig.DEFAULT_TEZ_ENGINE_SHUFFLE_STALLED_COPY_TIMEOUT);
-    this.readTimeout = 
-        job.getInt(TezJobConfig.TEZ_ENGINE_SHUFFLE_READ_TIMEOUT, 
-            TezJobConfig.DEFAULT_TEZ_ENGINE_SHUFFLE_READ_TIMEOUT);
-
-    setName("fetcher#" + id);
-    setDaemon(true);
-
-    synchronized (Fetcher.class) {
-      sslShuffle = job.getBoolean(TezJobConfig.TEZ_ENGINE_SHUFFLE_ENABLE_SSL,
-          TezJobConfig.DEFAULT_TEZ_ENGINE_SHUFFLE_ENABLE_SSL);
-      if (sslShuffle && sslFactory == null) {
-        sslFactory = new SSLFactory(SSLFactory.Mode.CLIENT, job);
-        try {
-          sslFactory.init();
-        } catch (Exception ex) {
-          sslFactory.destroy();
-          throw new RuntimeException(ex);
-        }
-      }
-    }
-  }
-  
-  public void run() {
-    try {
-      while (!stopped && !Thread.currentThread().isInterrupted()) {
-        MapHost host = null;
-        try {
-          // If merge is on, block
-          merger.waitForInMemoryMerge();
-
-          // Get a host to shuffle from
-          host = scheduler.getHost();
-          metrics.threadBusy();
-
-          // Shuffle
-          copyFromHost(host);
-        } finally {
-          if (host != null) {
-            scheduler.freeHost(host);
-            metrics.threadFree();            
-          }
-        }
-      }
-    } catch (InterruptedException ie) {
-      return;
-    } catch (Throwable t) {
-      shuffle.reportException(t);
-    }
-  }
-
-  public void shutDown() throws InterruptedException {
-    this.stopped = true;
-    interrupt();
-    try {
-      join(5000);
-    } catch (InterruptedException ie) {
-      LOG.warn("Got interrupt while joining " + getName(), ie);
-    }
-    if (sslFactory != null) {
-      sslFactory.destroy();
-    }
-  }
-
-  @VisibleForTesting
-  protected HttpURLConnection openConnection(URL url) throws IOException {
-    HttpURLConnection conn = (HttpURLConnection) url.openConnection();
-    if (sslShuffle) {
-      HttpsURLConnection httpsConn = (HttpsURLConnection) conn;
-      try {
-        httpsConn.setSSLSocketFactory(sslFactory.createSSLSocketFactory());
-      } catch (GeneralSecurityException ex) {
-        throw new IOException(ex);
-      }
-      httpsConn.setHostnameVerifier(sslFactory.getHostnameVerifier());
-    }
-    return conn;
-  }
-  
-  /**
-   * The crux of the matter...
-   * 
-   * @param host {@link MapHost} from which we need to  
-   *              shuffle available map-outputs.
-   */
-  @VisibleForTesting
-  protected void copyFromHost(MapHost host) throws IOException {
-    // Get completed maps on 'host'
-    List<InputAttemptIdentifier> srcAttempts = scheduler.getMapsForHost(host);
-    
-    // Sanity check to catch hosts with only 'OBSOLETE' maps, 
-    // especially at the tail of large jobs
-    if (srcAttempts.size() == 0) {
-      return;
-    }
-    
-    if(LOG.isDebugEnabled()) {
-      LOG.debug("Fetcher " + id + " going to fetch from " + host + " for: "
-        + srcAttempts);
-    }
-    
-    // List of maps to be fetched yet
-    Set<InputAttemptIdentifier> remaining = new HashSet<InputAttemptIdentifier>(srcAttempts);
-    
-    // Construct the url and connect
-    DataInputStream input;
-    boolean connectSucceeded = false;
-    
-    try {
-      URL url = getMapOutputURL(host, srcAttempts);
-      HttpURLConnection connection = openConnection(url);
-      
-      // generate hash of the url
-      String msgToEncode = SecureShuffleUtils.buildMsgFrom(url);
-      String encHash = SecureShuffleUtils.hashFromString(msgToEncode, jobTokenSecret);
-      
-      // put url hash into http header
-      connection.addRequestProperty(
-          SecureShuffleUtils.HTTP_HEADER_URL_HASH, encHash);
-      // set the read timeout
-      connection.setReadTimeout(readTimeout);
-      // put shuffle version into http header
-      connection.addRequestProperty(ShuffleHeader.HTTP_HEADER_NAME,
-          ShuffleHeader.DEFAULT_HTTP_HEADER_NAME);
-      connection.addRequestProperty(ShuffleHeader.HTTP_HEADER_VERSION,
-          ShuffleHeader.DEFAULT_HTTP_HEADER_VERSION);
-      connect(connection, connectionTimeout);
-      connectSucceeded = true;
-      input = new DataInputStream(connection.getInputStream());
-
-      // Validate response code
-      int rc = connection.getResponseCode();
-      if (rc != HttpURLConnection.HTTP_OK) {
-        throw new IOException(
-            "Got invalid response code " + rc + " from " + url +
-            ": " + connection.getResponseMessage());
-      }
-      // get the shuffle version
-      if (!ShuffleHeader.DEFAULT_HTTP_HEADER_NAME.equals(
-          connection.getHeaderField(ShuffleHeader.HTTP_HEADER_NAME))
-          || !ShuffleHeader.DEFAULT_HTTP_HEADER_VERSION.equals(
-              connection.getHeaderField(ShuffleHeader.HTTP_HEADER_VERSION))) {
-        throw new IOException("Incompatible shuffle response version");
-      }
-      // get the replyHash which is HMac of the encHash we sent to the server
-      String replyHash = connection.getHeaderField(SecureShuffleUtils.HTTP_HEADER_REPLY_URL_HASH);
-      if(replyHash==null) {
-        throw new IOException("security validation of TT Map output failed");
-      }
-      LOG.debug("url="+msgToEncode+";encHash="+encHash+";replyHash="+replyHash);
-      // verify that replyHash is HMac of encHash
-      SecureShuffleUtils.verifyReply(replyHash, encHash, jobTokenSecret);
-      LOG.info("for url="+msgToEncode+" sent hash and receievd reply");
-    } catch (IOException ie) {
-      ioErrs.increment(1);
-      LOG.warn("Failed to connect to " + host + " with " + remaining.size() + 
-               " map outputs", ie);
-
-      // If connect did not succeed, just mark all the maps as failed,
-      // indirectly penalizing the host
-      if (!connectSucceeded) {
-        for(InputAttemptIdentifier left: remaining) {
-          scheduler.copyFailed(left, host, connectSucceeded);
-        }
-      } else {
-        // If we got a read error at this stage, it implies there was a problem
-        // with the first map, typically lost map. So, penalize only that map
-        // and add the rest
-        InputAttemptIdentifier firstMap = srcAttempts.get(0);
-        scheduler.copyFailed(firstMap, host, connectSucceeded);
-      }
-      
-      // Add back all the remaining maps, WITHOUT marking them as failed
-      for(InputAttemptIdentifier left: remaining) {
-        // TODO Should the first one be skipped ?
-        scheduler.putBackKnownMapOutput(host, left);
-      }
-      
-      return;
-    }
-    
-    try {
-      // Loop through available map-outputs and fetch them
-      // On any error, faildTasks is not null and we exit
-      // after putting back the remaining maps to the 
-      // yet_to_be_fetched list and marking the failed tasks.
-      InputAttemptIdentifier[] failedTasks = null;
-      while (!remaining.isEmpty() && failedTasks == null) {
-        failedTasks = copyMapOutput(host, input, remaining);
-      }
-      
-      if(failedTasks != null && failedTasks.length > 0) {
-        LOG.warn("copyMapOutput failed for tasks "+Arrays.toString(failedTasks));
-        for(InputAttemptIdentifier left: failedTasks) {
-          scheduler.copyFailed(left, host, true);
-        }
-      }
-      
-      IOUtils.cleanup(LOG, input);
-      
-      // Sanity check
-      if (failedTasks == null && !remaining.isEmpty()) {
-        throw new IOException("server didn't return all expected map outputs: "
-            + remaining.size() + " left.");
-      }
-    } finally {
-      for (InputAttemptIdentifier left : remaining) {
-        scheduler.putBackKnownMapOutput(host, left);
-      }
-    }
-  }
-  
-  private static InputAttemptIdentifier[] EMPTY_ATTEMPT_ID_ARRAY = new InputAttemptIdentifier[0];
-  
-  private InputAttemptIdentifier[] copyMapOutput(MapHost host,
-                                DataInputStream input,
-                                Set<InputAttemptIdentifier> remaining) {
-    MapOutput mapOutput = null;
-    InputAttemptIdentifier srcAttemptId = null;
-    long decompressedLength = -1;
-    long compressedLength = -1;
-    
-    try {
-      long startTime = System.currentTimeMillis();
-      int forReduce = -1;
-      //Read the shuffle header
-      try {
-        ShuffleHeader header = new ShuffleHeader();
-        header.readFields(input);
-        String pathComponent = header.mapId;
-        srcAttemptId = scheduler.getIdentifierForPathComponent(pathComponent);
-        compressedLength = header.compressedLength;
-        decompressedLength = header.uncompressedLength;
-        forReduce = header.forReduce;
-      } catch (IllegalArgumentException e) {
-        badIdErrs.increment(1);
-        LOG.warn("Invalid map id ", e);
-        //Don't know which one was bad, so consider all of them as bad
-        return remaining.toArray(new InputAttemptIdentifier[remaining.size()]);
-      }
-
- 
-      // Do some basic sanity verification
-      if (!verifySanity(compressedLength, decompressedLength, forReduce,
-          remaining, srcAttemptId)) {
-        return new InputAttemptIdentifier[] {srcAttemptId};
-      }
-      
-      if(LOG.isDebugEnabled()) {
-        LOG.debug("header: " + srcAttemptId + ", len: " + compressedLength + 
-            ", decomp len: " + decompressedLength);
-      }
-      
-      // Get the location for the map output - either in-memory or on-disk
-      mapOutput = merger.reserve(srcAttemptId, decompressedLength, id);
-      
-      // Check if we can shuffle *now* ...
-      if (mapOutput.getType() == Type.WAIT) {
-        LOG.info("fetcher#" + id + " - MergerManager returned Status.WAIT ...");
-        //Not an error but wait to process data.
-        return EMPTY_ATTEMPT_ID_ARRAY;
-      } 
-      
-      // Go!
-      LOG.info("fetcher#" + id + " about to shuffle output of map " + 
-               mapOutput.getAttemptIdentifier() + " decomp: " +
-               decompressedLength + " len: " + compressedLength + " to " +
-               mapOutput.getType());
-      if (mapOutput.getType() == Type.MEMORY) {
-        shuffleToMemory(host, mapOutput, input, 
-                        (int) decompressedLength, (int) compressedLength);
-      } else {
-        shuffleToDisk(host, mapOutput, input, compressedLength);
-      }
-      
-      // Inform the shuffle scheduler
-      long endTime = System.currentTimeMillis();
-      scheduler.copySucceeded(srcAttemptId, host, compressedLength, 
-                              endTime - startTime, mapOutput);
-      // Note successful shuffle
-      remaining.remove(srcAttemptId);
-      metrics.successFetch();
-      return null;
-    } catch (IOException ioe) {
-      ioErrs.increment(1);
-      if (srcAttemptId == null || mapOutput == null) {
-        LOG.info("fetcher#" + id + " failed to read map header" + 
-                 srcAttemptId + " decomp: " + 
-                 decompressedLength + ", " + compressedLength, ioe);
-        if(srcAttemptId == null) {
-          return remaining.toArray(new InputAttemptIdentifier[remaining.size()]);
-        } else {
-          return new InputAttemptIdentifier[] {srcAttemptId};
-        }
-      }
-      
-      LOG.warn("Failed to shuffle output of " + srcAttemptId + 
-               " from " + host.getHostName(), ioe); 
-
-      // Inform the shuffle-scheduler
-      mapOutput.abort();
-      metrics.failedFetch();
-      return new InputAttemptIdentifier[] {srcAttemptId};
-    }
-
-  }
-  
-  /**
-   * Do some basic verification on the input received -- Being defensive
-   * @param compressedLength
-   * @param decompressedLength
-   * @param forReduce
-   * @param remaining
-   * @param mapId
-   * @return true/false, based on if the verification succeeded or not
-   */
-  private boolean verifySanity(long compressedLength, long decompressedLength,
-      int forReduce, Set<InputAttemptIdentifier> remaining, InputAttemptIdentifier srcAttemptId) {
-    if (compressedLength < 0 || decompressedLength < 0) {
-      wrongLengthErrs.increment(1);
-      LOG.warn(getName() + " invalid lengths in map output header: id: " +
-          srcAttemptId + " len: " + compressedLength + ", decomp len: " + 
-               decompressedLength);
-      return false;
-    }
-    
-    int reduceStartId = shuffle.getReduceStartId();
-    int reduceRange = shuffle.getReduceRange();
-    if (forReduce < reduceStartId || forReduce >= reduceStartId+reduceRange) {
-      wrongReduceErrs.increment(1);
-      LOG.warn(getName() + " data for the wrong reduce map: " +
-               srcAttemptId + " len: " + compressedLength + " decomp len: " +
-               decompressedLength + " for reduce " + forReduce);
-      return false;
-    }
-
-    // Sanity check
-    if (!remaining.contains(srcAttemptId)) {
-      wrongMapErrs.increment(1);
-      LOG.warn("Invalid map-output! Received output for " + srcAttemptId);
-      return false;
-    }
-    
-    return true;
-  }
-
-  /**
-   * Create the map-output-url. This will contain all the map ids
-   * separated by commas
-   * @param host
-   * @param maps
-   * @return
-   * @throws MalformedURLException
-   */
-  private URL getMapOutputURL(MapHost host, List<InputAttemptIdentifier> srcAttempts
-                              )  throws MalformedURLException {
-    // Get the base url
-    StringBuffer url = new StringBuffer(host.getBaseUrl());
-    
-    boolean first = true;
-    for (InputAttemptIdentifier mapId : srcAttempts) {
-      if (!first) {
-        url.append(",");
-      }
-      url.append(mapId.getPathComponent());
-      first = false;
-    }
-   
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("MapOutput URL for " + host + " -> " + url.toString());
-    }
-    return new URL(url.toString());
-  }
-  
-  /** 
-   * The connection establishment is attempted multiple times and is given up 
-   * only on the last failure. Instead of connecting with a timeout of 
-   * X, we try connecting with a timeout of x < X but multiple times. 
-   */
-  private void connect(URLConnection connection, int connectionTimeout)
-  throws IOException {
-    int unit = 0;
-    if (connectionTimeout < 0) {
-      throw new IOException("Invalid timeout "
-                            + "[timeout = " + connectionTimeout + " ms]");
-    } else if (connectionTimeout > 0) {
-      unit = Math.min(UNIT_CONNECT_TIMEOUT, connectionTimeout);
-    }
-    // set the connect timeout to the unit-connect-timeout
-    connection.setConnectTimeout(unit);
-    while (true) {
-      try {
-        connection.connect();
-        break;
-      } catch (IOException ioe) {
-        // update the total remaining connect-timeout
-        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) {
-          throw ioe;
-        }
-
-        // reset the connect timeout for the last try
-        if (connectionTimeout < unit) {
-          unit = connectionTimeout;
-          // reset the connect time out for the final connect
-          connection.setConnectTimeout(unit);
-        }
-      }
-    }
-  }
-
-  private void shuffleToMemory(MapHost host, MapOutput mapOutput, 
-                               InputStream input, 
-                               int decompressedLength, 
-                               int compressedLength) throws IOException {    
-    IFileInputStream checksumIn = 
-      new IFileInputStream(input, compressedLength, job);
-
-    input = checksumIn;       
-  
-    // Are map-outputs compressed?
-    if (codec != null) {
-      decompressor.reset();
-      input = codec.createInputStream(input, decompressor);
-    }
-  
-    // Copy map-output into an in-memory buffer
-    byte[] shuffleData = mapOutput.getMemory();
-    
-    try {
-      IOUtils.readFully(input, shuffleData, 0, shuffleData.length);
-      metrics.inputBytes(shuffleData.length);
-      LOG.info("Read " + shuffleData.length + " bytes from map-output for " +
-               mapOutput.getAttemptIdentifier());
-    } catch (IOException ioe) {      
-      // Close the streams
-      IOUtils.cleanup(LOG, input);
-
-      // Re-throw
-      throw ioe;
-    }
-
-  }
-  
-  private void shuffleToDisk(MapHost host, MapOutput mapOutput, 
-                             InputStream input, 
-                             long compressedLength) 
-  throws IOException {
-    // Copy data to local-disk
-    OutputStream output = mapOutput.getDisk();
-    long bytesLeft = compressedLength;
-    try {
-      final int BYTES_TO_READ = 64 * 1024;
-      byte[] buf = new byte[BYTES_TO_READ];
-      while (bytesLeft > 0) {
-        int n = input.read(buf, 0, (int) Math.min(bytesLeft, BYTES_TO_READ));
-        if (n < 0) {
-          throw new IOException("read past end of stream reading " + 
-                                mapOutput.getAttemptIdentifier());
-        }
-        output.write(buf, 0, n);
-        bytesLeft -= n;
-        metrics.inputBytes(n);
-      }
-
-      LOG.info("Read " + (compressedLength - bytesLeft) + 
-               " bytes from map-output for " +
-               mapOutput.getAttemptIdentifier());
-
-      output.close();
-    } catch (IOException ioe) {
-      // Close the streams
-      IOUtils.cleanup(LOG, input, output);
-
-      // Re-throw
-      throw ioe;
-    }
-
-    // Sanity check
-    if (bytesLeft != 0) {
-      throw new IOException("Incomplete map output received for " +
-                            mapOutput.getAttemptIdentifier() + " from " +
-                            host.getHostName() + " (" + 
-                            bytesLeft + " bytes missing of " + 
-                            compressedLength + ")"
-      );
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/InMemoryReader.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/InMemoryReader.java b/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/InMemoryReader.java
deleted file mode 100644
index 1beed44..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/InMemoryReader.java
+++ /dev/null
@@ -1,156 +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.engine.common.shuffle.impl;
-
-import java.io.File;
-import java.io.FileOutputStream;
-import java.io.IOException;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.io.DataInputBuffer;
-import org.apache.tez.engine.common.InputAttemptIdentifier;
-import org.apache.tez.engine.common.sort.impl.IFile;
-import org.apache.tez.engine.common.sort.impl.IFile.Reader;
-
-/**
- * <code>IFile.InMemoryReader</code> to read map-outputs present in-memory.
- */
-@InterfaceAudience.Private
-@InterfaceStability.Unstable
-public class InMemoryReader extends Reader {
-  private final InputAttemptIdentifier taskAttemptId;
-  private final MergeManager merger;
-  DataInputBuffer memDataIn = new DataInputBuffer();
-  private int start;
-  private int length;
-  private int prevKeyPos;
-
-  public InMemoryReader(MergeManager merger, InputAttemptIdentifier taskAttemptId,
-                        byte[] data, int start, int length)
-  throws IOException {
-    super(null, null, length - start, null, null);
-    this.merger = merger;
-    this.taskAttemptId = taskAttemptId;
-
-    buffer = data;
-    bufferSize = (int)fileLength;
-    memDataIn.reset(buffer, start, length);
-    this.start = start;
-    this.length = length;
-  }
-
-  @Override
-  public void reset(int offset) {
-    memDataIn.reset(buffer, start + offset, length);
-    bytesRead = offset;
-    eof = false;
-  }
-
-  @Override
-  public long getPosition() throws IOException {
-    // InMemoryReader does not initialize streams like Reader, so in.getPos()
-    // would not work. Instead, return the number of uncompressed bytes read,
-    // which will be correct since in-memory data is not compressed.
-    return bytesRead;
-  }
-  
-  @Override
-  public long getLength() { 
-    return fileLength;
-  }
-  
-  private void dumpOnError() {
-    File dumpFile = new File("../output/" + taskAttemptId + ".dump");
-    System.err.println("Dumping corrupt map-output of " + taskAttemptId + 
-                       " to " + dumpFile.getAbsolutePath());
-    try {
-      FileOutputStream fos = new FileOutputStream(dumpFile);
-      fos.write(buffer, 0, bufferSize);
-      fos.close();
-    } catch (IOException ioe) {
-      System.err.println("Failed to dump map-output of " + taskAttemptId);
-    }
-  }
-  
-  public KeyState readRawKey(DataInputBuffer key) throws IOException {
-    try {
-      if (!positionToNextRecord(memDataIn)) {
-        return KeyState.NO_KEY;
-      }
-      // Setup the key
-      int pos = memDataIn.getPosition();
-      byte[] data = memDataIn.getData();      
-      if(currentKeyLength == IFile.RLE_MARKER) {
-        key.reset(data, prevKeyPos, prevKeyLength);
-        currentKeyLength = prevKeyLength;
-        return KeyState.SAME_KEY;
-      }      
-      key.reset(data, pos, currentKeyLength);
-      prevKeyPos = pos;
-      // Position for the next value
-      long skipped = memDataIn.skip(currentKeyLength);
-      if (skipped != currentKeyLength) {
-        throw new IOException("Rec# " + recNo + 
-            ": Failed to skip past key of length: " + 
-            currentKeyLength);
-      }
-
-      // Record the byte
-      bytesRead += currentKeyLength;
-      return KeyState.NEW_KEY;
-    } catch (IOException ioe) {
-      dumpOnError();
-      throw ioe;
-    }
-  }
-  
-  public void nextRawValue(DataInputBuffer value) throws IOException {
-    try {
-      int pos = memDataIn.getPosition();
-      byte[] data = memDataIn.getData();
-      value.reset(data, pos, currentValueLength);
-
-      // Position for the next record
-      long skipped = memDataIn.skip(currentValueLength);
-      if (skipped != currentValueLength) {
-        throw new IOException("Rec# " + recNo + 
-            ": Failed to skip past value of length: " + 
-            currentValueLength);
-      }
-      // Record the byte
-      bytesRead += currentValueLength;
-
-      ++recNo;
-    } catch (IOException ioe) {
-      dumpOnError();
-      throw ioe;
-    }
-  }
-    
-  public void close() {
-    // Release
-    dataIn = null;
-    buffer = null;
-      // Inform the MergeManager
-    if (merger != null) {
-      merger.unreserve(bufferSize);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/InMemoryWriter.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/InMemoryWriter.java b/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/InMemoryWriter.java
deleted file mode 100644
index b3ebb8b..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/InMemoryWriter.java
+++ /dev/null
@@ -1,100 +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.engine.common.shuffle.impl;
-
-import java.io.DataOutputStream;
-import java.io.IOException;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.io.BoundedByteArrayOutputStream;
-import org.apache.hadoop.io.DataInputBuffer;
-import org.apache.hadoop.io.WritableUtils;
-import org.apache.tez.engine.common.sort.impl.IFile;
-import org.apache.tez.engine.common.sort.impl.IFileOutputStream;
-import org.apache.tez.engine.common.sort.impl.IFile.Writer;
-
-@InterfaceAudience.Private
-@InterfaceStability.Unstable
-public class InMemoryWriter extends Writer {
-  private static final Log LOG = LogFactory.getLog(InMemoryWriter.class);
-
-  private DataOutputStream out;
-
-  public InMemoryWriter(BoundedByteArrayOutputStream arrayStream) {
-    super(null);
-    this.out =
-      new DataOutputStream(new IFileOutputStream(arrayStream));
-  }
-
-  public void append(Object key, Object value) throws IOException {
-    throw new UnsupportedOperationException
-    ("InMemoryWriter.append(K key, V value");
-  }
-
-  public void append(DataInputBuffer key, DataInputBuffer value)
-  throws IOException {
-    int keyLength = key.getLength() - key.getPosition();
-    if (keyLength < 0) {
-      throw new IOException("Negative key-length not allowed: " + keyLength +
-                            " for " + key);
-    }
-
-    boolean sameKey = (key == IFile.REPEAT_KEY);
-
-    int valueLength = value.getLength() - value.getPosition();
-    if (valueLength < 0) {
-      throw new IOException("Negative value-length not allowed: " +
-                            valueLength + " for " + value);
-    }
-
-    if(sameKey) {
-      WritableUtils.writeVInt(out, IFile.RLE_MARKER);
-      WritableUtils.writeVInt(out, valueLength);
-      out.write(value.getData(), value.getPosition(), valueLength);
-    } else {
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("InMemWriter.append" +
-            " key.data=" + key.getData() +
-            " key.pos=" + key.getPosition() +
-            " key.len=" +key.getLength() +
-            " val.data=" + value.getData() +
-            " val.pos=" + value.getPosition() +
-            " val.len=" + value.getLength());
-      }
-      WritableUtils.writeVInt(out, keyLength);
-      WritableUtils.writeVInt(out, valueLength);
-      out.write(key.getData(), key.getPosition(), keyLength);
-      out.write(value.getData(), value.getPosition(), valueLength);
-    }
-
-  }
-
-  public void close() throws IOException {
-    // Write EOF_MARKER for key/value length
-    WritableUtils.writeVInt(out, IFile.EOF_MARKER);
-    WritableUtils.writeVInt(out, IFile.EOF_MARKER);
-
-    // Close the stream
-    out.close();
-    out = null;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/MapHost.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/MapHost.java b/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/MapHost.java
deleted file mode 100644
index 20ec472..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/MapHost.java
+++ /dev/null
@@ -1,124 +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.engine.common.shuffle.impl;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.hadoop.classification.InterfaceAudience.Private;
-import org.apache.tez.engine.common.InputAttemptIdentifier;
-
-@Private
-class MapHost {
-  
-  public static enum State {
-    IDLE,               // No map outputs available
-    BUSY,               // Map outputs are being fetched
-    PENDING,            // Known map outputs which need to be fetched
-    PENALIZED           // Host penalized due to shuffle failures
-  }
-  
-  private State state = State.IDLE;
-  private final String hostName;
-  private final int partitionId;
-  private final String baseUrl;
-  private final String identifier;
-  // Tracks attempt IDs
-  private List<InputAttemptIdentifier> maps = new ArrayList<InputAttemptIdentifier>();
-  
-  public MapHost(int partitionId, String hostName, String baseUrl) {
-    this.partitionId = partitionId;
-    this.hostName = hostName;
-    this.baseUrl = baseUrl;
-    this.identifier = createIdentifier(hostName, partitionId);
-  }
-  
-  public static String createIdentifier(String hostName, int partitionId) {
-    return hostName + ":" + Integer.toString(partitionId);
-  }
-  
-  public String getIdentifier() {
-    return identifier;
-  }
-  
-  public int getPartitionId() {
-    return partitionId;
-  }
-
-  public State getState() {
-    return state;
-  }
-
-  public String getHostName() {
-    return hostName;
-  }
-
-  public String getBaseUrl() {
-    return baseUrl;
-  }
-
-  public synchronized void addKnownMap(InputAttemptIdentifier srcAttempt) {
-    maps.add(srcAttempt);
-    if (state == State.IDLE) {
-      state = State.PENDING;
-    }
-  }
-
-  public synchronized List<InputAttemptIdentifier> getAndClearKnownMaps() {
-    List<InputAttemptIdentifier> currentKnownMaps = maps;
-    maps = new ArrayList<InputAttemptIdentifier>();
-    return currentKnownMaps;
-  }
-  
-  public synchronized void markBusy() {
-    state = State.BUSY;
-  }
-  
-  public synchronized void markPenalized() {
-    state = State.PENALIZED;
-  }
-  
-  public synchronized int getNumKnownMapOutputs() {
-    return maps.size();
-  }
-
-  /**
-   * Called when the node is done with its penalty or done copying.
-   * @return the host's new state
-   */
-  public synchronized State markAvailable() {
-    if (maps.isEmpty()) {
-      state = State.IDLE;
-    } else {
-      state = State.PENDING;
-    }
-    return state;
-  }
-  
-  @Override
-  public String toString() {
-    return hostName;
-  }
-  
-  /**
-   * Mark the host as penalized
-   */
-  public synchronized void penalize() {
-    state = State.PENALIZED;
-  }
-}


[09/50] [abbrv] TEZ-444. Rename *.new* packages back to what they should be, remove dead code from the old packages - mapreduce module (part of TEZ-398). (sseth)

Posted by ss...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/3d609458/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/SimpleInput.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/SimpleInput.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/SimpleInput.java
index 6817151..1e0b146 100644
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/SimpleInput.java
+++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/SimpleInput.java
@@ -32,23 +32,29 @@ import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.serializer.Deserializer;
 import org.apache.hadoop.io.serializer.SerializationFactory;
 import org.apache.hadoop.mapred.FileSplit;
+import org.apache.hadoop.mapred.InputFormat;
 import org.apache.hadoop.mapred.InputSplit;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.JobContext;
 import org.apache.hadoop.mapred.RecordReader;
-import org.apache.hadoop.mapreduce.InputFormat;
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.lib.input.FileInputFormatCounter;
 import org.apache.hadoop.mapreduce.split.SplitMetaInfoReaderTez;
 import org.apache.hadoop.mapreduce.split.JobSplit.TaskSplitIndex;
 import org.apache.hadoop.mapreduce.split.JobSplit.TaskSplitMetaInfo;
 import org.apache.hadoop.util.ReflectionUtils;
-import org.apache.tez.common.TezEngineTaskContext;
+import org.apache.tez.common.TezUtils;
 import org.apache.tez.common.counters.TaskCounter;
 import org.apache.tez.common.counters.TezCounter;
-import org.apache.tez.engine.api.Input;
-import org.apache.tez.engine.api.Master;
-import org.apache.tez.mapreduce.processor.MRTask;
-import org.apache.tez.mapreduce.processor.MRTaskReporter;
+import org.apache.tez.engine.newapi.Event;
+import org.apache.tez.engine.newapi.KVReader;
+import org.apache.tez.engine.newapi.LogicalInput;
+import org.apache.tez.engine.newapi.TezInputContext;
+import org.apache.tez.mapreduce.common.Utils;
+import org.apache.tez.mapreduce.hadoop.mapred.MRReporter;
+import org.apache.tez.mapreduce.hadoop.mapreduce.TaskAttemptContextImpl;
+
+import com.google.common.base.Preconditions;
 
 /**
  * {@link SimpleInput} is an {@link Input} which provides key/values pairs
@@ -57,161 +63,152 @@ import org.apache.tez.mapreduce.processor.MRTaskReporter;
  * It is compatible with all standard Apache Hadoop MapReduce 
  * {@link InputFormat} implementations.
  */
-@SuppressWarnings({ "unchecked", "rawtypes" })
-public class SimpleInput implements Input {
+
+public class SimpleInput implements LogicalInput {
 
   private static final Log LOG = LogFactory.getLog(SimpleInput.class);
   
-  MRTask task;
   
-  boolean useNewApi;
+  private TezInputContext inputContext;
   
-  JobConf jobConf;
+  private JobConf jobConf;
+  private Configuration incrementalConf;
+  private boolean recordReaderCreated = false;
+  
+  boolean useNewApi;
   
   org.apache.hadoop.mapreduce.TaskAttemptContext taskAttemptContext;
 
-  org.apache.hadoop.mapreduce.InputFormat newInputFormat;
-  org.apache.hadoop.mapreduce.RecordReader newRecordReader;
+  @SuppressWarnings("rawtypes")
+  private org.apache.hadoop.mapreduce.InputFormat newInputFormat;
+  @SuppressWarnings("rawtypes")
+  private org.apache.hadoop.mapreduce.RecordReader newRecordReader;
+  protected org.apache.hadoop.mapreduce.InputSplit newInputSplit;
   
-  org.apache.hadoop.mapred.InputFormat oldInputFormat;
-  org.apache.hadoop.mapred.RecordReader oldRecordReader;
+  @SuppressWarnings("rawtypes")
+  private InputFormat oldInputFormat;
+  @SuppressWarnings("rawtypes")
+  protected RecordReader oldRecordReader;
 
   protected TaskSplitIndex splitMetaInfo = new TaskSplitIndex();
   
-  Object key;
-  Object value;
-  
   private TezCounter inputRecordCounter;
   private TezCounter fileInputByteCounter; 
   private List<Statistics> fsStats;
-  private MRTaskReporter reporter;
 
-  public SimpleInput(TezEngineTaskContext task, int index)
-  {}
-  
-  public void setTask(MRTask task) {
-    this.task = task;
-  }
+  @Override
+  public List<Event> initialize(TezInputContext inputContext) throws IOException {
+    this.inputContext = inputContext;
+    Configuration conf = TezUtils.createConfFromUserPayload(inputContext.getUserPayload());
+    this.jobConf = new JobConf(conf);
 
-  public void initialize(Configuration conf, Master master) throws IOException,
-      InterruptedException {
-    if (task == null) {
-      return;
-    }
-    
-    if (conf instanceof JobConf) {
-      jobConf = (JobConf)conf;
-    } else {
-      jobConf = new JobConf(conf);
-    }
-    
     // Read split information.
-    TaskSplitMetaInfo[] allMetaInfo = readSplits(jobConf);
-    TaskSplitMetaInfo thisTaskMetaInfo = allMetaInfo[task.getTaskAttemptId()
-        .getTaskID().getId()];
-    splitMetaInfo = new TaskSplitIndex(thisTaskMetaInfo.getSplitLocation(),
+    TaskSplitMetaInfo[] allMetaInfo = readSplits(conf);
+    TaskSplitMetaInfo thisTaskMetaInfo = allMetaInfo[inputContext.getTaskIndex()];
+    this.splitMetaInfo = new TaskSplitIndex(thisTaskMetaInfo.getSplitLocation(),
         thisTaskMetaInfo.getStartOffset());
     
+    // TODO NEWTEZ Rename this to be specific to SimpleInput. This Input, in
+    // theory, can be used by the MapProcessor, ReduceProcessor or a custom
+    // processor. (The processor could provide the counter though)
+    this.inputRecordCounter = inputContext.getCounters().findCounter(TaskCounter.MAP_INPUT_RECORDS);
+    this.fileInputByteCounter = inputContext.getCounters().findCounter(FileInputFormatCounter.BYTES_READ);
     
-    useNewApi = jobConf.getUseNewMapper();
-    taskAttemptContext = task.getTaskAttemptContext();
-    
-    inputRecordCounter = task.getInputRecordsCounter();
-    fileInputByteCounter = task.getFileInputBytesCounter();
-
-    reporter = task.getMRReporter();
+    useNewApi = this.jobConf.getUseNewMapper();
 
     if (useNewApi) {
+      TaskAttemptContext taskAttemptContext = createTaskAttemptContext();
+      Class<? extends org.apache.hadoop.mapreduce.InputFormat<?, ?>> inputFormatClazz;
       try {
-        newInputFormat = 
-            ReflectionUtils.newInstance(
-                taskAttemptContext.getInputFormatClass(), jobConf);
-      } catch (ClassNotFoundException cnfe) {
-        throw new IOException(cnfe);
+        inputFormatClazz = taskAttemptContext.getInputFormatClass();
+      } catch (ClassNotFoundException e) {
+        throw new IOException("Unable to instantiate InputFormat class", e);
       }
-      
+
+      newInputFormat = ReflectionUtils.newInstance(inputFormatClazz, this.jobConf);
+
       newInputSplit = getNewSplitDetails(splitMetaInfo);
+
       List<Statistics> matchedStats = null;
       if (newInputSplit instanceof org.apache.hadoop.mapreduce.lib.input.FileSplit) {
-        matchedStats = MRTask.getFsStatistics(
+        matchedStats = Utils.getFsStatistics(
             ((org.apache.hadoop.mapreduce.lib.input.FileSplit)
-                newInputSplit).getPath(), jobConf);
+                newInputSplit).getPath(), this.jobConf);
       }
       fsStats = matchedStats;
-      newRecordReader = 
-          newInputFormat.createRecordReader(newInputSplit, taskAttemptContext);
-    } else {
-      oldInputFormat = jobConf.getInputFormat();
-      org.apache.hadoop.mapred.InputSplit oldInputSplit =
+      
+      try {
+        newRecordReader = newInputFormat.createRecordReader(newInputSplit, taskAttemptContext);
+        newRecordReader.initialize(newInputSplit, taskAttemptContext);
+      } catch (InterruptedException e) {
+        throw new IOException("Interrupted while creating record reader", e);
+      }
+    } else { // OLD API
+      oldInputFormat = this.jobConf.getInputFormat();
+      InputSplit oldInputSplit =
           getOldSplitDetails(splitMetaInfo);
       
+      
       List<Statistics> matchedStats = null;
       if (oldInputSplit instanceof FileSplit) {
-        matchedStats = 
-            MRTask.getFsStatistics(
-                ((FileSplit)oldInputSplit).getPath(), jobConf);
+        matchedStats = Utils.getFsStatistics(((FileSplit) oldInputSplit).getPath(), this.jobConf);
       }
       fsStats = matchedStats;
-
+      
       long bytesInPrev = getInputBytes();
-      oldRecordReader = 
-          jobConf.getInputFormat().getRecordReader(
-              oldInputSplit, jobConf, reporter);
+      oldRecordReader = oldInputFormat.getRecordReader(oldInputSplit,
+          this.jobConf, new MRReporter(inputContext, oldInputSplit));
       long bytesInCurr = getInputBytes();
       fileInputByteCounter.increment(bytesInCurr - bytesInPrev);
-
-      updateJobWithSplit(jobConf, oldInputSplit);
-    }
+      setIncrementalConfigParams(oldInputSplit);
+    }    
+    return null;
   }
 
-  public boolean hasNext() throws IOException, InterruptedException {
-    boolean hasNext = false;
-    long bytesInPrev = getInputBytes();
+  @Override
+  public KVReader getReader() throws IOException {
+    Preconditions
+        .checkState(recordReaderCreated == false,
+            "Only a single instance of record reader can be created for this input.");
+    recordReaderCreated = true;
+    return new MRInputKVReader();
+  }
 
-    if (useNewApi) { 
-        hasNext = newRecordReader.nextKeyValue();
-    } else {
-      hasNext = oldRecordReader.next(key, value);
-    }
-    
-    long bytesInCurr = getInputBytes();
-    fileInputByteCounter.increment(bytesInCurr - bytesInPrev);
-    reporter.setProgress(getProgress());
 
-    if (hasNext) {
-      inputRecordCounter.increment(1);
-    }
-    
-    return hasNext;
+  @Override
+  public void handleEvents(List<Event> inputEvents) {
+    // Not expecting any events at the moment.
   }
 
-  private SimpleValueIterator vIter = new SimpleValueIterator();
-  private SimpleIterable valuesIterable =
-      new SimpleIterable(vIter);
-
-  private org.apache.hadoop.mapreduce.InputSplit newInputSplit;
 
-  public void setKey(Object key) {
-    this.key = key;
-  }
-  
-  public void setValue(Object value) {
-    this.value = value;
+  @Override
+  public void setNumPhysicalInputs(int numInputs) {
+    // Not required at the moment. May be required if splits are sent via events.
   }
 
-  public Object getNextKey() throws IOException, InterruptedException {
+  @Override
+  public List<Event> close() throws IOException {
+    long bytesInPrev = getInputBytes();
     if (useNewApi) {
-      return newRecordReader.getCurrentKey();
+      newRecordReader.close();
     } else {
-      return key;
+      oldRecordReader.close();
     }
+    long bytesInCurr = getInputBytes();
+    fileInputByteCounter.increment(bytesInCurr - bytesInPrev);
+    
+    return null;
   }
 
-  public Iterable getNextValues() throws IOException,
-      InterruptedException {
-    value = newRecordReader.getCurrentValue();
-    vIter.setValue(value);
-    return valuesIterable;
+  /**
+   * SimpleInputs sets some additional parameters like split location when using
+   * the new API. This methods returns the list of additional updates, and
+   * should be used by Processors using the old MapReduce API with SimpleInput.
+   * 
+   * @return the additional fields set by SimpleInput
+   */
+  public Configuration getConfigUpdates() {
+    return new Configuration(incrementalConf);
   }
 
   public float getProgress() throws IOException, InterruptedException {
@@ -222,27 +219,22 @@ public class SimpleInput implements Input {
     }
   }
 
-  public void close() throws IOException {
-    long bytesInPrev = getInputBytes();
-    if (useNewApi) {
-      newRecordReader.close();
-    } else {
-      oldRecordReader.close();
-    }
-    long bytesInCurr = getInputBytes();
-    fileInputByteCounter.increment(bytesInCurr - bytesInPrev);
+  
+  private TaskAttemptContext createTaskAttemptContext() {
+    return new TaskAttemptContextImpl(this.jobConf, inputContext, true);
   }
+  
 
-  static class SimpleValueIterator implements Iterator {
+  private static class SimpleValueIterator implements Iterator<Object> {
 
     private Object value;
-    
+
     public void setValue(Object value) {
       this.value = value;
     }
-    
+
     public boolean hasNext() {
-      return false;
+      return value != null;
     }
 
     public Object next() {
@@ -256,28 +248,23 @@ public class SimpleInput implements Input {
     }
   }
 
-  static class SimpleIterable implements Iterable {
-    private final Iterator iterator;
-    public SimpleIterable(Iterator iterator) {
+  private static class SimpleIterable implements Iterable<Object> {
+    private final Iterator<Object> iterator;
+    public SimpleIterable(Iterator<Object> iterator) {
       this.iterator = iterator;
     }
-    
-    public Iterator iterator() {
+
+    @Override
+    public Iterator<Object> iterator() {
       return iterator;
     }
   }
-  
 
-  public RecordReader getOldRecordReader() {
-    return oldRecordReader;
-  }
-  
-  public org.apache.hadoop.mapreduce.RecordReader getNewRecordReader() {
-    return newRecordReader;
-  }
+
+
   
-  public org.apache.hadoop.mapred.InputSplit 
-  getOldSplitDetails(TaskSplitIndex splitMetaInfo) 
+  @SuppressWarnings("unchecked")
+  private InputSplit getOldSplitDetails(TaskSplitIndex splitMetaInfo)
       throws IOException {
     Path file = new Path(splitMetaInfo.getSplitLocation());
     FileSystem fs = FileSystem.getLocal(jobConf);
@@ -306,14 +293,15 @@ public class SimpleInput implements Input {
     deserializer.open(inFile);
     org.apache.hadoop.mapred.InputSplit split = deserializer.deserialize(null);
     long pos = inFile.getPos();
-    reporter.getCounter(TaskCounter.SPLIT_RAW_BYTES).increment(pos - offset);
+    inputContext.getCounters().findCounter(TaskCounter.SPLIT_RAW_BYTES)
+        .increment(pos - offset);
     inFile.close();
     return split;
   }
 
-  public org.apache.hadoop.mapreduce.InputSplit 
-  getNewSplitDetails(TaskSplitIndex splitMetaInfo) 
-      throws IOException {
+  @SuppressWarnings("unchecked")
+  private org.apache.hadoop.mapreduce.InputSplit getNewSplitDetails(
+      TaskSplitIndex splitMetaInfo) throws IOException {
     Path file = new Path(splitMetaInfo.getSplitLocation());
     long offset = splitMetaInfo.getStartOffset();
     
@@ -343,17 +331,23 @@ public class SimpleInput implements Input {
     org.apache.hadoop.mapreduce.InputSplit split = 
         deserializer.deserialize(null);
     long pos = inFile.getPos();
-    reporter.getCounter(TaskCounter.SPLIT_RAW_BYTES).increment(pos - offset);
+    inputContext.getCounters().findCounter(TaskCounter.SPLIT_RAW_BYTES)
+        .increment(pos - offset);
     inFile.close();
     return split;
   }
 
-  private void updateJobWithSplit(final JobConf job, InputSplit inputSplit) {
+  private void setIncrementalConfigParams(InputSplit inputSplit) {
     if (inputSplit instanceof FileSplit) {
       FileSplit fileSplit = (FileSplit) inputSplit;
-      job.set(JobContext.MAP_INPUT_FILE, fileSplit.getPath().toString());
-      job.setLong(JobContext.MAP_INPUT_START, fileSplit.getStart());
-      job.setLong(JobContext.MAP_INPUT_PATH, fileSplit.getLength());
+      this.incrementalConf = new Configuration(false);
+
+      this.incrementalConf.set(JobContext.MAP_INPUT_FILE, fileSplit.getPath()
+          .toString());
+      this.incrementalConf.setLong(JobContext.MAP_INPUT_START,
+          fileSplit.getStart());
+      this.incrementalConf.setLong(JobContext.MAP_INPUT_PATH,
+          fileSplit.getLength());
     }
     LOG.info("Processing split: " + inputSplit);
   }
@@ -367,16 +361,6 @@ public class SimpleInput implements Input {
     return bytesRead;
   }
 
-  public void initializeNewRecordReader(
-      org.apache.hadoop.mapreduce.InputSplit split, TaskAttemptContext context) 
-  throws IOException, InterruptedException {
-    newRecordReader.initialize(split, context);
-  }
-  
-  public org.apache.hadoop.mapreduce.InputSplit getNewInputSplit() {
-    return newInputSplit;
-  }
-
   protected TaskSplitMetaInfo[] readSplits(Configuration conf)
       throws IOException {
     TaskSplitMetaInfo[] allTaskSplitMetaInfo;
@@ -384,4 +368,71 @@ public class SimpleInput implements Input {
         FileSystem.getLocal(conf));
     return allTaskSplitMetaInfo;
   }
+  
+  private class MRInputKVReader implements KVReader {
+    
+    Object key;
+    Object value;
+
+    private SimpleValueIterator valueIterator = new SimpleValueIterator();
+    private SimpleIterable valueIterable = new SimpleIterable(valueIterator);
+
+    private final boolean localNewApi;
+    
+    MRInputKVReader() {
+      localNewApi = useNewApi;
+      if (!localNewApi) {
+        key = oldRecordReader.createKey();
+        value =oldRecordReader.createValue();
+      }
+    }
+    
+    // Setup the values iterator once, and set value on the same object each time
+    // to prevent lots of objects being created.
+
+    
+    @SuppressWarnings("unchecked")
+    @Override
+    public boolean next() throws IOException {
+      boolean hasNext = false;
+      long bytesInPrev = getInputBytes();
+      if (localNewApi) {
+        try {
+          hasNext = newRecordReader.nextKeyValue();
+        } catch (InterruptedException e) {
+          Thread.currentThread().interrupt();
+          throw new IOException("Interrupted while checking for next key-value", e);
+        }
+      } else {
+        hasNext = oldRecordReader.next(key, value);
+      }
+      long bytesInCurr = getInputBytes();
+      fileInputByteCounter.increment(bytesInCurr - bytesInPrev);
+      
+      if (hasNext) {
+        inputRecordCounter.increment(1);
+      }
+      
+      return hasNext;
+    }
+
+    @Override
+    public KVRecord getCurrentKV() throws IOException {
+      KVRecord kvRecord = null;
+      if (localNewApi) {
+        try {
+          valueIterator.setValue(newRecordReader.getCurrentValue());
+          kvRecord = new KVRecord(newRecordReader.getCurrentKey(), valueIterable);
+        } catch (InterruptedException e) {
+          Thread.currentThread().interrupt();
+          throw new IOException("Interrupted while fetching next key-value", e);
+        }
+        
+      } else {
+        valueIterator.setValue(value);
+        kvRecord = new KVRecord(key, valueIterable);
+      }
+      return kvRecord;
+    }
+  };
 }

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/3d609458/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/SimpleInputLegacy.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/SimpleInputLegacy.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/SimpleInputLegacy.java
new file mode 100644
index 0000000..4e61aa7
--- /dev/null
+++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/SimpleInputLegacy.java
@@ -0,0 +1,36 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.mapreduce.input;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.mapred.RecordReader;
+
+public class SimpleInputLegacy extends SimpleInput {
+
+  @Private
+  public org.apache.hadoop.mapreduce.InputSplit getNewInputSplit() {
+    return this.newInputSplit;
+  }  
+  
+  @SuppressWarnings("rawtypes")
+  @Private
+  public RecordReader getOldRecordReader() {
+    return this.oldRecordReader;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/3d609458/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/newcombine/MRCombiner.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/newcombine/MRCombiner.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/newcombine/MRCombiner.java
deleted file mode 100644
index 5566fd8..0000000
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/newcombine/MRCombiner.java
+++ /dev/null
@@ -1,242 +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.newcombine;
-
-import java.io.IOException;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.io.DataInputBuffer;
-import org.apache.hadoop.io.RawComparator;
-import org.apache.hadoop.mapred.OutputCollector;
-import org.apache.hadoop.mapred.RawKeyValueIterator;
-import org.apache.hadoop.mapred.Reducer;
-import org.apache.hadoop.mapreduce.Counter;
-import org.apache.hadoop.mapreduce.RecordWriter;
-import org.apache.hadoop.mapreduce.ReduceContext;
-import org.apache.hadoop.mapreduce.TaskAttemptContext;
-import org.apache.hadoop.mapreduce.TaskAttemptID;
-import org.apache.hadoop.mapreduce.TaskCounter;
-import org.apache.hadoop.mapreduce.TaskID;
-import org.apache.hadoop.mapreduce.TaskType;
-import org.apache.hadoop.mapreduce.lib.reduce.WrappedReducer;
-import org.apache.hadoop.mapreduce.task.ReduceContextImpl;
-import org.apache.hadoop.util.Progress;
-import org.apache.hadoop.util.ReflectionUtils;
-import org.apache.tez.common.TezUtils;
-import org.apache.tez.common.counters.TezCounter;
-import org.apache.tez.engine.common.ConfigUtils;
-import org.apache.tez.engine.common.ValuesIterator;
-import org.apache.tez.engine.common.combine.Combiner;
-import org.apache.tez.engine.common.sort.impl.IFile.Writer;
-import org.apache.tez.engine.common.sort.impl.TezRawKeyValueIterator;
-import org.apache.tez.engine.newapi.TezInputContext;
-import org.apache.tez.engine.newapi.TezOutputContext;
-import org.apache.tez.engine.newapi.TezTaskContext;
-import org.apache.tez.mapreduce.hadoop.MRConfig;
-import org.apache.tez.mapreduce.hadoop.MRJobConfig;
-import org.apache.tez.mapreduce.hadoop.mapred.MRCounters;
-import org.apache.tez.mapreduce.newprocessor.MRTaskReporter;
-
-@SuppressWarnings({"rawtypes", "unchecked"})
-public class MRCombiner implements Combiner {
-
-  private static Log LOG = LogFactory.getLog(MRCombiner.class);
-  
-  private final Configuration conf;
-  private final Class<?> keyClass;
-  private final Class<?> valClass;
-  private final RawComparator<?> comparator;
-  private final boolean useNewApi;
-  
-  private final TezCounter combineInputKeyCounter;
-  private final TezCounter combineInputValueCounter;
-  
-  private final MRTaskReporter reporter;
-  private final TaskAttemptID mrTaskAttemptID;
-
-  public MRCombiner(TezTaskContext taskContext) throws IOException {
-    this.conf = TezUtils.createConfFromUserPayload(taskContext.getUserPayload());
-
-    assert(taskContext instanceof TezInputContext || taskContext instanceof TezOutputContext);
-    if (taskContext instanceof TezOutputContext) {
-      this.keyClass = ConfigUtils.getIntermediateOutputKeyClass(conf);
-      this.valClass = ConfigUtils.getIntermediateOutputValueClass(conf);
-      this.comparator = ConfigUtils.getIntermediateOutputKeyComparator(conf);
-      this.reporter = new MRTaskReporter((TezOutputContext)taskContext);
-    } else {
-      this.keyClass = ConfigUtils.getIntermediateInputKeyClass(conf);
-      this.valClass = ConfigUtils.getIntermediateInputValueClass(conf);
-      this.comparator = ConfigUtils.getIntermediateInputKeyComparator(conf);
-      this.reporter = new MRTaskReporter((TezInputContext)taskContext);
-    }
-
-    this.useNewApi = ConfigUtils.useNewApi(conf);
-    
-    combineInputKeyCounter = taskContext.getCounters().findCounter(TaskCounter.COMBINE_INPUT_RECORDS);
-    combineInputValueCounter = taskContext.getCounters().findCounter(TaskCounter.COMBINE_OUTPUT_RECORDS);
-    
-    boolean isMap = conf.getBoolean(MRConfig.IS_MAP_PROCESSOR,false);
-    this.mrTaskAttemptID = new TaskAttemptID(
-        new TaskID(String.valueOf(taskContext.getApplicationId()
-            .getClusterTimestamp()), taskContext.getApplicationId().getId(),
-            isMap ? TaskType.MAP : TaskType.REDUCE,
-            taskContext.getTaskIndex()), taskContext.getTaskAttemptNumber());
-    
-    LOG.info("Using combineKeyClass: " + keyClass + ", combineValueClass: " + valClass + ", combineComparator: " +comparator + ", useNewApi: " + useNewApi);
-  }
-
-  @Override
-  public void combine(TezRawKeyValueIterator rawIter, Writer writer)
-      throws InterruptedException, IOException {
-    if (useNewApi) {
-      runNewCombiner(rawIter, writer);
-    } else {
-      runOldCombiner(rawIter, writer);
-    }
-    
-  }
-
-  ///////////////// Methods for old API //////////////////////
-  
-  private void runOldCombiner(final TezRawKeyValueIterator rawIter, final Writer writer) throws IOException {
-    Class<? extends Reducer> reducerClazz = (Class<? extends Reducer>) conf.getClass("mapred.combiner.class", null, Reducer.class);
-    
-    Reducer combiner = ReflectionUtils.newInstance(reducerClazz, conf);
-    
-    OutputCollector collector = new OutputCollector() {
-      @Override
-      public void collect(Object key, Object value) throws IOException {
-        writer.append(key, value);
-      }
-    };
-    
-    CombinerValuesIterator values = new CombinerValuesIterator(rawIter, keyClass, valClass, comparator);
-    
-    while (values.moveToNext()) {
-      combiner.reduce(values.getKey(), values.getValues().iterator(), collector, reporter);
-    }
-  }
-  
-  private final class CombinerValuesIterator<KEY,VALUE> extends ValuesIterator<KEY, VALUE> {
-    public CombinerValuesIterator(TezRawKeyValueIterator rawIter,
-        Class<KEY> keyClass, Class<VALUE> valClass,
-        RawComparator<KEY> comparator) throws IOException {
-      super(rawIter, comparator, keyClass, valClass, conf,
-          combineInputKeyCounter, combineInputValueCounter);
-    }
-  }
-  
-  ///////////////// End of methods for old API //////////////////////
-  
-  ///////////////// Methods for new API //////////////////////
-  
-  private void runNewCombiner(final TezRawKeyValueIterator rawIter, final Writer writer) throws InterruptedException, IOException {
-    
-    RecordWriter recordWriter = new RecordWriter() {
-
-      @Override
-      public void write(Object key, Object value) throws IOException,
-          InterruptedException {
-        writer.append(key, value);
-      }
-
-      @Override
-      public void close(TaskAttemptContext context) throws IOException,
-          InterruptedException {
-        // Will be closed by whoever invokes the combiner.
-      }
-    };
-    
-    Class<? extends org.apache.hadoop.mapreduce.Reducer> reducerClazz = (Class<? extends org.apache.hadoop.mapreduce.Reducer>) conf
-        .getClass(MRJobConfig.COMBINE_CLASS_ATTR, null,
-            org.apache.hadoop.mapreduce.Reducer.class);
-    org.apache.hadoop.mapreduce.Reducer reducer = ReflectionUtils.newInstance(reducerClazz, conf);
-    
-    org.apache.hadoop.mapreduce.Reducer.Context reducerContext =
-        createReduceContext(
-            conf,
-            mrTaskAttemptID,
-            rawIter,
-            new MRCounters.MRCounter(combineInputKeyCounter),
-            new MRCounters.MRCounter(combineInputValueCounter),
-            recordWriter,
-            reporter,
-            (RawComparator)comparator,
-            keyClass,
-            valClass);
-    
-    reducer.run(reducerContext);
-    recordWriter.close(reducerContext);
-  }
-
-  private static <KEYIN, VALUEIN, KEYOUT, VALUEOUT> org.apache.hadoop.mapreduce.Reducer<KEYIN, VALUEIN, KEYOUT, VALUEOUT>.Context createReduceContext(
-      Configuration conf,
-      TaskAttemptID mrTaskAttemptID,
-      final TezRawKeyValueIterator rawIter,
-      Counter combineInputKeyCounter,
-      Counter combineInputValueCounter,
-      RecordWriter<KEYOUT, VALUEOUT> recordWriter,
-      MRTaskReporter reporter,
-      RawComparator<KEYIN> comparator,
-      Class<KEYIN> keyClass,
-      Class<VALUEIN> valClass) throws InterruptedException, IOException {
-
-    RawKeyValueIterator r = new RawKeyValueIterator() {
-
-      @Override
-      public boolean next() throws IOException {
-        return rawIter.next();
-      }
-
-      @Override
-      public DataInputBuffer getValue() throws IOException {
-        return rawIter.getValue();
-      }
-
-      @Override
-      public Progress getProgress() {
-        return rawIter.getProgress();
-      }
-
-      @Override
-      public DataInputBuffer getKey() throws IOException {
-        return rawIter.getKey();
-      }
-
-      @Override
-      public void close() throws IOException {
-        rawIter.close();
-      }
-    };
-
-    ReduceContext<KEYIN, VALUEIN, KEYOUT, VALUEOUT> rContext = new ReduceContextImpl<KEYIN, VALUEIN, KEYOUT, VALUEOUT>(
-        conf, mrTaskAttemptID, r, combineInputKeyCounter,
-        combineInputValueCounter, recordWriter, null, reporter, comparator,
-        keyClass, valClass);
-
-    org.apache.hadoop.mapreduce.Reducer<KEYIN, VALUEIN, KEYOUT, VALUEOUT>.Context reducerContext = new WrappedReducer<KEYIN, VALUEIN, KEYOUT, VALUEOUT>()
-        .getReducerContext(rContext);
-    return reducerContext;
-  }
-
-  
- 
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/3d609458/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/newinput/ShuffledMergedInputLegacy.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/newinput/ShuffledMergedInputLegacy.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/newinput/ShuffledMergedInputLegacy.java
deleted file mode 100644
index a83620e..0000000
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/newinput/ShuffledMergedInputLegacy.java
+++ /dev/null
@@ -1,29 +0,0 @@
-/**
- * <code>ShuffleMergedInput</code> in a {@link LogicalInput} which shuffles
- * intermediate sorted data, merges them and provides key/<values> to the
- * consumer.
- * 
- * The Copy and Merge will be triggered by the initialization - which is handled
- * by the Tez framework. Input is not consumable until the Copy and Merge are
- * complete. Methods are provided to check for this, as well as to wait for
- * completion. Attempting to get a reader on a non-complete input will block.
- * 
- */
-
-package org.apache.tez.mapreduce.newinput;
-
-import java.io.IOException;
-
-import org.apache.hadoop.classification.InterfaceAudience.Private;
-import org.apache.tez.engine.common.sort.impl.TezRawKeyValueIterator;
-import org.apache.tez.engine.lib.input.ShuffledMergedInput;
-
-public class ShuffledMergedInputLegacy extends ShuffledMergedInput {
-
-  @Private
-  public TezRawKeyValueIterator getIterator() throws IOException, InterruptedException {
-    // wait for input so that iterator is available
-    waitForInputReady();
-    return rawIter;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/3d609458/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/newinput/SimpleInput.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/newinput/SimpleInput.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/newinput/SimpleInput.java
deleted file mode 100644
index 73d8cc7..0000000
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/newinput/SimpleInput.java
+++ /dev/null
@@ -1,438 +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.newinput;
-
-import java.io.IOException;
-import java.util.Iterator;
-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.fs.FSDataInputStream;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.FileSystem.Statistics;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.serializer.Deserializer;
-import org.apache.hadoop.io.serializer.SerializationFactory;
-import org.apache.hadoop.mapred.FileSplit;
-import org.apache.hadoop.mapred.InputFormat;
-import org.apache.hadoop.mapred.InputSplit;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.JobContext;
-import org.apache.hadoop.mapred.RecordReader;
-import org.apache.hadoop.mapreduce.TaskAttemptContext;
-import org.apache.hadoop.mapreduce.lib.input.FileInputFormatCounter;
-import org.apache.hadoop.mapreduce.split.SplitMetaInfoReaderTez;
-import org.apache.hadoop.mapreduce.split.JobSplit.TaskSplitIndex;
-import org.apache.hadoop.mapreduce.split.JobSplit.TaskSplitMetaInfo;
-import org.apache.hadoop.util.ReflectionUtils;
-import org.apache.tez.common.TezUtils;
-import org.apache.tez.common.counters.TaskCounter;
-import org.apache.tez.common.counters.TezCounter;
-import org.apache.tez.engine.newapi.Event;
-import org.apache.tez.engine.newapi.KVReader;
-import org.apache.tez.engine.newapi.LogicalInput;
-import org.apache.tez.engine.newapi.TezInputContext;
-import org.apache.tez.mapreduce.common.Utils;
-import org.apache.tez.mapreduce.hadoop.newmapred.MRReporter;
-import org.apache.tez.mapreduce.hadoop.newmapreduce.TaskAttemptContextImpl;
-
-import com.google.common.base.Preconditions;
-
-/**
- * {@link SimpleInput} is an {@link Input} which provides key/values pairs
- * for the consumer.
- *
- * It is compatible with all standard Apache Hadoop MapReduce 
- * {@link InputFormat} implementations.
- */
-
-public class SimpleInput implements LogicalInput {
-
-  private static final Log LOG = LogFactory.getLog(SimpleInput.class);
-  
-  
-  private TezInputContext inputContext;
-  
-  private JobConf jobConf;
-  private Configuration incrementalConf;
-  private boolean recordReaderCreated = false;
-  
-  boolean useNewApi;
-  
-  org.apache.hadoop.mapreduce.TaskAttemptContext taskAttemptContext;
-
-  @SuppressWarnings("rawtypes")
-  private org.apache.hadoop.mapreduce.InputFormat newInputFormat;
-  @SuppressWarnings("rawtypes")
-  private org.apache.hadoop.mapreduce.RecordReader newRecordReader;
-  protected org.apache.hadoop.mapreduce.InputSplit newInputSplit;
-  
-  @SuppressWarnings("rawtypes")
-  private InputFormat oldInputFormat;
-  @SuppressWarnings("rawtypes")
-  protected RecordReader oldRecordReader;
-
-  protected TaskSplitIndex splitMetaInfo = new TaskSplitIndex();
-  
-  private TezCounter inputRecordCounter;
-  private TezCounter fileInputByteCounter; 
-  private List<Statistics> fsStats;
-
-  @Override
-  public List<Event> initialize(TezInputContext inputContext) throws IOException {
-    this.inputContext = inputContext;
-    Configuration conf = TezUtils.createConfFromUserPayload(inputContext.getUserPayload());
-    this.jobConf = new JobConf(conf);
-
-    // Read split information.
-    TaskSplitMetaInfo[] allMetaInfo = readSplits(conf);
-    TaskSplitMetaInfo thisTaskMetaInfo = allMetaInfo[inputContext.getTaskIndex()];
-    this.splitMetaInfo = new TaskSplitIndex(thisTaskMetaInfo.getSplitLocation(),
-        thisTaskMetaInfo.getStartOffset());
-    
-    // TODO NEWTEZ Rename this to be specific to SimpleInput. This Input, in
-    // theory, can be used by the MapProcessor, ReduceProcessor or a custom
-    // processor. (The processor could provide the counter though)
-    this.inputRecordCounter = inputContext.getCounters().findCounter(TaskCounter.MAP_INPUT_RECORDS);
-    this.fileInputByteCounter = inputContext.getCounters().findCounter(FileInputFormatCounter.BYTES_READ);
-    
-    useNewApi = this.jobConf.getUseNewMapper();
-
-    if (useNewApi) {
-      TaskAttemptContext taskAttemptContext = createTaskAttemptContext();
-      Class<? extends org.apache.hadoop.mapreduce.InputFormat<?, ?>> inputFormatClazz;
-      try {
-        inputFormatClazz = taskAttemptContext.getInputFormatClass();
-      } catch (ClassNotFoundException e) {
-        throw new IOException("Unable to instantiate InputFormat class", e);
-      }
-
-      newInputFormat = ReflectionUtils.newInstance(inputFormatClazz, this.jobConf);
-
-      newInputSplit = getNewSplitDetails(splitMetaInfo);
-
-      List<Statistics> matchedStats = null;
-      if (newInputSplit instanceof org.apache.hadoop.mapreduce.lib.input.FileSplit) {
-        matchedStats = Utils.getFsStatistics(
-            ((org.apache.hadoop.mapreduce.lib.input.FileSplit)
-                newInputSplit).getPath(), this.jobConf);
-      }
-      fsStats = matchedStats;
-      
-      try {
-        newRecordReader = newInputFormat.createRecordReader(newInputSplit, taskAttemptContext);
-        newRecordReader.initialize(newInputSplit, taskAttemptContext);
-      } catch (InterruptedException e) {
-        throw new IOException("Interrupted while creating record reader", e);
-      }
-    } else { // OLD API
-      oldInputFormat = this.jobConf.getInputFormat();
-      InputSplit oldInputSplit =
-          getOldSplitDetails(splitMetaInfo);
-      
-      
-      List<Statistics> matchedStats = null;
-      if (oldInputSplit instanceof FileSplit) {
-        matchedStats = Utils.getFsStatistics(((FileSplit) oldInputSplit).getPath(), this.jobConf);
-      }
-      fsStats = matchedStats;
-      
-      long bytesInPrev = getInputBytes();
-      oldRecordReader = oldInputFormat.getRecordReader(oldInputSplit,
-          this.jobConf, new MRReporter(inputContext, oldInputSplit));
-      long bytesInCurr = getInputBytes();
-      fileInputByteCounter.increment(bytesInCurr - bytesInPrev);
-      setIncrementalConfigParams(oldInputSplit);
-    }    
-    return null;
-  }
-
-  @Override
-  public KVReader getReader() throws IOException {
-    Preconditions
-        .checkState(recordReaderCreated == false,
-            "Only a single instance of record reader can be created for this input.");
-    recordReaderCreated = true;
-    return new MRInputKVReader();
-  }
-
-
-  @Override
-  public void handleEvents(List<Event> inputEvents) {
-    // Not expecting any events at the moment.
-  }
-
-
-  @Override
-  public void setNumPhysicalInputs(int numInputs) {
-    // Not required at the moment. May be required if splits are sent via events.
-  }
-
-  @Override
-  public List<Event> close() throws IOException {
-    long bytesInPrev = getInputBytes();
-    if (useNewApi) {
-      newRecordReader.close();
-    } else {
-      oldRecordReader.close();
-    }
-    long bytesInCurr = getInputBytes();
-    fileInputByteCounter.increment(bytesInCurr - bytesInPrev);
-    
-    return null;
-  }
-
-  /**
-   * SimpleInputs sets some additional parameters like split location when using
-   * the new API. This methods returns the list of additional updates, and
-   * should be used by Processors using the old MapReduce API with SimpleInput.
-   * 
-   * @return the additional fields set by SimpleInput
-   */
-  public Configuration getConfigUpdates() {
-    return new Configuration(incrementalConf);
-  }
-
-  public float getProgress() throws IOException, InterruptedException {
-    if (useNewApi) {
-      return newRecordReader.getProgress();
-    } else {
-      return oldRecordReader.getProgress();
-    }
-  }
-
-  
-  private TaskAttemptContext createTaskAttemptContext() {
-    return new TaskAttemptContextImpl(this.jobConf, inputContext, true);
-  }
-  
-
-  private static class SimpleValueIterator implements Iterator<Object> {
-
-    private Object value;
-
-    public void setValue(Object value) {
-      this.value = value;
-    }
-
-    public boolean hasNext() {
-      return value != null;
-    }
-
-    public Object next() {
-      Object value = this.value;
-      this.value = null;
-      return value;
-    }
-
-    public void remove() {
-      throw new UnsupportedOperationException();
-    }
-  }
-
-  private static class SimpleIterable implements Iterable<Object> {
-    private final Iterator<Object> iterator;
-    public SimpleIterable(Iterator<Object> iterator) {
-      this.iterator = iterator;
-    }
-
-    @Override
-    public Iterator<Object> iterator() {
-      return iterator;
-    }
-  }
-
-
-
-  
-  @SuppressWarnings("unchecked")
-  private InputSplit getOldSplitDetails(TaskSplitIndex splitMetaInfo)
-      throws IOException {
-    Path file = new Path(splitMetaInfo.getSplitLocation());
-    FileSystem fs = FileSystem.getLocal(jobConf);
-    file = fs.makeQualified(file);
-    LOG.info("Reading input split file from : " + file);
-    long offset = splitMetaInfo.getStartOffset();
-    
-    FSDataInputStream inFile = fs.open(file);
-    inFile.seek(offset);
-    String className = Text.readString(inFile);
-    Class<org.apache.hadoop.mapred.InputSplit> cls;
-    try {
-      cls = 
-          (Class<org.apache.hadoop.mapred.InputSplit>) 
-          jobConf.getClassByName(className);
-    } catch (ClassNotFoundException ce) {
-      IOException wrap = new IOException("Split class " + className + 
-          " not found");
-      wrap.initCause(ce);
-      throw wrap;
-    }
-    SerializationFactory factory = new SerializationFactory(jobConf);
-    Deserializer<org.apache.hadoop.mapred.InputSplit> deserializer = 
-        (Deserializer<org.apache.hadoop.mapred.InputSplit>) 
-        factory.getDeserializer(cls);
-    deserializer.open(inFile);
-    org.apache.hadoop.mapred.InputSplit split = deserializer.deserialize(null);
-    long pos = inFile.getPos();
-    inputContext.getCounters().findCounter(TaskCounter.SPLIT_RAW_BYTES)
-        .increment(pos - offset);
-    inFile.close();
-    return split;
-  }
-
-  @SuppressWarnings("unchecked")
-  private org.apache.hadoop.mapreduce.InputSplit getNewSplitDetails(
-      TaskSplitIndex splitMetaInfo) throws IOException {
-    Path file = new Path(splitMetaInfo.getSplitLocation());
-    long offset = splitMetaInfo.getStartOffset();
-    
-    // Split information read from local filesystem.
-    FileSystem fs = FileSystem.getLocal(jobConf);
-    file = fs.makeQualified(file);
-    LOG.info("Reading input split file from : " + file);
-    FSDataInputStream inFile = fs.open(file);
-    inFile.seek(offset);
-    String className = Text.readString(inFile);
-    Class<org.apache.hadoop.mapreduce.InputSplit> cls;
-    try {
-      cls = 
-          (Class<org.apache.hadoop.mapreduce.InputSplit>) 
-          jobConf.getClassByName(className);
-    } catch (ClassNotFoundException ce) {
-      IOException wrap = new IOException("Split class " + className + 
-          " not found");
-      wrap.initCause(ce);
-      throw wrap;
-    }
-    SerializationFactory factory = new SerializationFactory(jobConf);
-    Deserializer<org.apache.hadoop.mapreduce.InputSplit> deserializer = 
-        (Deserializer<org.apache.hadoop.mapreduce.InputSplit>) 
-        factory.getDeserializer(cls);
-    deserializer.open(inFile);
-    org.apache.hadoop.mapreduce.InputSplit split = 
-        deserializer.deserialize(null);
-    long pos = inFile.getPos();
-    inputContext.getCounters().findCounter(TaskCounter.SPLIT_RAW_BYTES)
-        .increment(pos - offset);
-    inFile.close();
-    return split;
-  }
-
-  private void setIncrementalConfigParams(InputSplit inputSplit) {
-    if (inputSplit instanceof FileSplit) {
-      FileSplit fileSplit = (FileSplit) inputSplit;
-      this.incrementalConf = new Configuration(false);
-
-      this.incrementalConf.set(JobContext.MAP_INPUT_FILE, fileSplit.getPath()
-          .toString());
-      this.incrementalConf.setLong(JobContext.MAP_INPUT_START,
-          fileSplit.getStart());
-      this.incrementalConf.setLong(JobContext.MAP_INPUT_PATH,
-          fileSplit.getLength());
-    }
-    LOG.info("Processing split: " + inputSplit);
-  }
-
-  private long getInputBytes() {
-    if (fsStats == null) return 0;
-    long bytesRead = 0;
-    for (Statistics stat: fsStats) {
-      bytesRead = bytesRead + stat.getBytesRead();
-    }
-    return bytesRead;
-  }
-
-  protected TaskSplitMetaInfo[] readSplits(Configuration conf)
-      throws IOException {
-    TaskSplitMetaInfo[] allTaskSplitMetaInfo;
-    allTaskSplitMetaInfo = SplitMetaInfoReaderTez.readSplitMetaInfo(conf,
-        FileSystem.getLocal(conf));
-    return allTaskSplitMetaInfo;
-  }
-  
-  private class MRInputKVReader implements KVReader {
-    
-    Object key;
-    Object value;
-
-    private SimpleValueIterator valueIterator = new SimpleValueIterator();
-    private SimpleIterable valueIterable = new SimpleIterable(valueIterator);
-
-    private final boolean localNewApi;
-    
-    MRInputKVReader() {
-      localNewApi = useNewApi;
-      if (!localNewApi) {
-        key = oldRecordReader.createKey();
-        value =oldRecordReader.createValue();
-      }
-    }
-    
-    // Setup the values iterator once, and set value on the same object each time
-    // to prevent lots of objects being created.
-
-    
-    @SuppressWarnings("unchecked")
-    @Override
-    public boolean next() throws IOException {
-      boolean hasNext = false;
-      long bytesInPrev = getInputBytes();
-      if (localNewApi) {
-        try {
-          hasNext = newRecordReader.nextKeyValue();
-        } catch (InterruptedException e) {
-          Thread.currentThread().interrupt();
-          throw new IOException("Interrupted while checking for next key-value", e);
-        }
-      } else {
-        hasNext = oldRecordReader.next(key, value);
-      }
-      long bytesInCurr = getInputBytes();
-      fileInputByteCounter.increment(bytesInCurr - bytesInPrev);
-      
-      if (hasNext) {
-        inputRecordCounter.increment(1);
-      }
-      
-      return hasNext;
-    }
-
-    @Override
-    public KVRecord getCurrentKV() throws IOException {
-      KVRecord kvRecord = null;
-      if (localNewApi) {
-        try {
-          valueIterator.setValue(newRecordReader.getCurrentValue());
-          kvRecord = new KVRecord(newRecordReader.getCurrentKey(), valueIterable);
-        } catch (InterruptedException e) {
-          Thread.currentThread().interrupt();
-          throw new IOException("Interrupted while fetching next key-value", e);
-        }
-        
-      } else {
-        valueIterator.setValue(value);
-        kvRecord = new KVRecord(key, valueIterable);
-      }
-      return kvRecord;
-    }
-  };
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/3d609458/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/newinput/SimpleInputLegacy.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/newinput/SimpleInputLegacy.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/newinput/SimpleInputLegacy.java
deleted file mode 100644
index 8f07a38..0000000
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/newinput/SimpleInputLegacy.java
+++ /dev/null
@@ -1,36 +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.newinput;
-
-import org.apache.hadoop.classification.InterfaceAudience.Private;
-import org.apache.hadoop.mapred.RecordReader;
-
-public class SimpleInputLegacy extends SimpleInput {
-
-  @Private
-  public org.apache.hadoop.mapreduce.InputSplit getNewInputSplit() {
-    return this.newInputSplit;
-  }  
-  
-  @SuppressWarnings("rawtypes")
-  @Private
-  public RecordReader getOldRecordReader() {
-    return this.oldRecordReader;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/3d609458/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/newoutput/SimpleOutput.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/newoutput/SimpleOutput.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/newoutput/SimpleOutput.java
deleted file mode 100644
index a8fb900..0000000
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/newoutput/SimpleOutput.java
+++ /dev/null
@@ -1,326 +0,0 @@
-package org.apache.tez.mapreduce.newoutput;
-
-import java.io.IOException;
-import java.text.NumberFormat;
-import java.util.List;
-import java.util.concurrent.atomic.AtomicBoolean;
-
-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.Path;
-import org.apache.hadoop.fs.FileSystem.Statistics;
-import org.apache.hadoop.mapred.FileOutputCommitter;
-import org.apache.hadoop.mapred.FileOutputFormat;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.JobContext;
-import org.apache.hadoop.mapred.TaskAttemptID;
-import org.apache.hadoop.mapred.TaskID;
-import org.apache.hadoop.mapreduce.OutputCommitter;
-import org.apache.hadoop.mapreduce.OutputFormat;
-import org.apache.hadoop.mapreduce.TaskAttemptContext;
-import org.apache.hadoop.mapreduce.TaskType;
-import org.apache.hadoop.mapreduce.lib.output.FileOutputFormatCounter;
-import org.apache.hadoop.util.ReflectionUtils;
-import org.apache.tez.common.TezUtils;
-import org.apache.tez.common.counters.TaskCounter;
-import org.apache.tez.common.counters.TezCounter;
-import org.apache.tez.engine.newapi.Event;
-import org.apache.tez.engine.newapi.KVWriter;
-import org.apache.tez.engine.newapi.LogicalOutput;
-import org.apache.tez.engine.newapi.TezOutputContext;
-import org.apache.tez.mapreduce.common.Utils;
-import org.apache.tez.mapreduce.hadoop.MRConfig;
-import org.apache.tez.mapreduce.hadoop.MRJobConfig;
-import org.apache.tez.mapreduce.hadoop.newmapred.MRReporter;
-import org.apache.tez.mapreduce.hadoop.newmapreduce.TaskAttemptContextImpl;
-import org.apache.tez.mapreduce.newprocessor.MRTaskReporter;
-
-public class SimpleOutput implements LogicalOutput {
-
-  private static final Log LOG = LogFactory.getLog(SimpleOutput.class);
-
-  private static final NumberFormat NUMBER_FORMAT = NumberFormat.getInstance();
-  static {
-    NUMBER_FORMAT.setMinimumIntegerDigits(5);
-    NUMBER_FORMAT.setGroupingUsed(false);
-  }
-
-  private TezOutputContext outputContext;
-  private JobConf jobConf;
-  boolean useNewApi;
-  private AtomicBoolean closed = new AtomicBoolean(false);
-
-  @SuppressWarnings("rawtypes")
-  org.apache.hadoop.mapreduce.OutputFormat newOutputFormat;
-  @SuppressWarnings("rawtypes")
-  org.apache.hadoop.mapreduce.RecordWriter newRecordWriter;
-
-  @SuppressWarnings("rawtypes")
-  org.apache.hadoop.mapred.OutputFormat oldOutputFormat;
-  @SuppressWarnings("rawtypes")
-  org.apache.hadoop.mapred.RecordWriter oldRecordWriter;
-
-  private TezCounter outputRecordCounter;
-  private TezCounter fileOutputByteCounter;
-  private List<Statistics> fsStats;
-
-  private TaskAttemptContext newApiTaskAttemptContext;
-  private org.apache.hadoop.mapred.TaskAttemptContext oldApiTaskAttemptContext;
-
-  private boolean isMapperOutput;
-
-  private OutputCommitter committer;
-
-  @Override
-  public List<Event> initialize(TezOutputContext outputContext)
-      throws IOException, InterruptedException {
-    LOG.info("Initializing Simple Output");
-    this.outputContext = outputContext;
-    Configuration conf = TezUtils.createConfFromUserPayload(
-        outputContext.getUserPayload());
-    this.jobConf = new JobConf(conf);
-    this.useNewApi = this.jobConf.getUseNewMapper();
-    this.isMapperOutput = jobConf.getBoolean(MRConfig.IS_MAP_PROCESSOR,
-        false);
-    jobConf.setInt(MRJobConfig.APPLICATION_ATTEMPT_ID,
-        outputContext.getDAGAttemptNumber());
-
-    outputRecordCounter = outputContext.getCounters().findCounter(
-        TaskCounter.MAP_OUTPUT_RECORDS);
-    fileOutputByteCounter = outputContext.getCounters().findCounter(
-        FileOutputFormatCounter.BYTES_WRITTEN);
-
-    if (useNewApi) {
-      newApiTaskAttemptContext = createTaskAttemptContext();
-      try {
-        newOutputFormat =
-            ReflectionUtils.newInstance(
-                newApiTaskAttemptContext.getOutputFormatClass(), jobConf);
-      } catch (ClassNotFoundException cnfe) {
-        throw new IOException(cnfe);
-      }
-
-      List<Statistics> matchedStats = null;
-      if (newOutputFormat instanceof
-          org.apache.hadoop.mapreduce.lib.output.FileOutputFormat) {
-        matchedStats =
-            Utils.getFsStatistics(
-                org.apache.hadoop.mapreduce.lib.output.FileOutputFormat
-                    .getOutputPath(newApiTaskAttemptContext),
-                jobConf);
-      }
-      fsStats = matchedStats;
-
-      long bytesOutPrev = getOutputBytes();
-      try {
-        newRecordWriter =
-            newOutputFormat.getRecordWriter(newApiTaskAttemptContext);
-      } catch (InterruptedException e) {
-        throw new IOException("Interrupted while creating record writer", e);
-      }
-      long bytesOutCurr = getOutputBytes();
-      fileOutputByteCounter.increment(bytesOutCurr - bytesOutPrev);
-    } else {
-      TaskAttemptID taskAttemptId = new TaskAttemptID(new TaskID(Long.toString(
-          outputContext.getApplicationId().getClusterTimestamp()),
-          outputContext.getApplicationId().getId(),
-          (isMapperOutput ? TaskType.MAP : TaskType.REDUCE),
-          outputContext.getTaskIndex()),
-          outputContext.getTaskAttemptNumber());
-      jobConf.set(JobContext.TASK_ATTEMPT_ID, taskAttemptId.toString());
-      jobConf.set(JobContext.TASK_ID, taskAttemptId.getTaskID().toString());
-      jobConf.setBoolean(JobContext.TASK_ISMAP, isMapperOutput);
-      jobConf.setInt(JobContext.TASK_PARTITION,
-          taskAttemptId.getTaskID().getId());
-      jobConf.set(JobContext.ID, taskAttemptId.getJobID().toString());
-
-      oldApiTaskAttemptContext =
-          new org.apache.tez.mapreduce.hadoop.newmapred.TaskAttemptContextImpl(
-              jobConf, taskAttemptId,
-              new MRTaskReporter(outputContext));
-      oldOutputFormat = jobConf.getOutputFormat();
-
-      List<Statistics> matchedStats = null;
-      if (oldOutputFormat
-          instanceof org.apache.hadoop.mapred.FileOutputFormat) {
-        matchedStats =
-            Utils.getFsStatistics(
-                org.apache.hadoop.mapred.FileOutputFormat.getOutputPath(
-                    jobConf),
-                jobConf);
-      }
-      fsStats = matchedStats;
-
-      FileSystem fs = FileSystem.get(jobConf);
-      String finalName = getOutputName();
-
-      long bytesOutPrev = getOutputBytes();
-      oldRecordWriter =
-          oldOutputFormat.getRecordWriter(
-              fs, jobConf, finalName, new MRReporter(outputContext));
-      long bytesOutCurr = getOutputBytes();
-      fileOutputByteCounter.increment(bytesOutCurr - bytesOutPrev);
-    }
-    initCommitter(jobConf, useNewApi);
-
-    LOG.info("Initialized Simple Output"
-        + ", using_new_api: " + useNewApi);
-    return null;
-  }
-
-  public void initCommitter(JobConf job, boolean useNewApi)
-      throws IOException, InterruptedException {
-
-    if (useNewApi) {
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("using new api for output committer");
-      }
-
-      OutputFormat<?, ?> outputFormat = null;
-      try {
-        outputFormat = ReflectionUtils.newInstance(
-            newApiTaskAttemptContext.getOutputFormatClass(), job);
-      } catch (ClassNotFoundException cnfe) {
-        throw new IOException("Unknown OutputFormat", cnfe);
-      }
-      this.committer = outputFormat.getOutputCommitter(
-          newApiTaskAttemptContext);
-    } else {
-      this.committer = job.getOutputCommitter();
-    }
-
-    Path outputPath = FileOutputFormat.getOutputPath(job);
-    if (outputPath != null) {
-      if ((this.committer instanceof FileOutputCommitter)) {
-        FileOutputFormat.setWorkOutputPath(job,
-            ((FileOutputCommitter) this.committer).getTaskAttemptPath(
-                oldApiTaskAttemptContext));
-      } else {
-        FileOutputFormat.setWorkOutputPath(job, outputPath);
-      }
-    }
-    if (useNewApi) {
-      this.committer.setupTask(newApiTaskAttemptContext);
-    } else {
-      this.committer.setupTask(oldApiTaskAttemptContext);
-    }
-  }
-
-  public boolean isCommitRequired() throws IOException {
-    if (useNewApi) {
-      return committer.needsTaskCommit(newApiTaskAttemptContext);
-    } else {
-      return committer.needsTaskCommit(oldApiTaskAttemptContext);
-    }
-  }
-
-  private TaskAttemptContext createTaskAttemptContext() {
-    return new TaskAttemptContextImpl(this.jobConf, outputContext,
-        isMapperOutput);
-  }
-
-  private long getOutputBytes() {
-    if (fsStats == null) return 0;
-    long bytesWritten = 0;
-    for (Statistics stat: fsStats) {
-      bytesWritten = bytesWritten + stat.getBytesWritten();
-    }
-    return bytesWritten;
-  }
-
-  private String getOutputName() {
-    return "part-" + NUMBER_FORMAT.format(outputContext.getTaskIndex());
-  }
-
-  @Override
-  public KVWriter getWriter() throws IOException {
-    return new KVWriter() {
-      private final boolean useNewWriter = useNewApi;
-
-      @SuppressWarnings("unchecked")
-      @Override
-      public void write(Object key, Object value) throws IOException {
-        long bytesOutPrev = getOutputBytes();
-        if (useNewWriter) {
-          try {
-            newRecordWriter.write(key, value);
-          } catch (InterruptedException e) {
-            Thread.currentThread().interrupt();
-            throw new IOException("Interrupted while writing next key-value",e);
-          }
-        } else {
-          oldRecordWriter.write(key, value);
-        }
-
-        long bytesOutCurr = getOutputBytes();
-        fileOutputByteCounter.increment(bytesOutCurr - bytesOutPrev);
-        outputRecordCounter.increment(1);
-      }
-    };
-  }
-
-  @Override
-  public void handleEvents(List<Event> outputEvents) {
-    // Not expecting any events at the moment.
-  }
-
-  @Override
-  public synchronized List<Event> close() throws IOException {
-    if (closed.getAndSet(true)) {
-      return null;
-    }
-
-    LOG.info("Closing Simple Output");
-    long bytesOutPrev = getOutputBytes();
-    if (useNewApi) {
-      try {
-        newRecordWriter.close(newApiTaskAttemptContext);
-      } catch (InterruptedException e) {
-        throw new IOException("Interrupted while closing record writer", e);
-      }
-    } else {
-      oldRecordWriter.close(null);
-    }
-    long bytesOutCurr = getOutputBytes();
-    fileOutputByteCounter.increment(bytesOutCurr - bytesOutPrev);
-    LOG.info("Closed Simple Output");
-    return null;
-  }
-
-  @Override
-  public void setNumPhysicalOutputs(int numOutputs) {
-    // Nothing to do for now
-  }
-
-  /**
-   * SimpleOutput expects that a Processor call commit prior to the
-   * Processor's completion
-   * @throws IOException
-   */
-  public void commit() throws IOException {
-    close();
-    if (useNewApi) {
-      committer.commitTask(newApiTaskAttemptContext);
-    } else {
-      committer.commitTask(oldApiTaskAttemptContext);
-    }
-  }
-
-
-  /**
-   * SimpleOutput expects that a Processor call abort in case of any error
-   * ( including an error during commit ) prior to the Processor's completion
-   * @throws IOException
-   */
-  public void abort() throws IOException {
-    close();
-    if (useNewApi) {
-      committer.abortTask(newApiTaskAttemptContext);
-    } else {
-      committer.abortTask(oldApiTaskAttemptContext);
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/3d609458/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/newpartition/MRPartitioner.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/newpartition/MRPartitioner.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/newpartition/MRPartitioner.java
deleted file mode 100644
index dcea35c..0000000
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/newpartition/MRPartitioner.java
+++ /dev/null
@@ -1,88 +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.newpartition;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.util.ReflectionUtils;
-import org.apache.tez.common.TezJobConfig;
-import org.apache.tez.engine.common.ConfigUtils;
-import org.apache.tez.mapreduce.hadoop.MRJobConfig;
-
-@SuppressWarnings({ "rawtypes", "unchecked" })
-public class MRPartitioner implements org.apache.tez.engine.api.Partitioner {
-
-  static final Log LOG = LogFactory.getLog(MRPartitioner.class);
-
-  private final boolean useNewApi;
-  private int partitions = 1;
-
-  private org.apache.hadoop.mapreduce.Partitioner newPartitioner;
-  private org.apache.hadoop.mapred.Partitioner oldPartitioner;
-
-  public MRPartitioner(Configuration conf) {
-    this.useNewApi = ConfigUtils.useNewApi(conf);
-    this.partitions = conf.getInt(TezJobConfig.TEZ_ENGINE_NUM_EXPECTED_PARTITIONS, 1);
-
-    if (useNewApi) {
-      if (partitions > 1) {
-        newPartitioner = (org.apache.hadoop.mapreduce.Partitioner) ReflectionUtils
-            .newInstance(
-                (Class<? extends org.apache.hadoop.mapreduce.Partitioner<?, ?>>) conf
-                    .getClass(MRJobConfig.PARTITIONER_CLASS_ATTR,
-                        org.apache.hadoop.mapreduce.lib.partition.HashPartitioner.class), conf);
-      } else {
-        newPartitioner = new org.apache.hadoop.mapreduce.Partitioner() {
-          @Override
-          public int getPartition(Object key, Object value, int numPartitions) {
-            return numPartitions - 1;
-          }
-        };
-      }
-    } else {
-      if (partitions > 1) {
-        oldPartitioner = (org.apache.hadoop.mapred.Partitioner) ReflectionUtils.newInstance(
-            (Class<? extends org.apache.hadoop.mapred.Partitioner>) conf.getClass(
-                "mapred.partitioner.class", org.apache.hadoop.mapred.lib.HashPartitioner.class), conf);
-      } else {
-        oldPartitioner = new org.apache.hadoop.mapred.Partitioner() {
-          @Override
-          public void configure(JobConf job) {
-          }
-
-          @Override
-          public int getPartition(Object key, Object value, int numPartitions) {
-            return numPartitions - 1;
-          }
-        };
-      }
-    }
-  }
-
-  @Override
-  public int getPartition(Object key, Object value, int numPartitions) {
-    if (useNewApi) {
-      return newPartitioner.getPartition(key, value, numPartitions);
-    } else {
-      return oldPartitioner.getPartition(key, value, numPartitions);
-    }
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/3d609458/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/newprocessor/FileSystemStatisticsUpdater.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/newprocessor/FileSystemStatisticsUpdater.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/newprocessor/FileSystemStatisticsUpdater.java
deleted file mode 100644
index 5f97049..0000000
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/newprocessor/FileSystemStatisticsUpdater.java
+++ /dev/null
@@ -1,84 +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.newprocessor;
-
-import java.util.List;
-
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.tez.common.counters.FileSystemCounter;
-import org.apache.tez.common.counters.TezCounter;
-import org.apache.tez.common.counters.TezCounters;
-
-  /**
-   * An updater that tracks the last number reported for a given file
-   * system and only creates the counters when they are needed.
-   */
-  class FileSystemStatisticUpdater {
-    private List<FileSystem.Statistics> stats;
-    private TezCounter readBytesCounter, writeBytesCounter,
-        readOpsCounter, largeReadOpsCounter, writeOpsCounter;
-    private String scheme;
-    private TezCounters counters;
-
-    FileSystemStatisticUpdater(TezCounters counters, List<FileSystem.Statistics> stats, String scheme) {
-      this.stats = stats;
-      this.scheme = scheme;
-      this.counters = counters;
-    }
-
-    void updateCounters() {
-      if (readBytesCounter == null) {
-        readBytesCounter = counters.findCounter(scheme,
-            FileSystemCounter.BYTES_READ);
-      }
-      if (writeBytesCounter == null) {
-        writeBytesCounter = counters.findCounter(scheme,
-            FileSystemCounter.BYTES_WRITTEN);
-      }
-      if (readOpsCounter == null) {
-        readOpsCounter = counters.findCounter(scheme,
-            FileSystemCounter.READ_OPS);
-      }
-      if (largeReadOpsCounter == null) {
-        largeReadOpsCounter = counters.findCounter(scheme,
-            FileSystemCounter.LARGE_READ_OPS);
-      }
-      if (writeOpsCounter == null) {
-        writeOpsCounter = counters.findCounter(scheme,
-            FileSystemCounter.WRITE_OPS);
-      }
-      long readBytes = 0;
-      long writeBytes = 0;
-      long readOps = 0;
-      long largeReadOps = 0;
-      long writeOps = 0;
-      for (FileSystem.Statistics stat: stats) {
-        readBytes = readBytes + stat.getBytesRead();
-        writeBytes = writeBytes + stat.getBytesWritten();
-        readOps = readOps + stat.getReadOps();
-        largeReadOps = largeReadOps + stat.getLargeReadOps();
-        writeOps = writeOps + stat.getWriteOps();
-      }
-      readBytesCounter.setValue(readBytes);
-      writeBytesCounter.setValue(writeBytes);
-      readOpsCounter.setValue(readOps);
-      largeReadOpsCounter.setValue(largeReadOps);
-      writeOpsCounter.setValue(writeOps);
-    }
-  }
-  

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/3d609458/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/newprocessor/GcTimeUpdater.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/newprocessor/GcTimeUpdater.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/newprocessor/GcTimeUpdater.java
deleted file mode 100644
index b05f011..0000000
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/newprocessor/GcTimeUpdater.java
+++ /dev/null
@@ -1,71 +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.newprocessor;
-
-import java.lang.management.GarbageCollectorMXBean;
-import java.lang.management.ManagementFactory;
-import java.util.List;
-
-import org.apache.tez.common.counters.TezCounter;
-import org.apache.tez.common.counters.TezCounters;
-import org.apache.tez.common.counters.TaskCounter;
-
-/**
-   * An updater that tracks the amount of time this task has spent in GC.
-   */
-  class GcTimeUpdater {
-    private long lastGcMillis = 0;
-    private List<GarbageCollectorMXBean> gcBeans = null;
-    TezCounters counters;
-
-    public GcTimeUpdater(TezCounters counters) {
-      this.gcBeans = ManagementFactory.getGarbageCollectorMXBeans();
-      getElapsedGc(); // Initialize 'lastGcMillis' with the current time spent.
-      this.counters = counters;
-    }
-
-    /**
-     * @return the number of milliseconds that the gc has used for CPU
-     * since the last time this method was called.
-     */
-    protected long getElapsedGc() {
-      long thisGcMillis = 0;
-      for (GarbageCollectorMXBean gcBean : gcBeans) {
-        thisGcMillis += gcBean.getCollectionTime();
-      }
-
-      long delta = thisGcMillis - lastGcMillis;
-      this.lastGcMillis = thisGcMillis;
-      return delta;
-    }
-
-    /**
-     * Increment the gc-elapsed-time counter.
-     */
-    public void incrementGcCounter() {
-      if (null == counters) {
-        return; // nothing to do.
-      }
-
-      TezCounter gcCounter =
-        counters.findCounter(TaskCounter.GC_TIME_MILLIS);
-      if (null != gcCounter) {
-        gcCounter.increment(getElapsedGc());
-      }
-    }
-  }


[44/50] [abbrv] Rename tez-engine-api to tez-runtime-api and tez-engine is split into 2: - tez-engine-library for user-visible Input/Output/Processor implementations - tez-engine-internals for framework internals

Posted by ss...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezHeartbeatRequest.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezHeartbeatRequest.java b/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezHeartbeatRequest.java
deleted file mode 100644
index dc1a447..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezHeartbeatRequest.java
+++ /dev/null
@@ -1,137 +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.engine.api.impl;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.Writable;
-import org.apache.tez.dag.records.TezTaskAttemptID;
-
-
-public class TezHeartbeatRequest implements Writable {
-
-  private String containerIdentifier;
-  private List<TezEvent> events;
-  private TezTaskAttemptID currentTaskAttemptID;
-  private int startIndex;
-  private int maxEvents;
-  private long requestId;
-
-  public TezHeartbeatRequest() {
-  }
-
-  public TezHeartbeatRequest(long requestId, List<TezEvent> events,
-      String containerIdentifier, TezTaskAttemptID taskAttemptID,
-      int startIndex, int maxEvents) {
-    this.containerIdentifier = containerIdentifier;
-    this.requestId = requestId;
-    this.events = Collections.unmodifiableList(events);
-    this.startIndex = startIndex;
-    this.maxEvents = maxEvents;
-    this.currentTaskAttemptID = taskAttemptID;
-  }
-
-  public String getContainerIdentifier() {
-    return containerIdentifier;
-  }
-
-  public List<TezEvent> getEvents() {
-    return events;
-  }
-
-  public int getStartIndex() {
-    return startIndex;
-  }
-
-  public int getMaxEvents() {
-    return maxEvents;
-  }
-
-  public long getRequestId() {
-    return requestId;
-  }
-
-  public TezTaskAttemptID getCurrentTaskAttemptID() {
-    return currentTaskAttemptID;
-  }
-
-  @Override
-  public void write(DataOutput out) throws IOException {
-    if (events != null) {
-      out.writeBoolean(true);
-      out.writeInt(events.size());
-      for (TezEvent e : events) {
-        e.write(out);
-      }
-    } else {
-      out.writeBoolean(false);
-    }
-    if (currentTaskAttemptID != null) {
-      out.writeBoolean(true);
-      currentTaskAttemptID.write(out);
-    } else {
-      out.writeBoolean(false);
-    }
-    out.writeInt(startIndex);
-    out.writeInt(maxEvents);
-    out.writeLong(requestId);
-    Text.writeString(out, containerIdentifier);
-  }
-
-  @Override
-  public void readFields(DataInput in) throws IOException {
-    if (in.readBoolean()) {
-      int eventsCount = in.readInt();
-      events = new ArrayList<TezEvent>(eventsCount);
-      for (int i = 0; i < eventsCount; ++i) {
-        TezEvent e = new TezEvent();
-        e.readFields(in);
-        events.add(e);
-      }
-    }
-    if (in.readBoolean()) {
-      currentTaskAttemptID = new TezTaskAttemptID();
-      currentTaskAttemptID.readFields(in);
-    } else {
-      currentTaskAttemptID = null;
-    }
-    startIndex = in.readInt();
-    maxEvents = in.readInt();
-    requestId = in.readLong();
-    containerIdentifier = Text.readString(in);
-  }
-
-  @Override
-  public String toString() {
-    return "{ "
-        + " containerId=" + containerIdentifier
-        + ", requestId=" + requestId
-        + ", startIndex=" + startIndex
-        + ", maxEventsToGet=" + maxEvents
-        + ", taskAttemptId" + currentTaskAttemptID
-        + ", eventCount=" + (events != null ? events.size() : 0)
-        + " }";
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezHeartbeatResponse.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezHeartbeatResponse.java b/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezHeartbeatResponse.java
deleted file mode 100644
index 22ae7eb..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezHeartbeatResponse.java
+++ /dev/null
@@ -1,105 +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.engine.api.impl;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-
-import org.apache.hadoop.io.Writable;
-
-public class TezHeartbeatResponse implements Writable {
-
-  private long lastRequestId;
-  private boolean shouldDie = false;
-  private List<TezEvent> events;
-
-  public TezHeartbeatResponse() {
-  }
-
-  public TezHeartbeatResponse(List<TezEvent> events) {
-    this.events = Collections.unmodifiableList(events);
-  }
-
-  public List<TezEvent> getEvents() {
-    return events;
-  }
-
-  public boolean shouldDie() {
-    return shouldDie;
-  }
-
-  public long getLastRequestId() {
-    return lastRequestId;
-  }
-
-  public void setEvents(List<TezEvent> events) {
-    this.events = Collections.unmodifiableList(events);
-  }
-
-  public void setLastRequestId(long lastRequestId ) {
-    this.lastRequestId = lastRequestId;
-  }
-
-  public void setShouldDie() {
-    this.shouldDie = true;
-  }
-
-  @Override
-  public void write(DataOutput out) throws IOException {
-    out.writeLong(lastRequestId);
-    out.writeBoolean(shouldDie);
-    if(events != null) {
-      out.writeBoolean(true);
-      out.writeInt(events.size());
-      for (TezEvent e : events) {
-        e.write(out);
-      }
-    } else {
-      out.writeBoolean(false);
-    }
-  }
-
-  @Override
-  public void readFields(DataInput in) throws IOException {
-    lastRequestId = in.readLong();
-    shouldDie = in.readBoolean();
-    if(in.readBoolean()) {
-      int eventCount = in.readInt();
-      events = new ArrayList<TezEvent>(eventCount);
-      for (int i = 0; i < eventCount; ++i) {
-        TezEvent e = new TezEvent();
-        e.readFields(in);
-        events.add(e);
-      }
-    }
-  }
-
-  @Override
-  public String toString() {
-    return "{ "
-        + " lastRequestId=" + lastRequestId
-        + ", shouldDie=" + shouldDie
-        + ", eventCount=" + (events != null ? events.size() : 0)
-        + " }";
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezInputContextImpl.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezInputContextImpl.java b/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezInputContextImpl.java
deleted file mode 100644
index 245cd3b..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezInputContextImpl.java
+++ /dev/null
@@ -1,84 +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.engine.api.impl;
-
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.hadoop.classification.InterfaceAudience.Private;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.tez.common.counters.TezCounters;
-import org.apache.tez.dag.records.TezTaskAttemptID;
-import org.apache.tez.engine.api.Event;
-import org.apache.tez.engine.api.TezInputContext;
-import org.apache.tez.engine.api.impl.EventMetaData.EventProducerConsumerType;
-import org.apache.tez.engine.newruntime.RuntimeTask;
-
-public class TezInputContextImpl extends TezTaskContextImpl
-    implements TezInputContext {
-
-  private final byte[] userPayload;
-  private final String sourceVertexName;
-  private final EventMetaData sourceInfo;
-
-  @Private
-  public TezInputContextImpl(Configuration conf, int appAttemptNumber,
-      TezUmbilical tezUmbilical, String taskVertexName,
-      String sourceVertexName, TezTaskAttemptID taskAttemptID,
-      TezCounters counters, byte[] userPayload,
-      RuntimeTask runtimeTask, Map<String, ByteBuffer> serviceConsumerMetadata) {
-    super(conf, appAttemptNumber, taskVertexName, taskAttemptID,
-        counters, runtimeTask, tezUmbilical, serviceConsumerMetadata);
-    this.userPayload = userPayload;
-    this.sourceVertexName = sourceVertexName;
-    this.sourceInfo = new EventMetaData(
-        EventProducerConsumerType.INPUT, taskVertexName, sourceVertexName,
-        taskAttemptID);
-    this.uniqueIdentifier = String.format("%s_%s_%06d_%02d_%s", taskAttemptID
-        .getTaskID().getVertexID().getDAGId().toString(), taskVertexName,
-        getTaskIndex(), getTaskAttemptNumber(), sourceVertexName);
-  }
-
-  @Override
-  public void sendEvents(List<Event> events) {
-    List<TezEvent> tezEvents = new ArrayList<TezEvent>(events.size());
-    for (Event e : events) {
-      TezEvent tEvt = new TezEvent(e, sourceInfo);
-      tezEvents.add(tEvt);
-    }
-    tezUmbilical.addEvents(tezEvents);
-  }
-
-  @Override
-  public byte[] getUserPayload() {
-    return userPayload;
-  }
-
-  @Override
-  public String getSourceVertexName() {
-    return sourceVertexName;
-  }
-
-  @Override
-  public void fatalError(Throwable exception, String message) {
-    super.signalFatalError(exception, message, sourceInfo);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezOutputContextImpl.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezOutputContextImpl.java b/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezOutputContextImpl.java
deleted file mode 100644
index 6b42e13..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezOutputContextImpl.java
+++ /dev/null
@@ -1,85 +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.engine.api.impl;
-
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.hadoop.classification.InterfaceAudience.Private;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.tez.common.counters.TezCounters;
-import org.apache.tez.dag.records.TezTaskAttemptID;
-import org.apache.tez.engine.api.Event;
-import org.apache.tez.engine.api.TezOutputContext;
-import org.apache.tez.engine.api.impl.EventMetaData.EventProducerConsumerType;
-import org.apache.tez.engine.newruntime.RuntimeTask;
-
-public class TezOutputContextImpl extends TezTaskContextImpl
-    implements TezOutputContext {
-
-  private final byte[] userPayload;
-  private final String destinationVertexName;
-  private final EventMetaData sourceInfo;
-
-  @Private
-  public TezOutputContextImpl(Configuration conf, int appAttemptNumber,
-      TezUmbilical tezUmbilical, String taskVertexName,
-      String destinationVertexName,
-      TezTaskAttemptID taskAttemptID, TezCounters counters,
-      byte[] userPayload, RuntimeTask runtimeTask,
-      Map<String, ByteBuffer> serviceConsumerMetadata) {
-    super(conf, appAttemptNumber, taskVertexName, taskAttemptID,
-        counters, runtimeTask, tezUmbilical, serviceConsumerMetadata);
-    this.userPayload = userPayload;
-    this.destinationVertexName = destinationVertexName;
-    this.sourceInfo = new EventMetaData(EventProducerConsumerType.OUTPUT,
-        taskVertexName, destinationVertexName, taskAttemptID);
-    this.uniqueIdentifier = String.format("%s_%s_%06d_%02d_%s", taskAttemptID
-        .getTaskID().getVertexID().getDAGId().toString(), taskVertexName,
-        getTaskIndex(), getTaskAttemptNumber(), destinationVertexName);
-  }
-
-  @Override
-  public void sendEvents(List<Event> events) {
-    List<TezEvent> tezEvents = new ArrayList<TezEvent>(events.size());
-    for (Event e : events) {
-      TezEvent tEvt = new TezEvent(e, sourceInfo);
-      tezEvents.add(tEvt);
-    }
-    tezUmbilical.addEvents(tezEvents);
-  }
-
-  @Override
-  public byte[] getUserPayload() {
-    return userPayload;
-  }
-
-  @Override
-  public String getDestinationVertexName() {
-    return destinationVertexName;
-  }
-
-  @Override
-  public void fatalError(Throwable exception, String message) {
-    super.signalFatalError(exception, message, sourceInfo);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezProcessorContextImpl.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezProcessorContextImpl.java b/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezProcessorContextImpl.java
deleted file mode 100644
index 7ffcfd6..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezProcessorContextImpl.java
+++ /dev/null
@@ -1,86 +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.engine.api.impl;
-
-import java.nio.ByteBuffer;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.tez.common.counters.TezCounters;
-import org.apache.tez.dag.records.TezTaskAttemptID;
-import org.apache.tez.engine.api.Event;
-import org.apache.tez.engine.api.TezProcessorContext;
-import org.apache.tez.engine.api.impl.EventMetaData.EventProducerConsumerType;
-import org.apache.tez.engine.newruntime.RuntimeTask;
-
-public class TezProcessorContextImpl extends TezTaskContextImpl
-  implements TezProcessorContext {
-
-  private final byte[] userPayload;
-  private final EventMetaData sourceInfo;
-
-  public TezProcessorContextImpl(Configuration conf, int appAttemptNumber,
-      TezUmbilical tezUmbilical, String vertexName,
-      TezTaskAttemptID taskAttemptID, TezCounters counters,
-      byte[] userPayload, RuntimeTask runtimeTask,
-      Map<String, ByteBuffer> serviceConsumerMetadata) {
-    super(conf, appAttemptNumber, vertexName, taskAttemptID,
-        counters, runtimeTask, tezUmbilical, serviceConsumerMetadata);
-    this.userPayload = userPayload;
-    this.sourceInfo = new EventMetaData(EventProducerConsumerType.PROCESSOR,
-        taskVertexName, "", taskAttemptID);
-    this.uniqueIdentifier = String.format("%s_%s_%06d_%02d", taskAttemptID
-        .getTaskID().getVertexID().getDAGId().toString(), taskVertexName,
-        getTaskIndex(), getTaskAttemptNumber());
-  }
-
-  @Override
-  public void sendEvents(List<Event> events) {
-    List<TezEvent> tezEvents = new ArrayList<TezEvent>(events.size());
-    for (Event e : events) {
-      TezEvent tEvt = new TezEvent(e, sourceInfo);
-      tezEvents.add(tEvt);
-    }
-    tezUmbilical.addEvents(tezEvents);
-  }
-
-  @Override
-  public byte[] getUserPayload() {
-    return userPayload;
-  }
-
-  @Override
-  public void setProgress(float progress) {
-    runtimeTask.setProgress(progress);
-  }
-
-  @Override
-  public void fatalError(Throwable exception, String message) {
-    super.signalFatalError(exception, message, sourceInfo);
-  }
-
-  @Override
-  public boolean canCommit() throws IOException {
-    return tezUmbilical.canCommit(this.taskAttemptID);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezTaskContextImpl.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezTaskContextImpl.java b/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezTaskContextImpl.java
deleted file mode 100644
index d5a4037..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezTaskContextImpl.java
+++ /dev/null
@@ -1,145 +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.engine.api.impl;
-
-import java.nio.ByteBuffer;
-import java.util.Arrays;
-import java.util.Map;
-
-import org.apache.hadoop.classification.InterfaceAudience.Private;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.util.StringUtils;
-import org.apache.hadoop.yarn.api.records.ApplicationId;
-import org.apache.hadoop.yarn.util.AuxiliaryServiceHelper;
-import org.apache.tez.common.TezJobConfig;
-import org.apache.tez.common.counters.TezCounters;
-import org.apache.tez.dag.records.TezTaskAttemptID;
-import org.apache.tez.engine.api.TezTaskContext;
-import org.apache.tez.engine.newruntime.RuntimeTask;
-
-public abstract class TezTaskContextImpl implements TezTaskContext {
-
-  private final Configuration conf;
-  protected final String taskVertexName;
-  protected final TezTaskAttemptID taskAttemptID;
-  private final TezCounters counters;
-  private String[] workDirs;
-  protected String uniqueIdentifier;
-  protected final RuntimeTask runtimeTask;
-  protected final TezUmbilical tezUmbilical;
-  private final Map<String, ByteBuffer> serviceConsumerMetadata;
-  private final int appAttemptNumber;
-
-  @Private
-  public TezTaskContextImpl(Configuration conf, int appAttemptNumber,
-      String taskVertexName, TezTaskAttemptID taskAttemptID,
-      TezCounters counters, RuntimeTask runtimeTask,
-      TezUmbilical tezUmbilical, Map<String, ByteBuffer> serviceConsumerMetadata) {
-    this.conf = conf;
-    this.taskVertexName = taskVertexName;
-    this.taskAttemptID = taskAttemptID;
-    this.counters = counters;
-    // TODO Maybe change this to be task id specific at some point. For now
-    // Shuffle code relies on this being a path specified by YARN
-    this.workDirs = this.conf.getStrings(TezJobConfig.LOCAL_DIRS);
-    this.runtimeTask = runtimeTask;
-    this.tezUmbilical = tezUmbilical;
-    this.serviceConsumerMetadata = serviceConsumerMetadata;
-    // TODO NEWTEZ at some point dag attempt should not map to app attempt
-    this.appAttemptNumber = appAttemptNumber;
-  }
-
-  @Override
-  public ApplicationId getApplicationId() {
-    return taskAttemptID.getTaskID().getVertexID().getDAGId()
-        .getApplicationId();
-  }
-
-  @Override
-  public int getTaskIndex() {
-    return taskAttemptID.getTaskID().getId();
-  }
-
-  @Override
-  public int getDAGAttemptNumber() {
-    return appAttemptNumber;
-  }
-
-  @Override
-  public int getTaskAttemptNumber() {
-    return taskAttemptID.getId();
-  }
-
-  @Override
-  public String getDAGName() {
-    // TODO NEWTEZ Change to some form of the DAG name, for now using dagId as
-    // the unique identifier.
-    return taskAttemptID.getTaskID().getVertexID().getDAGId().toString();
-  }
-
-  @Override
-  public String getTaskVertexName() {
-    return taskVertexName;
-  }
-
-
-  @Override
-  public TezCounters getCounters() {
-    return counters;
-  }
-
-  @Override
-  public String[] getWorkDirs() {
-    return Arrays.copyOf(workDirs, workDirs.length);
-  }
-
-  @Override
-  public String getUniqueIdentifier() {
-    return uniqueIdentifier;
-  }
-
-  @Override
-  public ByteBuffer getServiceConsumerMetaData(String serviceName) {
-    return (ByteBuffer) serviceConsumerMetadata.get(serviceName)
-        .asReadOnlyBuffer().rewind();
-  }
-
-  @Override
-  public ByteBuffer getServiceProviderMetaData(String serviceName) {
-    return AuxiliaryServiceHelper.getServiceDataFromEnv(
-        serviceName, System.getenv());
-  }
-
-  protected void signalFatalError(Throwable t, String message,
-      EventMetaData sourceInfo) {
-    runtimeTask.setFatalError(t, message);
-    String diagnostics;
-    if (t != null && message != null) {
-      diagnostics = "exceptionThrown=" + StringUtils.stringifyException(t)
-          + ", errorMessage=" + message;
-    } else if (t == null && message == null) {
-      diagnostics = "Unknown error";
-    } else {
-      diagnostics = t != null ?
-          "exceptionThrown=" + StringUtils.stringifyException(t)
-          : " errorMessage=" + message;
-    }
-    tezUmbilical.signalFatalError(taskAttemptID, diagnostics, sourceInfo);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezUmbilical.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezUmbilical.java b/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezUmbilical.java
deleted file mode 100644
index 925d87b..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezUmbilical.java
+++ /dev/null
@@ -1,36 +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.engine.api.impl;
-
-import java.io.IOException;
-import java.util.Collection;
-
-import org.apache.tez.dag.records.TezTaskAttemptID;
-
-public interface TezUmbilical {
-
-  public void addEvents(Collection<TezEvent> events);
-
-  public void signalFatalError(TezTaskAttemptID taskAttemptID,
-      String diagnostics,
-      EventMetaData sourceInfo);
-
-  public boolean canCommit(TezTaskAttemptID taskAttemptID) throws IOException;
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/engine/broadcast/input/BroadcastInputManager.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/broadcast/input/BroadcastInputManager.java b/tez-engine/src/main/java/org/apache/tez/engine/broadcast/input/BroadcastInputManager.java
deleted file mode 100644
index 1211598..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/broadcast/input/BroadcastInputManager.java
+++ /dev/null
@@ -1,138 +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.engine.broadcast.input;
-
-import java.io.IOException;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.LocalDirAllocator;
-import org.apache.tez.common.Constants;
-import org.apache.tez.common.TezJobConfig;
-import org.apache.tez.dag.api.TezUncheckedException;
-import org.apache.tez.engine.api.TezInputContext;
-import org.apache.tez.engine.common.InputAttemptIdentifier;
-import org.apache.tez.engine.common.task.local.output.TezTaskOutputFiles;
-import org.apache.tez.engine.shuffle.common.DiskFetchedInput;
-import org.apache.tez.engine.shuffle.common.FetchedInput;
-import org.apache.tez.engine.shuffle.common.FetchedInputAllocator;
-import org.apache.tez.engine.shuffle.common.FetchedInputCallback;
-import org.apache.tez.engine.shuffle.common.MemoryFetchedInput;
-
-public class BroadcastInputManager implements FetchedInputAllocator,
-    FetchedInputCallback {
-
-  private final Configuration conf;
-
-  private final TezTaskOutputFiles fileNameAllocator;
-  private final LocalDirAllocator localDirAllocator;
-
-  // Configuration parameters
-  private final long memoryLimit;
-  private final long maxSingleShuffleLimit;
-
-  private long usedMemory = 0;
-
-  public BroadcastInputManager(TezInputContext inputContext, Configuration conf) {
-    this.conf = conf;
-
-    this.fileNameAllocator = new TezTaskOutputFiles(conf,
-        inputContext.getUniqueIdentifier());
-    this.localDirAllocator = new LocalDirAllocator(TezJobConfig.LOCAL_DIRS);
-
-    // Setup configuration
-    final float maxInMemCopyUse = conf.getFloat(
-        TezJobConfig.TEZ_ENGINE_SHUFFLE_INPUT_BUFFER_PERCENT,
-        TezJobConfig.DEFAULT_TEZ_ENGINE_SHUFFLE_INPUT_BUFFER_PERCENT);
-    if (maxInMemCopyUse > 1.0 || maxInMemCopyUse < 0.0) {
-      throw new IllegalArgumentException("Invalid value for "
-          + TezJobConfig.TEZ_ENGINE_SHUFFLE_INPUT_BUFFER_PERCENT + ": "
-          + maxInMemCopyUse);
-    }
-
-    // Allow unit tests to fix Runtime memory
-    this.memoryLimit = (long) (conf.getLong(Constants.TEZ_ENGINE_TASK_MEMORY,
-        Math.min(Runtime.getRuntime().maxMemory(), Integer.MAX_VALUE)) * maxInMemCopyUse);
-
-    final float singleShuffleMemoryLimitPercent = conf.getFloat(
-        TezJobConfig.TEZ_ENGINE_SHUFFLE_MEMORY_LIMIT_PERCENT,
-        TezJobConfig.DEFAULT_TEZ_ENGINE_SHUFFLE_MEMORY_LIMIT_PERCENT);
-    if (singleShuffleMemoryLimitPercent <= 0.0f
-        || singleShuffleMemoryLimitPercent > 1.0f) {
-      throw new IllegalArgumentException("Invalid value for "
-          + TezJobConfig.TEZ_ENGINE_SHUFFLE_MEMORY_LIMIT_PERCENT + ": "
-          + singleShuffleMemoryLimitPercent);
-    }
-
-    this.maxSingleShuffleLimit = (long) (memoryLimit * singleShuffleMemoryLimitPercent);
-  }
-
-  @Override
-  public synchronized FetchedInput allocate(long size,
-      InputAttemptIdentifier inputAttemptIdentifier) throws IOException {
-    if (size > maxSingleShuffleLimit
-        || this.usedMemory + size > this.memoryLimit) {
-      return new DiskFetchedInput(size, inputAttemptIdentifier, this, conf,
-          localDirAllocator, fileNameAllocator);
-    } else {
-      this.usedMemory += size;
-      return new MemoryFetchedInput(size, inputAttemptIdentifier, this);
-    }
-  }
-
-  @Override
-  public void fetchComplete(FetchedInput fetchedInput) {
-    switch (fetchedInput.getType()) {
-    // Not tracking anything here.
-    case DISK:
-    case MEMORY:
-      break;
-    default:
-      throw new TezUncheckedException("InputType: " + fetchedInput.getType()
-          + " not expected for Broadcast fetch");
-    }
-  }
-
-  @Override
-  public void fetchFailed(FetchedInput fetchedInput) {
-    cleanup(fetchedInput);
-  }
-
-  @Override
-  public void freeResources(FetchedInput fetchedInput) {
-    cleanup(fetchedInput);
-  }
-
-  private void cleanup(FetchedInput fetchedInput) {
-    switch (fetchedInput.getType()) {
-    case DISK:
-      break;
-    case MEMORY:
-      unreserve(fetchedInput.getSize());
-      break;
-    default:
-      throw new TezUncheckedException("InputType: " + fetchedInput.getType()
-          + " not expected for Broadcast fetch");
-    }
-  }
-
-  private synchronized void unreserve(long size) {
-    this.usedMemory -= size;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/engine/broadcast/input/BroadcastKVReader.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/broadcast/input/BroadcastKVReader.java b/tez-engine/src/main/java/org/apache/tez/engine/broadcast/input/BroadcastKVReader.java
deleted file mode 100644
index 2c53e75..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/broadcast/input/BroadcastKVReader.java
+++ /dev/null
@@ -1,225 +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.engine.broadcast.input;
-
-import java.io.IOException;
-import java.util.Iterator;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.io.DataInputBuffer;
-import org.apache.hadoop.io.compress.CompressionCodec;
-import org.apache.hadoop.io.compress.DefaultCodec;
-import org.apache.hadoop.io.serializer.Deserializer;
-import org.apache.hadoop.io.serializer.SerializationFactory;
-import org.apache.hadoop.util.ReflectionUtils;
-import org.apache.tez.engine.api.KVReader;
-import org.apache.tez.engine.common.ConfigUtils;
-import org.apache.tez.engine.common.shuffle.impl.InMemoryReader;
-import org.apache.tez.engine.common.sort.impl.IFile;
-import org.apache.tez.engine.shuffle.common.FetchedInput;
-import org.apache.tez.engine.shuffle.common.FetchedInput.Type;
-import org.apache.tez.engine.shuffle.common.MemoryFetchedInput;
-
-public class BroadcastKVReader<K, V> implements KVReader {
-
-  private static final Log LOG = LogFactory.getLog(BroadcastKVReader.class);
-  
-  private final BroadcastShuffleManager shuffleManager;
-  private final Configuration conf;
-  private final CompressionCodec codec;
-  
-  private final Class<K> keyClass;
-  private final Class<V> valClass;
-  private final Deserializer<K> keyDeserializer;
-  private final Deserializer<V> valDeserializer;
-  private final DataInputBuffer keyIn;
-  private final DataInputBuffer valIn;
-
-  private final SimpleValueIterator valueIterator;
-  private final SimpleIterable valueIterable;
-  
-  private K key;
-  private V value;
-  
-  private FetchedInput currentFetchedInput;
-  private IFile.Reader currentReader;
-  
-  
-  public BroadcastKVReader(BroadcastShuffleManager shuffleManager,
-      Configuration conf) {
-    this.shuffleManager = shuffleManager;
-    this.conf = conf;
-
-    if (ConfigUtils.isIntermediateInputCompressed(this.conf)) {
-      Class<? extends CompressionCodec> codecClass = ConfigUtils
-          .getIntermediateInputCompressorClass(conf, DefaultCodec.class);
-      codec = ReflectionUtils.newInstance(codecClass, conf);
-    } else {
-      codec = null;
-    }
-
-    this.keyClass = ConfigUtils.getIntermediateInputKeyClass(conf);
-    this.valClass = ConfigUtils.getIntermediateInputKeyClass(conf);
-
-    this.keyIn = new DataInputBuffer();
-    this.valIn = new DataInputBuffer();
-
-    SerializationFactory serializationFactory = new SerializationFactory(conf);
-
-    this.keyDeserializer = serializationFactory.getDeserializer(keyClass); 
-    this.valDeserializer = serializationFactory.getDeserializer(valClass);
-    
-    this.valueIterator = new SimpleValueIterator();
-    this.valueIterable = new SimpleIterable(this.valueIterator);
-  }
-
-  // TODO NEWTEZ Maybe add an interface to check whether next will block.
-  
-  /**
-   * Moves to the next key/values(s) pair
-   * 
-   * @return true if another key/value(s) pair exists, false if there are no
-   *         more.
-   * @throws IOException
-   *           if an error occurs
-   */
-  @Override  
-  public boolean next() throws IOException {
-    if (readNextFromCurrentReader()) {
-      return true;
-    } else {
-      boolean nextInputExists = moveToNextInput();
-      while (nextInputExists) {
-        if(readNextFromCurrentReader()) {
-          return true;
-        }
-        nextInputExists = moveToNextInput();
-      }
-      return false;
-    }
-  }
-
-  @SuppressWarnings("unchecked")
-  @Override
-  public KVRecord getCurrentKV() throws IOException {
-    this.valueIterator.setValue(value);
-    return new KVRecord((Object)key, (Iterable<Object>)this.valueIterable);
-  }
-
-  /**
-   * Tries reading the next key and value from the current reader.
-   * @return true if the current reader has more records
-   * @throws IOException
-   */
-  private boolean readNextFromCurrentReader() throws IOException {
-    // Initial reader.
-    if (this.currentReader == null) {
-      return false;
-    } else {
-      boolean hasMore = this.currentReader.nextRawKey(keyIn);
-      if (hasMore) {
-        this.currentReader.nextRawValue(valIn);
-        this.key = keyDeserializer.deserialize(this.key);
-        this.value = valDeserializer.deserialize(this.value);
-        return true;
-      }
-      return false;
-    }
-  }
-  
-  /**
-   * Moves to the next available input. This method may block if the input is not ready yet.
-   * Also takes care of closing the previous input.
-   * 
-   * @return true if the next input exists, false otherwise
-   * @throws IOException
-   * @throws InterruptedException
-   */
-  private boolean moveToNextInput() throws IOException {
-    if (currentReader != null) { // Close the current reader.
-      currentReader.close();
-      currentFetchedInput.free();
-    }
-    try {
-      currentFetchedInput = shuffleManager.getNextInput();
-    } catch (InterruptedException e) {
-      LOG.warn("Interrupted while waiting for next available input", e);
-      throw new IOException(e);
-    }
-    if (currentFetchedInput == null) {
-      return false; // No more inputs
-    } else {
-      currentReader = openIFileReader(currentFetchedInput);
-      return true;
-    }
-  }
-
-  public IFile.Reader openIFileReader(FetchedInput fetchedInput)
-      throws IOException {
-    if (fetchedInput.getType() == Type.MEMORY) {
-      MemoryFetchedInput mfi = (MemoryFetchedInput) fetchedInput;
-
-      return new InMemoryReader(null, mfi.getInputAttemptIdentifier(),
-          mfi.getBytes(), 0, (int) mfi.getSize());
-    } else {
-      return new IFile.Reader(conf, fetchedInput.getInputStream(),
-          fetchedInput.getSize(), codec, null);
-    }
-  }
-
-  
-  
-  // TODO NEWTEZ Move this into a common class. Also used in MRInput
-  private class SimpleValueIterator implements Iterator<V> {
-
-    private V value;
-
-    public void setValue(V value) {
-      this.value = value;
-    }
-
-    public boolean hasNext() {
-      return value != null;
-    }
-
-    public V next() {
-      V value = this.value;
-      this.value = null;
-      return value;
-    }
-
-    public void remove() {
-      throw new UnsupportedOperationException();
-    }
-  }
-
-  private class SimpleIterable implements Iterable<V> {
-    private final Iterator<V> iterator;
-    public SimpleIterable(Iterator<V> iterator) {
-      this.iterator = iterator;
-    }
-
-    @Override
-    public Iterator<V> iterator() {
-      return iterator;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/engine/broadcast/input/BroadcastShuffleInputEventHandler.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/broadcast/input/BroadcastShuffleInputEventHandler.java b/tez-engine/src/main/java/org/apache/tez/engine/broadcast/input/BroadcastShuffleInputEventHandler.java
deleted file mode 100644
index e89e892..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/broadcast/input/BroadcastShuffleInputEventHandler.java
+++ /dev/null
@@ -1,88 +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.engine.broadcast.input;
-
-import java.util.List;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.tez.dag.api.TezUncheckedException;
-import org.apache.tez.engine.api.Event;
-import org.apache.tez.engine.api.TezInputContext;
-import org.apache.tez.engine.api.events.DataMovementEvent;
-import org.apache.tez.engine.api.events.InputFailedEvent;
-import org.apache.tez.engine.common.InputAttemptIdentifier;
-import org.apache.tez.engine.common.shuffle.impl.ShuffleInputEventHandler;
-import org.apache.tez.engine.common.shuffle.newimpl.ShuffleUserPayloads.DataMovementEventPayloadProto;
-
-import com.google.common.base.Preconditions;
-import com.google.protobuf.InvalidProtocolBufferException;
-
-public class BroadcastShuffleInputEventHandler {
-
-  private static final Log LOG = LogFactory.getLog(ShuffleInputEventHandler.class);
-  
-  private final BroadcastShuffleManager shuffleManager;
-  
-  public BroadcastShuffleInputEventHandler(TezInputContext inputContext, BroadcastShuffleManager shuffleManager) {
-    this.shuffleManager = shuffleManager;
-  }
-
-  public void handleEvents(List<Event> events) {
-    for (Event event : events) {
-      handleEvent(event);
-    }
-  }
-  
-  private void handleEvent(Event event) {
-    if (event instanceof DataMovementEvent) {
-      processDataMovementEvent((DataMovementEvent)event);
-    } else if (event instanceof InputFailedEvent) {
-      processInputFailedEvent((InputFailedEvent)event);
-    } else {
-      throw new TezUncheckedException("Unexpected event type: " + event.getClass().getName());
-    }
-  }
-  
-  
-  private void processDataMovementEvent(DataMovementEvent dme) {
-    Preconditions.checkArgument(dme.getSourceIndex() == 0,
-        "Unexpected srcIndex: " + dme.getSourceIndex()
-            + " on DataMovementEvent. Can only be 0");
-    DataMovementEventPayloadProto shufflePayload;
-    try {
-      shufflePayload = DataMovementEventPayloadProto.parseFrom(dme.getUserPayload());
-    } catch (InvalidProtocolBufferException e) {
-      throw new TezUncheckedException("Unable to parse DataMovementEvent payload", e);
-    }
-    if (shufflePayload.getOutputGenerated()) {
-      InputAttemptIdentifier srcAttemptIdentifier = new InputAttemptIdentifier(dme.getTargetIndex(), dme.getVersion(), shufflePayload.getPathComponent());
-      shuffleManager.addKnownInput(shufflePayload.getHost(), shufflePayload.getPort(), srcAttemptIdentifier, 0);
-    } else {
-      shuffleManager.addCompletedInputWithNoData(new InputAttemptIdentifier(dme.getTargetIndex(), dme.getVersion()));
-    }
-  }
-  
-  private void processInputFailedEvent(InputFailedEvent ife) {
-    InputAttemptIdentifier srcAttemptIdentifier = new InputAttemptIdentifier(ife.getTargetIndex(), ife.getVersion());
-    shuffleManager.obsoleteKnownInput(srcAttemptIdentifier);
-  }
-}
-

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/engine/broadcast/input/BroadcastShuffleManager.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/broadcast/input/BroadcastShuffleManager.java b/tez-engine/src/main/java/org/apache/tez/engine/broadcast/input/BroadcastShuffleManager.java
deleted file mode 100644
index 7b205fa..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/broadcast/input/BroadcastShuffleManager.java
+++ /dev/null
@@ -1,489 +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.engine.broadcast.input;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.util.Collections;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Set;
-import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.FutureTask;
-import java.util.concurrent.LinkedBlockingQueue;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.locks.Condition;
-import java.util.concurrent.locks.ReentrantLock;
-
-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.io.compress.CodecPool;
-import org.apache.hadoop.io.compress.CompressionCodec;
-import org.apache.hadoop.io.compress.Decompressor;
-import org.apache.hadoop.io.compress.DefaultCodec;
-import org.apache.hadoop.util.ReflectionUtils;
-import org.apache.tez.common.TezJobConfig;
-import org.apache.tez.engine.api.Event;
-import org.apache.tez.engine.api.TezInputContext;
-import org.apache.tez.engine.api.events.InputReadErrorEvent;
-import org.apache.tez.engine.common.ConfigUtils;
-import org.apache.tez.engine.common.InputAttemptIdentifier;
-import org.apache.tez.engine.common.InputIdentifier;
-import org.apache.tez.engine.common.TezEngineUtils;
-import org.apache.tez.engine.common.shuffle.server.ShuffleHandler;
-import org.apache.tez.engine.shuffle.common.FetchResult;
-import org.apache.tez.engine.shuffle.common.FetchedInput;
-import org.apache.tez.engine.shuffle.common.Fetcher;
-import org.apache.tez.engine.shuffle.common.FetcherCallback;
-import org.apache.tez.engine.shuffle.common.InputHost;
-import org.apache.tez.engine.shuffle.common.ShuffleUtils;
-import org.apache.tez.engine.shuffle.common.Fetcher.FetcherBuilder;
-import org.apache.tez.engine.shuffle.common.FetchedInputAllocator;
-
-import com.google.common.collect.Lists;
-import com.google.common.util.concurrent.FutureCallback;
-import com.google.common.util.concurrent.Futures;
-import com.google.common.util.concurrent.ListenableFuture;
-import com.google.common.util.concurrent.ListeningExecutorService;
-import com.google.common.util.concurrent.MoreExecutors;
-import com.google.common.util.concurrent.ThreadFactoryBuilder;
-
-public class BroadcastShuffleManager implements FetcherCallback {
-
-  private static final Log LOG = LogFactory.getLog(BroadcastShuffleManager.class);
-  
-  private TezInputContext inputContext;
-  private int numInputs;
-  private Configuration conf;
-  
-  private final BroadcastShuffleInputEventHandler inputEventHandler;
-  private final FetchedInputAllocator inputManager;
-  
-  private final ExecutorService fetcherRawExecutor;
-  private final ListeningExecutorService fetcherExecutor;
-
-  private final BlockingQueue<FetchedInput> completedInputs;
-  private final Set<InputIdentifier> completedInputSet;
-  private final Set<InputIdentifier> pendingInputs;
-  private final ConcurrentMap<String, InputHost> knownSrcHosts;
-  private final Set<InputHost> pendingHosts;
-  private final Set<InputAttemptIdentifier> obsoletedInputs;
-  
-  private final AtomicInteger numCompletedInputs = new AtomicInteger(0);
-  
-  private final long startTime;
-  private long lastProgressTime;
-  
-  private FutureTask<Void> runShuffleFuture;
-  
-  // Required to be held when manipulating pendingHosts
-  private ReentrantLock lock = new ReentrantLock();
-  private Condition wakeLoop = lock.newCondition();
-  
-  private final int numFetchers;
-  private final AtomicInteger numRunningFetchers = new AtomicInteger(0);
-  
-  // Parameters required by Fetchers
-  private final SecretKey shuffleSecret;
-  private final int connectionTimeout;
-  private final int readTimeout;
-  private final CompressionCodec codec;
-  private final Decompressor decompressor;
-  
-  private final FetchFutureCallback fetchFutureCallback = new FetchFutureCallback();
-  
-  private volatile Throwable shuffleError;
-  
-  // TODO NEWTEZ Add counters.
-  
-  public BroadcastShuffleManager(TezInputContext inputContext, Configuration conf, int numInputs) throws IOException {
-    this.inputContext = inputContext;
-    this.conf = conf;
-    this.numInputs = numInputs;
-    
-    this.inputEventHandler = new BroadcastShuffleInputEventHandler(inputContext, this);
-    this.inputManager = new BroadcastInputManager(inputContext, conf);
-
-    pendingInputs = Collections.newSetFromMap(new ConcurrentHashMap<InputIdentifier, Boolean>(numInputs));
-    completedInputSet = Collections.newSetFromMap(new ConcurrentHashMap<InputIdentifier, Boolean>(numInputs));
-    completedInputs = new LinkedBlockingQueue<FetchedInput>(numInputs);
-    knownSrcHosts = new ConcurrentHashMap<String, InputHost>();
-    pendingHosts = Collections.newSetFromMap(new ConcurrentHashMap<InputHost, Boolean>());
-    obsoletedInputs = Collections.newSetFromMap(new ConcurrentHashMap<InputAttemptIdentifier, Boolean>());
-    
-    int maxConfiguredFetchers = 
-        conf.getInt(
-            TezJobConfig.TEZ_ENGINE_SHUFFLE_PARALLEL_COPIES, 
-            TezJobConfig.DEFAULT_TEZ_ENGINE_SHUFFLE_PARALLEL_COPIES);
-    
-    this.numFetchers = Math.min(maxConfiguredFetchers, numInputs);
-    
-    this.fetcherRawExecutor = Executors.newFixedThreadPool(numFetchers,
-        new ThreadFactoryBuilder().setDaemon(true).setNameFormat("Fetcher #%d")
-            .build());
-    this.fetcherExecutor = MoreExecutors.listeningDecorator(fetcherRawExecutor);
-    
-    this.startTime = System.currentTimeMillis();
-    this.lastProgressTime = startTime;
-    
-    this.shuffleSecret = ShuffleUtils
-        .getJobTokenSecretFromTokenBytes(inputContext
-            .getServiceConsumerMetaData(ShuffleHandler.MAPREDUCE_SHUFFLE_SERVICEID));
-    
-    this.connectionTimeout = conf.getInt(
-        TezJobConfig.TEZ_ENGINE_SHUFFLE_CONNECT_TIMEOUT,
-        TezJobConfig.DEFAULT_TEZ_ENGINE_SHUFFLE_STALLED_COPY_TIMEOUT);
-    this.readTimeout = conf.getInt(
-        TezJobConfig.TEZ_ENGINE_SHUFFLE_READ_TIMEOUT,
-        TezJobConfig.DEFAULT_TEZ_ENGINE_SHUFFLE_READ_TIMEOUT);
-    
-    if (ConfigUtils.isIntermediateInputCompressed(conf)) {
-      Class<? extends CompressionCodec> codecClass = ConfigUtils
-          .getIntermediateInputCompressorClass(conf, DefaultCodec.class);
-      codec = ReflectionUtils.newInstance(codecClass, conf);
-      decompressor = CodecPool.getDecompressor(codec);
-    } else {
-      codec = null;
-      decompressor = null;
-    }
-  }
-  
-  public void run() {
-    RunBroadcastShuffleCallable callable = new RunBroadcastShuffleCallable();
-    runShuffleFuture = new FutureTask<Void>(callable);
-    new Thread(runShuffleFuture, "ShuffleRunner");
-  }
-  
-  private class RunBroadcastShuffleCallable implements Callable<Void> {
-
-    @Override
-    public Void call() throws Exception {
-      while (numCompletedInputs.get() < numInputs) {
-        if (numRunningFetchers.get() >= numFetchers || pendingHosts.size() == 0) {
-          synchronized(lock) {
-            wakeLoop.await();
-          }
-          if (shuffleError != null) {
-            // InputContext has already been informed of a fatal error.
-            // Initiate shutdown.
-            break;
-          }
-          
-          if (numCompletedInputs.get() < numInputs) {
-            synchronized (lock) {
-              int numFetchersToRun = Math.min(pendingHosts.size(), numFetchers - numRunningFetchers.get());
-              int count = 0;
-              for (Iterator<InputHost> inputHostIter = pendingHosts.iterator() ; inputHostIter.hasNext() ; ) {
-                InputHost inputHost = inputHostIter.next();
-                inputHostIter.remove();
-                if (inputHost.getNumPendingInputs() > 0) {
-                  Fetcher fetcher = constructFetcherForHost(inputHost);
-                  numRunningFetchers.incrementAndGet();
-                  ListenableFuture<FetchResult> future = fetcherExecutor
-                      .submit(fetcher);
-                  Futures.addCallback(future, fetchFutureCallback);
-                  if (++count >= numFetchersToRun) {
-                    break;
-                  }
-                }
-              }
-            }
-          }
-        }
-      }
-      // TODO NEWTEZ Maybe clean up inputs.
-      if (!fetcherExecutor.isShutdown()) {
-        fetcherExecutor.shutdownNow();
-      }
-      return null;
-    }
-  }
-  
-  private Fetcher constructFetcherForHost(InputHost inputHost) {
-    FetcherBuilder fetcherBuilder = new FetcherBuilder(
-        BroadcastShuffleManager.this, inputManager,
-        inputContext.getApplicationId(), shuffleSecret, conf);
-    fetcherBuilder.setConnectionParameters(connectionTimeout, readTimeout);
-    fetcherBuilder.setCompressionParameters(codec, decompressor);
-
-    // Remove obsolete inputs from the list being given to the fetcher. Also
-    // remove from the obsolete list.
-    List<InputAttemptIdentifier> pendingInputsForHost = inputHost
-        .clearAndGetPendingInputs();
-    for (Iterator<InputAttemptIdentifier> inputIter = pendingInputsForHost
-        .iterator(); inputIter.hasNext();) {
-      InputAttemptIdentifier input = inputIter.next();
-      // Avoid adding attempts which have already completed.
-      if (completedInputSet.contains(input.getInputIdentifier())) {
-        inputIter.remove();
-      }
-      // Avoid adding attempts which have been marked as OBSOLETE 
-      if (obsoletedInputs.contains(input)) {
-        inputIter.remove();
-        obsoletedInputs.remove(input);
-      }
-    }
-    // TODO NEWTEZ Maybe limit the number of inputs being given to a single
-    // fetcher, especially in the case where #hosts < #fetchers
-    fetcherBuilder.assignWork(inputHost.getHost(), inputHost.getPort(), 0,
-        inputHost.clearAndGetPendingInputs());
-    return fetcherBuilder.build();
-  }
-  
-  /////////////////// Methods for InputEventHandler
-  
-  public void addKnownInput(String hostName, int port,
-      InputAttemptIdentifier srcAttemptIdentifier, int partition) {
-    InputHost host = knownSrcHosts.get(hostName);
-    if (host == null) {
-      host = new InputHost(hostName, port, inputContext.getApplicationId());
-      InputHost old = knownSrcHosts.putIfAbsent(hostName, host);
-      if (old != null) {
-        host = old;
-      }
-    }
-    host.addKnownInput(srcAttemptIdentifier);
-    synchronized(lock) {
-      pendingHosts.add(host);
-      wakeLoop.signal();
-    }
-  }
-
-  public void addCompletedInputWithNoData(
-      InputAttemptIdentifier srcAttemptIdentifier) {
-    InputIdentifier inputIdentifier = srcAttemptIdentifier.getInputIdentifier();
-    LOG.info("No input data exists for SrcTask: " + inputIdentifier + ". Marking as complete.");
-    if (pendingInputs.remove(inputIdentifier)) {
-      completedInputSet.add(inputIdentifier);
-      completedInputs.add(new NullFetchedInput(srcAttemptIdentifier));
-      numCompletedInputs.incrementAndGet();
-    }
-
-    // Awake the loop to check for termination.
-    synchronized (lock) {
-      wakeLoop.signal();
-    } 
-  }
-
-  public synchronized void obsoleteKnownInput(InputAttemptIdentifier srcAttemptIdentifier) {
-    obsoletedInputs.add(srcAttemptIdentifier);
-    // TODO NEWTEZ Maybe inform the fetcher about this. For now, this is used during the initial fetch list construction.
-  }
-  
-  
-  public void handleEvents(List<Event> events) {
-    inputEventHandler.handleEvents(events);
-  }
-
-  /////////////////// End of Methods for InputEventHandler
-  /////////////////// Methods from FetcherCallbackHandler
-  
-  @Override
-  public void fetchSucceeded(String host,
-      InputAttemptIdentifier srcAttemptIdentifier, FetchedInput fetchedInput, long fetchedBytes,
-      long copyDuration) throws IOException {
-    InputIdentifier inputIdentifier = srcAttemptIdentifier.getInputIdentifier();
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Complete fetch for attempt: " + srcAttemptIdentifier + " to " + fetchedInput.getType());
-    }
-    
-    // Count irrespective of whether this is a copy of an already fetched input
-    synchronized(lock) {
-      lastProgressTime = System.currentTimeMillis();
-    }
-    
-    boolean committed = false;
-    if (!completedInputSet.contains(inputIdentifier)) {
-      synchronized (completedInputSet) {
-        if (!completedInputSet.contains(inputIdentifier)) {
-          fetchedInput.commit();
-          committed = true;
-          pendingInputs.remove(inputIdentifier);
-          completedInputSet.add(inputIdentifier);
-          completedInputs.add(fetchedInput);
-          numCompletedInputs.incrementAndGet();
-        }
-      }
-    }
-    if (!committed) {
-      fetchedInput.abort(); // If this fails, the fetcher may attempt another abort.
-    } else {
-      synchronized(lock) {
-        // Signal the wakeLoop to check for termination.
-        wakeLoop.signal();
-      }
-    }
-    // TODO NEWTEZ Maybe inform fetchers, in case they have an alternate attempt of the same task in their queue.
-  }
-
-  @Override
-  public void fetchFailed(String host,
-      InputAttemptIdentifier srcAttemptIdentifier, boolean connectFailed) {
-    // TODO NEWTEZ. Implement logic to report fetch failures after a threshold.
-    // For now, reporting immediately.
-    InputReadErrorEvent readError = new InputReadErrorEvent(
-        "Fetch failure while fetching from "
-            + TezEngineUtils.getTaskAttemptIdentifier(
-                inputContext.getSourceVertexName(),
-                srcAttemptIdentifier.getInputIdentifier().getSrcTaskIndex(),
-                srcAttemptIdentifier.getAttemptNumber()),
-        srcAttemptIdentifier.getInputIdentifier().getSrcTaskIndex(),
-        srcAttemptIdentifier.getAttemptNumber());
-    
-    List<Event> failedEvents = Lists.newArrayListWithCapacity(1);
-    failedEvents.add(readError);
-    inputContext.sendEvents(failedEvents);
-  }
-  /////////////////// End of Methods from FetcherCallbackHandler
-
-  public void shutdown() throws InterruptedException {
-    if (this.fetcherExecutor != null && !this.fetcherExecutor.isShutdown()) {
-      this.fetcherExecutor.shutdown();
-      this.fetcherExecutor.awaitTermination(2000l, TimeUnit.MILLISECONDS);
-      if (!this.fetcherExecutor.isShutdown()) {
-        this.fetcherExecutor.shutdownNow();
-      }
-    }
-  }
-  
-  /////////////////// Methods for walking the available inputs
-  
-  /**
-   * @return true if there is another input ready for consumption.
-   */
-  public boolean newInputAvailable() {
-    FetchedInput head = completedInputs.peek();
-    if (head == null || head instanceof NullFetchedInput) {
-      return false;
-    } else {
-      return true;
-    }
-  }
-
-  /**
-   * @return true if all of the required inputs have been fetched.
-   */
-  public boolean allInputsFetched() {
-    return numCompletedInputs.get() == numInputs;
-  }
-
-  /**
-   * @return the next available input, or null if there are no available inputs.
-   *         This method will block if there are currently no available inputs,
-   *         but more may become available.
-   */
-  public FetchedInput getNextInput() throws InterruptedException {
-    FetchedInput input = null;
-    do {
-      input = completedInputs.peek();
-      if (input == null) {
-        if (allInputsFetched()) {
-          break;
-        } else {
-          input = completedInputs.take(); // block
-        }
-      } else {
-        input = completedInputs.poll();
-      }
-    } while (input instanceof NullFetchedInput);
-    return input;
-  }
-
-  /////////////////// End of methods for walking the available inputs
-  
-  
-  /**
-   * Fake input that is added to the completed input list in case an input does not have any data.
-   *
-   */
-  private class NullFetchedInput extends FetchedInput {
-
-    public NullFetchedInput(InputAttemptIdentifier inputAttemptIdentifier) {
-      super(Type.MEMORY, -1, inputAttemptIdentifier, null);
-    }
-
-    @Override
-    public OutputStream getOutputStream() throws IOException {
-      throw new UnsupportedOperationException("Not supported for NullFetchedInput");
-    }
-
-    @Override
-    public InputStream getInputStream() throws IOException {
-      throw new UnsupportedOperationException("Not supported for NullFetchedInput");
-    }
-
-    @Override
-    public void commit() throws IOException {
-      throw new UnsupportedOperationException("Not supported for NullFetchedInput");
-    }
-
-    @Override
-    public void abort() throws IOException {
-      throw new UnsupportedOperationException("Not supported for NullFetchedInput");
-    }
-
-    @Override
-    public void free() {
-      throw new UnsupportedOperationException("Not supported for NullFetchedInput");
-    }
-  }
-  
-  
-  private class FetchFutureCallback implements FutureCallback<FetchResult> {
-
-    private void doBookKeepingForFetcherComplete() {
-      numRunningFetchers.decrementAndGet();
-      synchronized(lock) {
-        wakeLoop.signal();
-      }
-    }
-    
-    @Override
-    public void onSuccess(FetchResult result) {
-      Iterable<InputAttemptIdentifier> pendingInputs = result.getPendingInputs();
-      if (pendingInputs != null && pendingInputs.iterator().hasNext()) {
-        InputHost inputHost = knownSrcHosts.get(result.getHost());
-        assert inputHost != null;
-        for (InputAttemptIdentifier input : pendingInputs) {
-          inputHost.addKnownInput(input);
-        }
-        pendingHosts.add(inputHost);
-      }
-      doBookKeepingForFetcherComplete();
-    }
-
-    @Override
-    public void onFailure(Throwable t) {
-      LOG.error("Fetcher failed with error: " + t);
-      shuffleError = t;
-      inputContext.fatalError(t, "Fetched failed");
-      doBookKeepingForFetcherComplete();
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/engine/broadcast/output/FileBasedKVWriter.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/broadcast/output/FileBasedKVWriter.java b/tez-engine/src/main/java/org/apache/tez/engine/broadcast/output/FileBasedKVWriter.java
deleted file mode 100644
index 474d1cd..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/broadcast/output/FileBasedKVWriter.java
+++ /dev/null
@@ -1,125 +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.engine.broadcast.output;
-
-import java.io.IOException;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.LocalFileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.compress.CompressionCodec;
-import org.apache.hadoop.io.compress.DefaultCodec;
-import org.apache.hadoop.util.ReflectionUtils;
-import org.apache.tez.common.TezJobConfig;
-import org.apache.tez.common.TezUtils;
-import org.apache.tez.engine.api.KVWriter;
-import org.apache.tez.engine.api.TezOutputContext;
-import org.apache.tez.engine.common.ConfigUtils;
-import org.apache.tez.engine.common.TezEngineUtils;
-import org.apache.tez.engine.common.sort.impl.IFile;
-import org.apache.tez.engine.common.sort.impl.TezIndexRecord;
-import org.apache.tez.engine.common.sort.impl.TezSpillRecord;
-import org.apache.tez.engine.common.task.local.output.TezTaskOutput;
-
-public class FileBasedKVWriter implements KVWriter {
-
-  public static final int INDEX_RECORD_LENGTH = 24;
-
-  private final Configuration conf;
-  private int numRecords = 0;
-
-  @SuppressWarnings("rawtypes")
-  private Class keyClass;
-  @SuppressWarnings("rawtypes")
-  private Class valClass;
-  private CompressionCodec codec;
-  private FileSystem rfs;
-  private IFile.Writer writer;
-
-  private TezTaskOutput ouputFileManager;
-
-  // TODO NEWTEZ Define Counters
-  // Number of records
-  // Time waiting for a write to complete, if that's possible.
-  // Size of key-value pairs written.
-
-  public FileBasedKVWriter(TezOutputContext outputContext) throws IOException {
-    this.conf = TezUtils.createConfFromUserPayload(outputContext
-        .getUserPayload());
-    this.conf.setStrings(TezJobConfig.LOCAL_DIRS,
-        outputContext.getWorkDirs());
-
-    this.rfs = ((LocalFileSystem) FileSystem.getLocal(this.conf)).getRaw();
-
-    // Setup serialization
-    keyClass = ConfigUtils.getIntermediateOutputKeyClass(this.conf);
-    valClass = ConfigUtils.getIntermediateOutputValueClass(this.conf);
-
-    // Setup compression
-    if (ConfigUtils.shouldCompressIntermediateOutput(this.conf)) {
-      Class<? extends CompressionCodec> codecClass = ConfigUtils
-          .getIntermediateOutputCompressorClass(this.conf, DefaultCodec.class);
-      codec = ReflectionUtils.newInstance(codecClass, this.conf);
-    } else {
-      codec = null;
-    }
-
-    this.ouputFileManager = TezEngineUtils.instantiateTaskOutputManager(conf,
-        outputContext);
-
-    initWriter();
-  }
-
-  /**
-   * @return true if any output was generated. false otherwise
-   * @throws IOException
-   */
-  public boolean close() throws IOException {
-    this.writer.close();
-    TezIndexRecord rec = new TezIndexRecord(0, writer.getRawLength(),
-        writer.getCompressedLength());
-    TezSpillRecord sr = new TezSpillRecord(1);
-    sr.putIndex(rec, 0);
-
-    Path indexFile = ouputFileManager
-        .getOutputIndexFileForWrite(INDEX_RECORD_LENGTH);
-    sr.writeToFile(indexFile, conf);
-    return numRecords > 0;
-  }
-
-  @Override
-  public void write(Object key, Object value) throws IOException {
-    this.writer.append(key, value);
-    numRecords++;
-  }
-
-  public void initWriter() throws IOException {
-    Path outputFile = ouputFileManager.getOutputFileForWrite();
-
-    // TODO NEWTEZ Consider making the buffer size configurable. Also consider
-    // setting up an in-memory buffer which is occasionally flushed to disk so
-    // that the output does not block.
-
-    // TODO NEWTEZ maybe use appropriate counter
-    this.writer = new IFile.Writer(conf, rfs, outputFile, keyClass, valClass,
-        codec, null);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/engine/common/ConfigUtils.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/ConfigUtils.java b/tez-engine/src/main/java/org/apache/tez/engine/common/ConfigUtils.java
deleted file mode 100644
index f73adfd..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/ConfigUtils.java
+++ /dev/null
@@ -1,148 +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.engine.common;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.io.RawComparator;
-import org.apache.hadoop.io.WritableComparable;
-import org.apache.hadoop.io.WritableComparator;
-import org.apache.hadoop.io.compress.CompressionCodec;
-import org.apache.hadoop.io.compress.DefaultCodec;
-import org.apache.hadoop.util.ReflectionUtils;
-import org.apache.tez.common.TezJobConfig;
-
-@SuppressWarnings({"unchecked", "rawtypes"})
-public class ConfigUtils {
-
-  public static Class<? extends CompressionCodec> getIntermediateOutputCompressorClass(
-      Configuration conf, Class<DefaultCodec> defaultValue) {
-    Class<? extends CompressionCodec> codecClass = defaultValue;
-    String name = conf
-        .get(TezJobConfig.TEZ_ENGINE_INTERMEDIATE_OUTPUT_COMPRESS_CODEC);
-    if (name != null) {
-      try {
-        codecClass = conf.getClassByName(name).asSubclass(
-            CompressionCodec.class);
-      } catch (ClassNotFoundException e) {
-        throw new IllegalArgumentException("Compression codec " + name
-            + " was not found.", e);
-      }
-    }
-    return codecClass;
-  }
-  
-  public static Class<? extends CompressionCodec> getIntermediateInputCompressorClass(
-      Configuration conf, Class<DefaultCodec> defaultValue) {
-    Class<? extends CompressionCodec> codecClass = defaultValue;
-    String name = conf
-        .get(TezJobConfig.TEZ_ENGINE_INTERMEDIATE_INPUT_COMPRESS_CODEC);
-    if (name != null) {
-      try {
-        codecClass = conf.getClassByName(name).asSubclass(
-            CompressionCodec.class);
-      } catch (ClassNotFoundException e) {
-        throw new IllegalArgumentException("Compression codec " + name
-            + " was not found.", e);
-      }
-    }
-    return codecClass;
-  }
-
-
-  // TODO Move defaults over to a constants file.
-  
-  public static boolean shouldCompressIntermediateOutput(Configuration conf) {
-    return conf.getBoolean(
-        TezJobConfig.TEZ_ENGINE_INTERMEDIATE_OUTPUT_SHOULD_COMPRESS, false);
-  }
-
-  public static boolean isIntermediateInputCompressed(Configuration conf) {
-    return conf.getBoolean(
-        TezJobConfig.TEZ_ENGINE_INTERMEDIATE_INPUT_IS_COMPRESSED, false);
-  }
-
-  public static <V> Class<V> getIntermediateOutputValueClass(Configuration conf) {
-    Class<V> retv = (Class<V>) conf.getClass(
-        TezJobConfig.TEZ_ENGINE_INTERMEDIATE_OUTPUT_VALUE_CLASS, null,
-        Object.class);
-    return retv;
-  }
-  
-  public static <V> Class<V> getIntermediateInputValueClass(Configuration conf) {
-    Class<V> retv = (Class<V>) conf.getClass(
-        TezJobConfig.TEZ_ENGINE_INTERMEDIATE_INPUT_VALUE_CLASS, null,
-        Object.class);
-    return retv;
-  }
-
-  public static <K> Class<K> getIntermediateOutputKeyClass(Configuration conf) {
-    Class<K> retv = (Class<K>) conf.getClass(
-        TezJobConfig.TEZ_ENGINE_INTERMEDIATE_OUTPUT_KEY_CLASS, null,
-        Object.class);
-    return retv;
-  }
-
-  public static <K> Class<K> getIntermediateInputKeyClass(Configuration conf) {
-    Class<K> retv = (Class<K>) conf.getClass(
-        TezJobConfig.TEZ_ENGINE_INTERMEDIATE_INPUT_KEY_CLASS, null,
-        Object.class);
-    return retv;
-  }
-
-  public static <K> RawComparator<K> getIntermediateOutputKeyComparator(Configuration conf) {
-    Class<? extends RawComparator> theClass = conf.getClass(
-        TezJobConfig.TEZ_ENGINE_INTERMEDIATE_OUTPUT_KEY_COMPARATOR_CLASS, null,
-        RawComparator.class);
-    if (theClass != null)
-      return ReflectionUtils.newInstance(theClass, conf);
-    return WritableComparator.get(getIntermediateOutputKeyClass(conf).asSubclass(
-        WritableComparable.class));
-  }
-
-  public static <K> RawComparator<K> getIntermediateInputKeyComparator(Configuration conf) {
-    Class<? extends RawComparator> theClass = conf.getClass(
-        TezJobConfig.TEZ_ENGINE_INTERMEDIATE_INPUT_KEY_COMPARATOR_CLASS, null,
-        RawComparator.class);
-    if (theClass != null)
-      return ReflectionUtils.newInstance(theClass, conf);
-    return WritableComparator.get(getIntermediateInputKeyClass(conf).asSubclass(
-        WritableComparable.class));
-  }
-
-  
-  
-  // TODO Fix name
-  public static <V> RawComparator<V> getInputKeySecondaryGroupingComparator(
-      Configuration conf) {
-    Class<? extends RawComparator> theClass = conf
-        .getClass(
-            TezJobConfig.TEZ_ENGINE_INTERMEDIATE_INPUT_KEY_SECONDARY_COMPARATOR_CLASS,
-            null, RawComparator.class);
-    if (theClass == null) {
-      return getIntermediateInputKeyComparator(conf);
-    }
-
-    return ReflectionUtils.newInstance(theClass, conf);
-  }
-  
-  public static boolean useNewApi(Configuration conf) {
-    return conf.getBoolean("mapred.mapper.new-api", false);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/engine/common/InputAttemptIdentifier.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/InputAttemptIdentifier.java b/tez-engine/src/main/java/org/apache/tez/engine/common/InputAttemptIdentifier.java
deleted file mode 100644
index 076807e..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/InputAttemptIdentifier.java
+++ /dev/null
@@ -1,95 +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.engine.common;
-
-import org.apache.hadoop.classification.InterfaceAudience.Private;
-
-/**
- * Container for a task number and an attempt number for the task.
- */
-@Private
-public class InputAttemptIdentifier {
-
-  private final InputIdentifier inputIdentifier;
-  private final int attemptNumber;
-  private String pathComponent;
-  
-  public InputAttemptIdentifier(int taskIndex, int attemptNumber) {
-    this(new InputIdentifier(taskIndex), attemptNumber, null);
-  }
-  
-  public InputAttemptIdentifier(InputIdentifier inputIdentifier, int attemptNumber, String pathComponent) {
-    this.inputIdentifier = inputIdentifier;
-    this.attemptNumber = attemptNumber;
-    this.pathComponent = pathComponent;
-  }
-  
-  public InputAttemptIdentifier(int taskIndex, int attemptNumber, String pathComponent) {
-    this(new InputIdentifier(taskIndex), attemptNumber, pathComponent);
-  }
-
-  public InputIdentifier getInputIdentifier() {
-    return this.inputIdentifier;
-  }
-
-  public int getAttemptNumber() {
-    return attemptNumber;
-  }
-  
-  public String getPathComponent() {
-    return pathComponent;
-  }
-
-  // PathComponent does not need to be part of the hashCode and equals computation.
-  @Override
-  public int hashCode() {
-    final int prime = 31;
-    int result = 1;
-    result = prime * result + attemptNumber;
-    result = prime * result
-        + ((inputIdentifier == null) ? 0 : inputIdentifier.hashCode());
-    return result;
-  }
-
-  @Override
-  public boolean equals(Object obj) {
-    if (this == obj)
-      return true;
-    if (obj == null)
-      return false;
-    if (getClass() != obj.getClass())
-      return false;
-    InputAttemptIdentifier other = (InputAttemptIdentifier) obj;
-    if (attemptNumber != other.attemptNumber)
-      return false;
-    if (inputIdentifier == null) {
-      if (other.inputIdentifier != null)
-        return false;
-    } else if (!inputIdentifier.equals(other.inputIdentifier))
-      return false;
-    return true;
-  }
-
-  @Override
-  public String toString() {
-    return "InputAttemptIdentifier [inputIdentifier=" + inputIdentifier
-        + ", attemptNumber=" + attemptNumber + ", pathComponent="
-        + pathComponent + "]";
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/engine/common/InputIdentifier.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/InputIdentifier.java b/tez-engine/src/main/java/org/apache/tez/engine/common/InputIdentifier.java
deleted file mode 100644
index b694530..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/InputIdentifier.java
+++ /dev/null
@@ -1,56 +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.engine.common;
-
-public class InputIdentifier {
-
-  private final int srcTaskIndex;
-  
-  public InputIdentifier(int srcTaskIndex) {
-    this.srcTaskIndex = srcTaskIndex;
-  }
-
-  public int getSrcTaskIndex() {
-    return this.srcTaskIndex;
-  }
-
-  @Override
-  public int hashCode() {
-    return srcTaskIndex;
-  }
-
-  @Override
-  public boolean equals(Object obj) {
-    if (this == obj)
-      return true;
-    if (obj == null)
-      return false;
-    if (getClass() != obj.getClass())
-      return false;
-    InputIdentifier other = (InputIdentifier) obj;
-    if (srcTaskIndex != other.srcTaskIndex)
-      return false;
-    return true;
-  }
-
-  @Override
-  public String toString() {
-    return "InputIdentifier [srcTaskIndex=" + srcTaskIndex + "]";
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/engine/common/TezEngineUtils.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/TezEngineUtils.java b/tez-engine/src/main/java/org/apache/tez/engine/common/TezEngineUtils.java
deleted file mode 100644
index cc29e94..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/TezEngineUtils.java
+++ /dev/null
@@ -1,153 +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.engine.common;
-
-import java.io.IOException;
-import java.lang.reflect.Constructor;
-import java.lang.reflect.InvocationTargetException;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.tez.common.Constants;
-import org.apache.tez.common.TezJobConfig;
-import org.apache.tez.dag.api.TezUncheckedException;
-import org.apache.tez.engine.api.Partitioner;
-import org.apache.tez.engine.api.TezOutputContext;
-import org.apache.tez.engine.api.TezTaskContext;
-import org.apache.tez.engine.common.combine.Combiner;
-import org.apache.tez.engine.common.task.local.output.TezTaskOutput;
-import org.apache.tez.engine.common.task.local.output.TezTaskOutputFiles;
-
-public class TezEngineUtils {
-
-  private static final Log LOG = LogFactory
-      .getLog(TezEngineUtils.class);
-  
-  public static String getTaskIdentifier(String vertexName, int taskIndex) {
-    return String.format("%s_%06d", vertexName, taskIndex);
-  }
-
-  public static String getTaskAttemptIdentifier(int taskIndex,
-      int taskAttemptNumber) {
-    return String.format("%d_%d", taskIndex, taskAttemptNumber);
-  }
-
-  // TODO Maybe include a dag name in this.
-  public static String getTaskAttemptIdentifier(String vertexName,
-      int taskIndex, int taskAttemptNumber) {
-    return String.format("%s_%06d_%02d", vertexName, taskIndex,
-        taskAttemptNumber);
-  }
-
-  @SuppressWarnings("unchecked")
-  public static Combiner instantiateCombiner(Configuration conf, TezTaskContext taskContext) throws IOException {
-    Class<? extends Combiner> clazz;
-    String className = conf.get(TezJobConfig.TEZ_ENGINE_COMBINER_CLASS);
-    if (className == null) {
-      LOG.info("No combiner specified via " + TezJobConfig.TEZ_ENGINE_COMBINER_CLASS + ". Combiner will not be used");
-      return null;
-    }
-    LOG.info("Using Combiner class: " + className);
-    try {
-      clazz = (Class<? extends Combiner>) conf.getClassByName(className);
-    } catch (ClassNotFoundException e) {
-      throw new IOException("Unable to load combiner class: " + className);
-    }
-    
-    Combiner combiner = null;
-    
-      Constructor<? extends Combiner> ctor;
-      try {
-        ctor = clazz.getConstructor(TezTaskContext.class);
-        combiner = ctor.newInstance(taskContext);
-      } catch (SecurityException e) {
-        throw new IOException(e);
-      } catch (NoSuchMethodException e) {
-        throw new IOException(e);
-      } catch (IllegalArgumentException e) {
-        throw new IOException(e);
-      } catch (InstantiationException e) {
-        throw new IOException(e);
-      } catch (IllegalAccessException e) {
-        throw new IOException(e);
-      } catch (InvocationTargetException e) {
-        throw new IOException(e);
-      }
-      return combiner;
-  }
-  
-  @SuppressWarnings("unchecked")
-  public static Partitioner instantiatePartitioner(Configuration conf)
-      throws IOException {
-    Class<? extends Partitioner> clazz;
-    try {
-      clazz = (Class<? extends Partitioner>) conf
-          .getClassByName(conf.get(TezJobConfig.TEZ_ENGINE_PARTITIONER_CLASS));
-    } catch (ClassNotFoundException e) {
-      throw new IOException("Unable to find Partitioner class in config", e);
-    }
-
-    LOG.info("Using partitioner class: " + clazz.getName());
-
-    Partitioner partitioner = null;
-
-    try {
-      Constructor<? extends Partitioner> ctorWithConf = clazz
-          .getConstructor(Configuration.class);
-      partitioner = ctorWithConf.newInstance(conf);
-    } catch (SecurityException e) {
-      throw new IOException(e);
-    } catch (NoSuchMethodException e) {
-      try {
-        // Try a 0 argument constructor.
-        partitioner = clazz.newInstance();
-      } catch (InstantiationException e1) {
-        throw new IOException(e1);
-      } catch (IllegalAccessException e1) {
-        throw new IOException(e1);
-      }
-    } catch (IllegalArgumentException e) {
-      throw new IOException(e);
-    } catch (InstantiationException e) {
-      throw new IOException(e);
-    } catch (IllegalAccessException e) {
-      throw new IOException(e);
-    } catch (InvocationTargetException e) {
-      throw new IOException(e);
-    }
-    return partitioner;
-  }
-  
-  public static TezTaskOutput instantiateTaskOutputManager(Configuration conf, TezOutputContext outputContext) {
-    Class<?> clazz = conf.getClass(Constants.TEZ_ENGINE_TASK_OUTPUT_MANAGER,
-        TezTaskOutputFiles.class);
-    try {
-      Constructor<?> ctor = clazz.getConstructor(Configuration.class, String.class);
-      ctor.setAccessible(true);
-      TezTaskOutput instance = (TezTaskOutput) ctor.newInstance(conf, outputContext.getUniqueIdentifier());
-      return instance;
-    } catch (Exception e) {
-      throw new TezUncheckedException(
-          "Unable to instantiate configured TezOutputFileManager: "
-              + conf.get(Constants.TEZ_ENGINE_TASK_OUTPUT_MANAGER,
-                  TezTaskOutputFiles.class.getName()), e);
-    }
-  }
-}


[21/50] [abbrv] TEZ-443. Merge tez-dag-api and tez-engine-api into a single module - tez-api (part of TEZ-398). (sseth)

Posted by ss...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-common/src/main/java/org/apache/tez/common/Constants.java
----------------------------------------------------------------------
diff --git a/tez-common/src/main/java/org/apache/tez/common/Constants.java b/tez-common/src/main/java/org/apache/tez/common/Constants.java
deleted file mode 100644
index 8ea2909..0000000
--- a/tez-common/src/main/java/org/apache/tez/common/Constants.java
+++ /dev/null
@@ -1,57 +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.common;
-
-import org.apache.hadoop.classification.InterfaceAudience.Private;
-
-
-public class Constants {
-
-  // TODO NEWTEZ Check which of these constants are expecting specific pieces of information which are being removed - like taskAttemptId
-  
-  public static final String TEZ = "tez";
-
-  public static final String MAP_OUTPUT_FILENAME_STRING = "file.out";
-  public static final String MAP_OUTPUT_INDEX_SUFFIX_STRING = ".index";
-  public static final String REDUCE_INPUT_FILE_FORMAT_STRING = "%s/map_%d.out";
-
-  public static final int MAP_OUTPUT_INDEX_RECORD_LENGTH = 24;
-  public static String MERGED_OUTPUT_PREFIX = ".merged";
-  
-  // TODO NEWTEZ Remove this constant once the old code is removed.
-  public static final String TEZ_ENGINE_TASK_ATTEMPT_ID = 
-      "tez.engine.task.attempt.id";
-
-  public static final String TEZ_ENGINE_TASK_OUTPUT_FILENAME_STRING = "file.out";
-
-  public static final String TEZ_ENGINE_TASK_OUTPUT_INDEX_SUFFIX_STRING = ".index";
-
-  public static final String TEZ_ENGINE_TASK_INPUT_FILE_FORMAT_STRING = "%s/task_%d.out"; 
-
-  public static final String TEZ_ENGINE_JOB_CREDENTIALS = 
-      "tez.engine.job.credentials";
-  
-  @Private
-  public static final String TEZ_ENGINE_TASK_MEMORY =  "tez.engine.task.memory";
-  
-  public static final String TASK_OUTPUT_DIR = "output";
-  
-  public static final String TEZ_ENGINE_TASK_OUTPUT_MANAGER = 
-      "tez.engine.task.local.output.manager";
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-common/src/main/java/org/apache/tez/common/ContainerContext.java
----------------------------------------------------------------------
diff --git a/tez-common/src/main/java/org/apache/tez/common/ContainerContext.java b/tez-common/src/main/java/org/apache/tez/common/ContainerContext.java
deleted file mode 100644
index df92bdc..0000000
--- a/tez-common/src/main/java/org/apache/tez/common/ContainerContext.java
+++ /dev/null
@@ -1,64 +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.common;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.Writable;
-
-// TODO EVENTUALLY move this over to PB. Fix package/module.
-// TODO EVENTUALLY unit tests for functionality.
-public class ContainerContext implements Writable {
-
-  String containerIdentifier;
-  String pid;
-
-  public ContainerContext() {
-    containerIdentifier = "";
-    pid = "";
-  }
-
-  public ContainerContext(String containerIdStr, String pid) {
-    this.containerIdentifier = containerIdStr;
-    this.pid = pid;
-  }
-
-  public String getContainerIdentifier() {
-    return containerIdentifier;
-  }
-
-  public String getPid() {
-    return pid;
-  }
-
-  @Override
-  public void readFields(DataInput in) throws IOException {
-    this.containerIdentifier = Text.readString(in);
-    this.pid = Text.readString(in);
-  }
-
-  @Override
-  public void write(DataOutput out) throws IOException {
-    Text.writeString(out, containerIdentifier);
-    Text.writeString(out, pid);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-common/src/main/java/org/apache/tez/common/InputSpec.java
----------------------------------------------------------------------
diff --git a/tez-common/src/main/java/org/apache/tez/common/InputSpec.java b/tez-common/src/main/java/org/apache/tez/common/InputSpec.java
deleted file mode 100644
index e9faa26..0000000
--- a/tez-common/src/main/java/org/apache/tez/common/InputSpec.java
+++ /dev/null
@@ -1,85 +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.common;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.Writable;
-
-public class InputSpec implements Writable {
-
-  private String vertexName;
-  private int inDegree;
-  private String inputClassName;
-  
-  public InputSpec() {
-  }
-  
-  public InputSpec(String vertexName, int inDegree,
-      String inputClassName) {
-    this.vertexName = vertexName;
-    this.inDegree = inDegree;
-    this.inputClassName = inputClassName;
-  }
-  
-  /**
-   * @return the name of the input vertex.
-   */
-  public String getVertexName() {
-    return this.vertexName;
-  }
-  
-  /**
-   * @return the number of inputs for this task, which will be available from
-   *         the specified vertex.
-   */
-  public int getNumInputs() {
-    return this.inDegree;
-  }
-
-  /**
-   * @return Input class name
-   */
-  public String getInputClassName() {
-    return this.inputClassName;
-  }
-  
-  @Override
-  public void write(DataOutput out) throws IOException {
-    Text.writeString(out, vertexName);
-    out.writeInt(inDegree);
-    Text.writeString(out, inputClassName);
-  }
-
-  @Override
-  public void readFields(DataInput in) throws IOException {
-    vertexName = Text.readString(in);
-    this.inDegree = in.readInt();
-    inputClassName = Text.readString(in);
-  }
-  
-  @Override
-  public String toString() {
-    return "VertexName: " + vertexName + ", InDegree: " + inDegree
-        + ", InputClassName=" + inputClassName;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-common/src/main/java/org/apache/tez/common/OutputSpec.java
----------------------------------------------------------------------
diff --git a/tez-common/src/main/java/org/apache/tez/common/OutputSpec.java b/tez-common/src/main/java/org/apache/tez/common/OutputSpec.java
deleted file mode 100644
index 52ec5d8..0000000
--- a/tez-common/src/main/java/org/apache/tez/common/OutputSpec.java
+++ /dev/null
@@ -1,84 +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.common;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.Writable;
-
-public class OutputSpec implements Writable {
-
-  private String vertexName;
-  private int outDegree;
-  private String outputClassName;
-
-  public OutputSpec() {
-  }
-
-  public OutputSpec(String vertexName, int outDegree,
-      String outputClassName) {
-    this.vertexName = vertexName;
-    this.outDegree = outDegree;
-    this.outputClassName = outputClassName;
-  }
-
-  /**
-   * @return the name of the output vertex.
-   */
-  public String getVertexName() {
-    return this.vertexName;
-  }
-
-  /**
-   * @return the number of outputs to be generated by this task.
-   */
-  public int getNumOutputs() {
-    return this.outDegree;
-  }
-
-  /**
-   * @return Output class name
-   */
-  public String getOutputClassName() {
-    return this.outputClassName;
-  }
-
-  @Override
-  public void write(DataOutput out) throws IOException {
-    Text.writeString(out, vertexName);
-    out.writeInt(outDegree);
-    Text.writeString(out, outputClassName);
-  }
-
-  @Override
-  public void readFields(DataInput in) throws IOException {
-    vertexName = Text.readString(in);
-    this.outDegree = in.readInt();
-    outputClassName = Text.readString(in);
-  }
-  
-  @Override
-  public String toString() {
-    return "VertexName: " + vertexName + ", OutDegree: " + outDegree
-        + ", OutputClassName=" + outputClassName;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-common/src/main/java/org/apache/tez/common/TezJobConfig.java
----------------------------------------------------------------------
diff --git a/tez-common/src/main/java/org/apache/tez/common/TezJobConfig.java b/tez-common/src/main/java/org/apache/tez/common/TezJobConfig.java
deleted file mode 100644
index 7c4540c..0000000
--- a/tez-common/src/main/java/org/apache/tez/common/TezJobConfig.java
+++ /dev/null
@@ -1,314 +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.common;
-
-import org.apache.hadoop.classification.InterfaceAudience.Private;
-import org.apache.hadoop.classification.InterfaceStability.Evolving;
-
-
-/**
- * Meant for user configurable job properties. For others look at {@link Constants}
- *
- */
-
-// TODO EVENTUALLY A description for each property.
-@Private
-@Evolving
-public class TezJobConfig {
-
-
-
-
-  /** The number of milliseconds between progress reports. */
-  public static final int PROGRESS_INTERVAL = 3000;
-
-  public static final long DEFAULT_COMBINE_RECORDS_BEFORE_PROGRESS = 10000;
-
-  /**
-   * Configuration key to enable/disable IFile readahead.
-   */
-  public static final String TEZ_ENGINE_IFILE_READAHEAD =
-      "tez.engine.ifile.readahead";
-  public static final boolean DEFAULT_TEZ_ENGINE_IFILE_READAHEAD = true;
-
-  /**
-   * Configuration key to set the IFile readahead length in bytes.
-   */
-  public static final String TEZ_ENGINE_IFILE_READAHEAD_BYTES =
-      "tez.engine.ifile.readahead.bytes";
-  public static final int DEFAULT_TEZ_ENGINE_IFILE_READAHEAD_BYTES =
-      4 * 1024 * 1024;
-
-  /**
-   * 
-   */
-  public static final String RECORDS_BEFORE_PROGRESS = 
-      "tez.task.merge.progress.records";
-  public static final long DEFAULT_RECORDS_BEFORE_PROGRESS = 10000; 
-
-  /**
-   * List of directories avialble to the engine. 
-   */
-  public static final String LOCAL_DIRS = "tez.engine.local.dirs";
-  public static final String DEFAULT_LOCAL_DIRS = "/tmp";
-
-  /**
-   * One local dir for the speicfic job.
-   */
-  public static final String JOB_LOCAL_DIR = "tez.engine.job.local.dir";
-  
-  /**
-   * The directory which contains the localized files for this task.
-   */
-  @Private
-  public static final String TASK_LOCAL_RESOURCE_DIR = "tez.engine.task-local-resource.dir";
-  public static final String DEFAULT_TASK_LOCAL_RESOURCE_DIR = "/tmp";
-  
-  public static final String TEZ_TASK_WORKING_DIR = "tez.engine.task.working.dir";
-
-  /**
-   * 
-   */
-  public static final String TEZ_ENGINE_IO_SORT_FACTOR = 
-      "tez.engine.io.sort.factor";
-  public static final int DEFAULT_TEZ_ENGINE_IO_SORT_FACTOR = 100;
-
-  /**
-   * 
-   */
-  public static final String TEZ_ENGINE_SORT_SPILL_PERCENT = 
-      "tez.engine.sort.spill.percent";
-  public static float DEFAULT_TEZ_ENGINE_SORT_SPILL_PERCENT = 0.8f; 
-
-  /**
-   * 
-   */
-  public static final String TEZ_ENGINE_IO_SORT_MB = "tez.engine.io.sort.mb";
-  public static final int DEFAULT_TEZ_ENGINE_IO_SORT_MB = 100;
-
-  /**
-   * 
-   */
-  public static final String TEZ_ENGINE_INDEX_CACHE_MEMORY_LIMIT_BYTES = 
-      "tez.engine.index.cache.memory.limit.bytes";
-  public static final int DEFAULT_TEZ_ENGINE_INDEX_CACHE_MEMORY_LIMIT_BYTES = 
-      1024 * 1024;
-
-  /**
-   * 
-   */
-  public static final String TEZ_ENGINE_COMBINE_MIN_SPILLS = 
-      "tez.engine.combine.min.spills";
-  public static final int  DEFAULT_TEZ_ENGINE_COMBINE_MIN_SPILLS = 3;
-  
-  /**
-   * 
-   */
-  public static final String TEZ_ENGINE_SORT_THREADS = 
-	      "tez.engine.sort.threads";
-  public static final int DEFAULT_TEZ_ENGINE_SORT_THREADS = 1;
-
-  /**
-   * Specifies a partitioner class, which is used in Tez engine components like OnFileSortedOutput
-   */
-  public static final String TEZ_ENGINE_PARTITIONER_CLASS = "tez.engine.partitioner.class";
-  
-  /**
-   * Specifies a combiner class (primarily for Shuffle)
-   */
-  public static final String TEZ_ENGINE_COMBINER_CLASS = "tez.engine.combiner.class";
-  
-  public static final String TEZ_ENGINE_NUM_EXPECTED_PARTITIONS = "tez.engine.num.expected.partitions";
-  
-  /**
-   * 
-   */
-  public static final String COUNTERS_MAX_KEY = "tez.engine.job.counters.max";
-  public static final int COUNTERS_MAX_DEFAULT = 120;
-
-  /**
-   * 
-   */
-  public static final String COUNTER_GROUP_NAME_MAX_KEY = "tez.engine.job.counters.group.name.max";
-  public static final int COUNTER_GROUP_NAME_MAX_DEFAULT = 128;
-
-  /**
-   * 
-   */
-  public static final String COUNTER_NAME_MAX_KEY = "tez.engine.job.counters.counter.name.max";
-  public static final int COUNTER_NAME_MAX_DEFAULT = 64;
-
-  /**
-   * 
-   */
-  public static final String COUNTER_GROUPS_MAX_KEY = "tez.engine.job.counters.groups.max";
-  public static final int COUNTER_GROUPS_MAX_DEFAULT = 50;
-
-  
-  /**
-   * Temporary interface for MR only (not chained Tez) to indicate whether
-   * in-memory shuffle should be used.
-   */
-  @Private
-  public static final String TEZ_ENGINE_SHUFFLE_USE_IN_MEMORY =
-      "tez.engine.shuffle.use.in-memory";
-  public static final boolean DEFAULT_TEZ_ENGINE_SHUFFLE_USE_IN_MEMORY = false;
-
-  // TODO NEWTEZ Remove these config parameters. Will be part of an event.
-  @Private
-  public static final String TEZ_ENGINE_SHUFFLE_PARTITION_RANGE = 
-      "tez.engine.shuffle.partition-range";
-  public static int TEZ_ENGINE_SHUFFLE_PARTITION_RANGE_DEFAULT = 1;
-
-  /**
-   * 
-   */
-  public static final String TEZ_ENGINE_SHUFFLE_PARALLEL_COPIES = 
-      "tez.engine.shuffle.parallel.copies";
-  public static final int DEFAULT_TEZ_ENGINE_SHUFFLE_PARALLEL_COPIES = 20;
-
-  /**
-   * TODO Is this user configurable.
-   */
-  public static final String TEZ_ENGINE_METRICS_SESSION_ID = 
-      "tez.engine.metrics.session.id";
-  public static final String DEFAULT_TEZ_ENGINE_METRICS_SESSION_ID = "";
-
-  /**
-   * 
-   */
-  public static final String TEZ_ENGINE_SHUFFLE_FETCH_FAILURES = 
-      "tez.engine.shuffle.fetch.failures.limit";
-  public final static int DEFAULT_TEZ_ENGINE_SHUFFLE_FETCH_FAILURES_LIMIT = 10;
-
-  /**
-   * 
-   */
-  public static final String TEZ_ENGINE_SHUFFLE_NOTIFY_READERROR = 
-      "tez.engine.shuffle.notify.readerror";
-  public static final boolean DEFAULT_TEZ_ENGINE_SHUFFLE_NOTIFY_READERROR = true;
-
-  /**
-   * 
-   */
-  public static final String TEZ_ENGINE_SHUFFLE_CONNECT_TIMEOUT = 
-      "tez.engine.shuffle.connect.timeout";
-  public static final int DEFAULT_TEZ_ENGINE_SHUFFLE_STALLED_COPY_TIMEOUT = 
-      3 * 60 * 1000;
-
-  /**
-   * 
-   */
-  public static final String TEZ_ENGINE_SHUFFLE_READ_TIMEOUT = "tez.engine.shuffle.read.timeout";
-  public final static int DEFAULT_TEZ_ENGINE_SHUFFLE_READ_TIMEOUT = 
-      3 * 60 * 1000;
-
-  /**
-   * 
-   */
-  public static final String TEZ_ENGINE_SHUFFLE_ENABLE_SSL = 
-      "tez.engine.shuffle.ssl.enable";
-  public static final boolean DEFAULT_TEZ_ENGINE_SHUFFLE_ENABLE_SSL = false;
-
-  /**
-   * 
-   */
-  public static final String TEZ_ENGINE_SHUFFLE_INPUT_BUFFER_PERCENT = 
-      "tez.engine.shuffle.input.buffer.percent";
-  public static final float DEFAULT_TEZ_ENGINE_SHUFFLE_INPUT_BUFFER_PERCENT =
-      0.90f;
-
-  /**
-   * 
-   */
-  public static final String TEZ_ENGINE_SHUFFLE_MEMORY_LIMIT_PERCENT = 
-      "tez.engine.shuffle.memory.limit.percent";
-  public static final float DEFAULT_TEZ_ENGINE_SHUFFLE_MEMORY_LIMIT_PERCENT = 
-      0.25f;
-
-  /**
-   * 
-   */
-  public static final String TEZ_ENGINE_SHUFFLE_MERGE_PERCENT = 
-      "tez.engine.shuffle.merge.percent";
-  public static final float DEFAULT_TEZ_ENGINE_SHUFFLE_MERGE_PERCENT = 0.90f;
-  
-  /**
-   * TODO TEZAM3 default value ?
-   */
-  public static final String TEZ_ENGINE_SHUFFLE_MEMTOMEM_SEGMENTS = 
-      "tez.engine.shuffle.memory-to-memory.segments";
-
-  /**
-   * 
-   */
-  public static final String TEZ_ENGINE_SHUFFLE_ENABLE_MEMTOMEM = 
-      "tez.engine.shuffle.memory-to-memory.enable";
-  public static final boolean DEFAULT_TEZ_ENGINE_SHUFFLE_ENABLE_MEMTOMEM = 
-      false;
-
-  /**
-   * 
-   */
-  public static final String TEZ_ENGINE_INPUT_BUFFER_PERCENT = 
-      "tez.engine.task.input.buffer.percent";
-  public static final float DEFAULT_TEZ_ENGINE_INPUT_BUFFER_PERCENT = 0.0f;
-
-  // TODO Rename. 
-  public static final String TEZ_ENGINE_GROUP_COMPARATOR_CLASS = 
-      "tez.engine.group.comparator.class";
-  
-  // TODO Better name.
-  public static final String TEZ_ENGINE_INTERNAL_SORTER_CLASS = 
-      "tez.engine.internal.sorter.class";
-  
-  public static final String TEZ_ENGINE_INTERMEDIATE_OUTPUT_KEY_COMPARATOR_CLASS = 
-      "tez.engine.intermediate-output.key.comparator.class";
-  public static final String TEZ_ENGINE_INTERMEDIATE_INPUT_KEY_COMPARATOR_CLASS = 
-      "tez.engine.intermediate-input.key.comparator.class";
-
-  public static final String TEZ_ENGINE_INTERMEDIATE_OUTPUT_KEY_CLASS = 
-      "tez.engine.intermediate-output.key.class";
-  public static final String TEZ_ENGINE_INTERMEDIATE_INPUT_KEY_CLASS = 
-      "tez.engine.intermediate-input.key.class";
-  
-  public static final String TEZ_ENGINE_INTERMEDIATE_OUTPUT_VALUE_CLASS = 
-      "tez.engine.intermediate-output.value.class";
-  public static final String TEZ_ENGINE_INTERMEDIATE_INPUT_VALUE_CLASS = 
-      "tez.engine.intermediate-input.value.class";
-  
-  public static final String TEZ_ENGINE_INTERMEDIATE_OUTPUT_SHOULD_COMPRESS = 
-      "tez.engine.intermediate-output.should-compress";
-  public static final String TEZ_ENGINE_INTERMEDIATE_INPUT_IS_COMPRESSED = 
-      "tez.engine.intermdiate-input.is-compressed";
-  
-  public static final String TEZ_ENGINE_INTERMEDIATE_OUTPUT_COMPRESS_CODEC = 
-      "tez.engine.intermediate-output.compress.codec";
-  public static final String TEZ_ENGINE_INTERMEDIATE_INPUT_COMPRESS_CODEC = 
-      "tez.engine.intermediate-input.compress.codec";
-
-  public static final String TEZ_ENGINE_INTERMEDIATE_INPUT_KEY_SECONDARY_COMPARATOR_CLASS = 
-      "tez.engine.intermediate-input.key.secondary.comparator.class";
-  
-  // TODO This should be in DAGConfiguration
-  /* config for tracking the local file where all the credentials for the job
-   * credentials.
-   */
-  public static final String DAG_CREDENTIALS_BINARY =  "tez.dag.credentials.binary";
-  
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-common/src/main/java/org/apache/tez/common/TezTaskContext.java
----------------------------------------------------------------------
diff --git a/tez-common/src/main/java/org/apache/tez/common/TezTaskContext.java b/tez-common/src/main/java/org/apache/tez/common/TezTaskContext.java
deleted file mode 100644
index 4eed78b..0000000
--- a/tez-common/src/main/java/org/apache/tez/common/TezTaskContext.java
+++ /dev/null
@@ -1,88 +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.common;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.Writable;
-import org.apache.tez.dag.records.TezDAGID;
-import org.apache.tez.dag.records.TezTaskAttemptID;
-
-public abstract class TezTaskContext implements Writable {
-
-  // Serialized Fields
-  private TezTaskAttemptID taskAttemptId;
-  private String user;
-  private String jobName;
-  private String vertexName;
-
-  public TezTaskContext() {
-  }
-
-  public TezTaskContext(TezTaskAttemptID taskAttemptID, String user, String jobName,
-      String vertexName) {
-    this.taskAttemptId = taskAttemptID;
-    this.user = user;
-    this.jobName = jobName;
-    this.vertexName = vertexName;
-  }
-
-  public TezTaskAttemptID getTaskAttemptId() {
-    return taskAttemptId;
-  }
-
-  
-
-  public TezDAGID getDAGID() {
-    return taskAttemptId.getTaskID().getVertexID().getDAGId();
-  }
-
-  public String getUser() {
-    return user;
-  }
-
-  public String getJobName() {
-    return jobName;
-  }
-  
-  public String getVertexName() {
-    return this.vertexName;
-  }
-
-  public void statusUpdate() throws IOException, InterruptedException {
-  }
-
-  @Override
-  public void write(DataOutput out) throws IOException {
-    taskAttemptId.write(out);
-    Text.writeString(out, user);
-    Text.writeString(out, jobName);
-    Text.writeString(out, vertexName);
-  }
-
-  public void readFields(DataInput in) throws IOException {
-    taskAttemptId = TezTaskAttemptID.read(in);
-    user = Text.readString(in);
-    jobName = Text.readString(in);
-    vertexName = Text.readString(in);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-common/src/main/java/org/apache/tez/common/TezTaskStatus.java
----------------------------------------------------------------------
diff --git a/tez-common/src/main/java/org/apache/tez/common/TezTaskStatus.java b/tez-common/src/main/java/org/apache/tez/common/TezTaskStatus.java
deleted file mode 100644
index de81f87..0000000
--- a/tez-common/src/main/java/org/apache/tez/common/TezTaskStatus.java
+++ /dev/null
@@ -1,104 +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.common;
-
-import java.util.List;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.io.Writable;
-import org.apache.tez.common.counters.TezCounters;
-import org.apache.tez.dag.records.TezTaskAttemptID;
-
-public interface TezTaskStatus extends Writable {
-
-  //enumeration for reporting current phase of a task.
-  @InterfaceAudience.Private
-  @InterfaceStability.Unstable
-  public static enum Phase{STARTING, MAP, SHUFFLE, SORT, REDUCE, CLEANUP}
-
-  // what state is the task in?
-  @InterfaceAudience.Private
-  @InterfaceStability.Unstable
-  public static enum State {RUNNING, SUCCEEDED, FAILED, UNASSIGNED, KILLED, 
-                            COMMIT_PENDING, FAILED_UNCLEAN, KILLED_UNCLEAN}
-
-  public abstract TezTaskAttemptID getTaskAttemptId();
-
-  public abstract float getProgress();
-
-  public abstract void setProgress(float progress);
-
-  public abstract State getRunState();
-
-  public abstract void setRunState(State runState);
-
-  public abstract String getDiagnosticInfo();
-
-  public abstract void setDiagnosticInfo(String info);
-
-  // TODOTEZDAG Remove stateString / rename
-  public abstract String getStateString();
-
-  public abstract void setStateString(String stateString);
-
-  public abstract long getFinishTime();
-
-  public abstract void setFinishTime(long finishTime);
-  
-  // TODOTEZDAG Can shuffle / merge be made generic ? Otherwise just a single finish time.
-  public abstract long getShuffleFinishTime();
-
-  public abstract void setShuffleFinishTime(long shuffleFinishTime);
-  
-  public abstract long getMapFinishTime();
-
-  public abstract void setMapFinishTime(long mapFinishTime);
-  
-  public abstract long getSortFinishTime();
-  
-  public abstract void setSortFinishTime(long sortFinishTime);
-  
-  public abstract long getStartTime();
-  
-  public abstract void setStartTime(long startTime);
-
-  // TODOTEZDAG Remove phase
-  public abstract Phase getPhase();
-
-  public abstract void setPhase(Phase phase);
-
-  public abstract TezCounters getCounters();
-
-  public abstract void setCounters(TezCounters counters);
-
-  public abstract List<TezTaskAttemptID> getFailedDependencies();
-
-  public abstract void addFailedDependency(TezTaskAttemptID taskAttempttId);
-
-  public abstract void clearStatus();
-
-  public abstract void statusUpdate(float f, String string, TezCounters counters);
-
-  // TODOTEZDAG maybe remove ?
-  public abstract long getLocalOutputSize();
-
-  public abstract void setOutputSize(long l);
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-common/src/main/java/org/apache/tez/common/counters/AbstractCounter.java
----------------------------------------------------------------------
diff --git a/tez-common/src/main/java/org/apache/tez/common/counters/AbstractCounter.java b/tez-common/src/main/java/org/apache/tez/common/counters/AbstractCounter.java
deleted file mode 100644
index e64a26c..0000000
--- a/tez-common/src/main/java/org/apache/tez/common/counters/AbstractCounter.java
+++ /dev/null
@@ -1,52 +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.common.counters;
-
-import com.google.common.base.Objects;
-import org.apache.hadoop.classification.InterfaceAudience;
-
-/**
- * An abstract counter class to provide common implementation of
- * the counter interface in both mapred and mapreduce packages.
- */
-@InterfaceAudience.Private
-public abstract class AbstractCounter implements TezCounter {
-
-  @Deprecated
-  @Override
-  public void setDisplayName(String name) {}
-
-  @Override
-  public synchronized boolean equals(Object genericRight) {
-    if (genericRight instanceof TezCounter) {
-      synchronized (genericRight) {
-        TezCounter right = (TezCounter) genericRight;
-        return getName().equals(right.getName()) &&
-               getDisplayName().equals(right.getDisplayName()) &&
-               getValue() == right.getValue();
-      }
-    }
-    return false;
-  }
-
-  @Override
-  public synchronized int hashCode() {
-    return Objects.hashCode(getName(), getDisplayName(), getValue());
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-common/src/main/java/org/apache/tez/common/counters/AbstractCounterGroup.java
----------------------------------------------------------------------
diff --git a/tez-common/src/main/java/org/apache/tez/common/counters/AbstractCounterGroup.java b/tez-common/src/main/java/org/apache/tez/common/counters/AbstractCounterGroup.java
deleted file mode 100644
index d8896ed..0000000
--- a/tez-common/src/main/java/org/apache/tez/common/counters/AbstractCounterGroup.java
+++ /dev/null
@@ -1,208 +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.common.counters;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.util.Iterator;
-import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.ConcurrentSkipListMap;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.WritableUtils;
-
-import com.google.common.collect.Iterators;
-
-/**
- * An abstract class to provide common implementation of the
- * generic counter group in both mapred and mapreduce package.
- *
- * @param <T> type of the counter for the group
- */
-@InterfaceAudience.Private
-public abstract class AbstractCounterGroup<T extends TezCounter>
-    implements CounterGroupBase<T> {
-
-  private final String name;
-  private String displayName;
-  private final ConcurrentMap<String, T> counters =
-      new ConcurrentSkipListMap<String, T>();
-  private final Limits limits;
-
-  public AbstractCounterGroup(String name, String displayName,
-                              Limits limits) {
-    this.name = name;
-    this.displayName = displayName;
-    this.limits = limits;
-  }
-
-  @Override
-  public String getName() {
-    return name;
-  }
-
-  @Override
-  public synchronized String getDisplayName() {
-    return displayName;
-  }
-
-  @Override
-  public synchronized void setDisplayName(String displayName) {
-    this.displayName = displayName;
-  }
-
-  @Override
-  public synchronized void addCounter(T counter) {
-    counters.put(counter.getName(), counter);
-    limits.incrCounters();
-  }
-
-  @Override
-  public synchronized T addCounter(String counterName, String displayName,
-                                   long value) {
-    String saveName = Limits.filterCounterName(counterName);
-    T counter = findCounterImpl(saveName, false);
-    if (counter == null) {
-      return addCounterImpl(saveName, displayName, value);
-    }
-    counter.setValue(value);
-    return counter;
-  }
-
-  private T addCounterImpl(String name, String displayName, long value) {
-    T counter = newCounter(name, displayName, value);
-    addCounter(counter);
-    return counter;
-  }
-
-  @Override
-  public synchronized T findCounter(String counterName, String displayName) {
-    // Take lock to avoid two threads not finding a counter and trying to add
-    // the same counter.
-    String saveName = Limits.filterCounterName(counterName);
-    T counter = findCounterImpl(saveName, false);
-    if (counter == null) {
-      return addCounterImpl(saveName, displayName, 0);
-    }
-    return counter;
-  }
-
-  @Override
-  public T findCounter(String counterName, boolean create) {
-    return findCounterImpl(Limits.filterCounterName(counterName), create);
-  }
-
-  // Lock the object. Cannot simply use concurrent constructs on the counters
-  // data-structure (like putIfAbsent) because of localization, limits etc.
-  private synchronized T findCounterImpl(String counterName, boolean create) {
-    T counter = counters.get(counterName);
-    if (counter == null && create) {
-      String localized =
-          ResourceBundles.getCounterName(getName(), counterName, counterName);
-      return addCounterImpl(counterName, localized, 0);
-    }
-    return counter;
-  }
-
-  @Override
-  public T findCounter(String counterName) {
-    return findCounter(counterName, true);
-  }
-
-  /**
-   * Abstract factory method to create a new counter of type T
-   * @param counterName of the counter
-   * @param displayName of the counter
-   * @param value of the counter
-   * @return a new counter
-   */
-  protected abstract T newCounter(String counterName, String displayName,
-                                  long value);
-
-  /**
-   * Abstract factory method to create a new counter of type T
-   * @return a new counter object
-   */
-  protected abstract T newCounter();
-
-  @Override
-  public Iterator<T> iterator() {
-    return counters.values().iterator();
-  }
-
-  /**
-   * GenericGroup ::= displayName #counter counter*
-   */
-  @Override
-  public synchronized void write(DataOutput out) throws IOException {
-    Text.writeString(out, displayName);
-    WritableUtils.writeVInt(out, counters.size());
-    for(TezCounter counter: counters.values()) {
-      counter.write(out);
-    }
-  }
-
-  @Override
-  public synchronized void readFields(DataInput in) throws IOException {
-    displayName = Text.readString(in);
-    counters.clear();
-    int size = WritableUtils.readVInt(in);
-    for (int i = 0; i < size; i++) {
-      T counter = newCounter();
-      counter.readFields(in);
-      counters.put(counter.getName(), counter);
-      limits.incrCounters();
-    }
-  }
-
-  @Override
-  public synchronized int size() {
-    return counters.size();
-  }
-
-  @Override
-  public synchronized boolean equals(Object genericRight) {
-    if (genericRight instanceof CounterGroupBase<?>) {
-      @SuppressWarnings("unchecked")
-      CounterGroupBase<T> right = (CounterGroupBase<T>) genericRight;
-      return Iterators.elementsEqual(iterator(), right.iterator());
-    }
-    return false;
-  }
-
-  @Override
-  public synchronized int hashCode() {
-    return counters.hashCode();
-  }
-
-  @Override
-  public void incrAllCounters(CounterGroupBase<T> rightGroup) {
-    try {
-      for (TezCounter right : rightGroup) {
-        TezCounter left = findCounter(right.getName(), right.getDisplayName());
-        left.increment(right.getValue());
-      }
-    } catch (LimitExceededException e) {
-      counters.clear();
-      throw e;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-common/src/main/java/org/apache/tez/common/counters/AbstractCounters.java
----------------------------------------------------------------------
diff --git a/tez-common/src/main/java/org/apache/tez/common/counters/AbstractCounters.java b/tez-common/src/main/java/org/apache/tez/common/counters/AbstractCounters.java
deleted file mode 100644
index fd4fdee..0000000
--- a/tez-common/src/main/java/org/apache/tez/common/counters/AbstractCounters.java
+++ /dev/null
@@ -1,385 +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.common.counters;
-
-import static org.apache.tez.common.counters.CounterGroupFactory.getFrameworkGroupId;
-import static org.apache.tez.common.counters.CounterGroupFactory.isFrameworkGroup;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.Map;
-import java.util.concurrent.ConcurrentSkipListMap;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.io.WritableUtils;
-
-import com.google.common.collect.Iterables;
-import com.google.common.collect.Iterators;
-import com.google.common.collect.Maps;
-
-/**
- * An abstract class to provide common implementation for the Counters
- * container in both mapred and mapreduce packages.
- *
- * @param <C> type of counter inside the counters
- * @param <G> type of group inside the counters
- */
-@InterfaceAudience.Public
-@InterfaceStability.Stable
-public abstract class AbstractCounters<C extends TezCounter,
-                                       G extends CounterGroupBase<C>>
-    implements Writable, Iterable<G> {
-
-  protected static final Log LOG = LogFactory.getLog("mapreduce.Counters");
-
-  /**
-   * A cache from enum values to the associated counter.
-   */
-  private Map<Enum<?>, C> cache = Maps.newIdentityHashMap();
-  //framework & fs groups
-  private Map<String, G> fgroups = new ConcurrentSkipListMap<String, G>();
-  // other groups
-  private Map<String, G> groups = new ConcurrentSkipListMap<String, G>();
-  private final CounterGroupFactory<C, G> groupFactory;
-
-  // For framework counter serialization without strings
-  enum GroupType { FRAMEWORK, FILESYSTEM };
-
-  // Writes only framework and fs counters if false.
-  private boolean writeAllCounters = true;
-
-  private static final Map<String, String> legacyMap = Maps.newHashMap();
-  static {
-    legacyMap.put("org.apache.hadoop.mapred.Task$Counter",
-                  TaskCounter.class.getName());
-    legacyMap.put("org.apache.hadoop.mapred.JobInProgress$Counter",
-                  JobCounter.class.getName());
-    legacyMap.put("FileSystemCounters", FileSystemCounter.class.getName());
-  }
-
-  private final Limits limits = new Limits();
-
-  @InterfaceAudience.Private
-  public AbstractCounters(CounterGroupFactory<C, G> gf) {
-    groupFactory = gf;
-  }
-
-  /**
-   * Construct from another counters object.
-   * @param <C1> type of the other counter
-   * @param <G1> type of the other counter group
-   * @param counters the counters object to copy
-   * @param groupFactory the factory for new groups
-   */
-  @InterfaceAudience.Private
-  public <C1 extends TezCounter, G1 extends CounterGroupBase<C1>>
-  AbstractCounters(AbstractCounters<C1, G1> counters,
-                   CounterGroupFactory<C, G> groupFactory) {
-    this.groupFactory = groupFactory;
-    for(G1 group: counters) {
-      String name = group.getName();
-      G newGroup = groupFactory.newGroup(name, group.getDisplayName(), limits);
-      (isFrameworkGroup(name) ? fgroups : groups).put(name, newGroup);
-      for(TezCounter counter: group) {
-        newGroup.addCounter(counter.getName(), counter.getDisplayName(),
-                            counter.getValue());
-      }
-    }
-  }
-
-  /** Add a group.
-   * @param group object to add
-   * @return the group
-   */
-  @InterfaceAudience.Private
-  public synchronized G addGroup(G group) {
-    String name = group.getName();
-    if (isFrameworkGroup(name)) {
-      fgroups.put(name, group);
-    } else {
-      limits.checkGroups(groups.size() + 1);
-      groups.put(name, group);
-    }
-    return group;
-  }
-
-  /**
-   * Add a new group
-   * @param name of the group
-   * @param displayName of the group
-   * @return the group
-   */
-  @InterfaceAudience.Private
-  public G addGroup(String name, String displayName) {
-    return addGroup(groupFactory.newGroup(name, displayName, limits));
-  }
-
-  /**
-   * Find a counter, create one if necessary
-   * @param groupName of the counter
-   * @param counterName name of the counter
-   * @return the matching counter
-   */
-  public C findCounter(String groupName, String counterName) {
-    G grp = getGroup(groupName);
-    return grp.findCounter(counterName);
-  }
-
-  /**
-   * Find the counter for the given enum. The same enum will always return the
-   * same counter.
-   * @param key the counter key
-   * @return the matching counter object
-   */
-  public synchronized C findCounter(Enum<?> key) {
-    C counter = cache.get(key);
-    if (counter == null) {
-      counter = findCounter(key.getDeclaringClass().getName(), key.name());
-      cache.put(key, counter);
-    }
-    return counter;
-  }
-
-  /**
-   * Find the file system counter for the given scheme and enum.
-   * @param scheme of the file system
-   * @param key the enum of the counter
-   * @return the file system counter
-   */
-  @InterfaceAudience.Private
-  public synchronized C findCounter(String scheme, FileSystemCounter key) {
-    return ((FileSystemCounterGroup<C>) getGroup(
-        FileSystemCounter.class.getName()).getUnderlyingGroup()).
-        findCounter(scheme, key);
-  }
-
-  /**
-   * Returns the names of all counter classes.
-   * @return Set of counter names.
-   */
-  public synchronized Iterable<String> getGroupNames() {
-    HashSet<String> deprecated = new HashSet<String>();
-    for(Map.Entry<String, String> entry : legacyMap.entrySet()) {
-      String newGroup = entry.getValue();
-      boolean isFGroup = isFrameworkGroup(newGroup);
-      if(isFGroup ? fgroups.containsKey(newGroup) : groups.containsKey(newGroup)) {
-        deprecated.add(entry.getKey());
-      }
-    }
-    return Iterables.concat(fgroups.keySet(), groups.keySet(), deprecated);
-  }
-
-  @Override
-  public Iterator<G> iterator() {
-    return Iterators.concat(fgroups.values().iterator(),
-                            groups.values().iterator());
-  }
-
-  /**
-   * Returns the named counter group, or an empty group if there is none
-   * with the specified name.
-   * @param groupName name of the group
-   * @return the group
-   */
-  public synchronized G getGroup(String groupName) {
-
-    // filterGroupName
-    boolean groupNameInLegacyMap = true;
-    String newGroupName = legacyMap.get(groupName);
-    if (newGroupName == null) {
-      groupNameInLegacyMap = false;
-      newGroupName = Limits.filterGroupName(groupName);
-    }
-
-    boolean isFGroup = isFrameworkGroup(newGroupName);
-    G group = isFGroup ? fgroups.get(newGroupName) : groups.get(newGroupName);
-    if (group == null) {
-      group = groupFactory.newGroup(newGroupName, limits);
-      if (isFGroup) {
-        fgroups.put(newGroupName, group);
-      } else {
-        limits.checkGroups(groups.size() + 1);
-        groups.put(newGroupName, group);
-      }
-      if (groupNameInLegacyMap) {
-        LOG.warn("Group " + groupName + " is deprecated. Use " + newGroupName
-            + " instead");
-      }
-    }
-    return group;
-  }
-
-  /**
-   * Returns the total number of counters, by summing the number of counters
-   * in each group.
-   * @return the total number of counters
-   */
-  public synchronized int countCounters() {
-    int result = 0;
-    for (G group : this) {
-      result += group.size();
-    }
-    return result;
-  }
-
-  /**
-   * Write the set of groups.
-   * Counters ::= version #fgroups (groupId, group)* #groups (group)*
-   */
-  @Override
-  public synchronized void write(DataOutput out) throws IOException {
-    WritableUtils.writeVInt(out, groupFactory.version());
-    WritableUtils.writeVInt(out, fgroups.size());  // framework groups first
-    for (G group : fgroups.values()) {
-      if (group.getUnderlyingGroup() instanceof FrameworkCounterGroup<?, ?>) {
-        WritableUtils.writeVInt(out, GroupType.FRAMEWORK.ordinal());
-        WritableUtils.writeVInt(out, getFrameworkGroupId(group.getName()));
-        group.write(out);
-      } else if (group.getUnderlyingGroup() instanceof FileSystemCounterGroup<?>) {
-        WritableUtils.writeVInt(out, GroupType.FILESYSTEM.ordinal());
-        group.write(out);
-      }
-    }
-    if (writeAllCounters) {
-      WritableUtils.writeVInt(out, groups.size());
-      for (G group : groups.values()) {
-        Text.writeString(out, group.getName());
-        group.write(out);
-      }
-    } else {
-      WritableUtils.writeVInt(out, 0);
-    }
-  }
-
-  @Override
-  public synchronized void readFields(DataInput in) throws IOException {
-    int version = WritableUtils.readVInt(in);
-    if (version != groupFactory.version()) {
-      throw new IOException("Counters version mismatch, expected "+
-          groupFactory.version() +" got "+ version);
-    }
-    int numFGroups = WritableUtils.readVInt(in);
-    fgroups.clear();
-    GroupType[] groupTypes = GroupType.values();
-    while (numFGroups-- > 0) {
-      GroupType groupType = groupTypes[WritableUtils.readVInt(in)];
-      G group;
-      switch (groupType) {
-        case FILESYSTEM: // with nothing
-          group = groupFactory.newFileSystemGroup();
-          break;
-        case FRAMEWORK:  // with group id
-          group = groupFactory.newFrameworkGroup(WritableUtils.readVInt(in));
-          break;
-        default: // Silence dumb compiler, as it would've thrown earlier
-          throw new IOException("Unexpected counter group type: "+ groupType);
-      }
-      group.readFields(in);
-      fgroups.put(group.getName(), group);
-    }
-    int numGroups = WritableUtils.readVInt(in);
-    while (numGroups-- > 0) {
-      limits.checkGroups(groups.size() + 1);
-      G group = groupFactory.newGenericGroup(Text.readString(in), null, limits);
-      group.readFields(in);
-      groups.put(group.getName(), group);
-    }
-  }
-
-  /**
-   * Return textual representation of the counter values.
-   * @return the string
-   */
-  @Override
-  public synchronized String toString() {
-    StringBuilder sb = new StringBuilder("Counters: " + countCounters());
-    for (G group: this) {
-      sb.append("\n\t").append(group.getDisplayName());
-      for (TezCounter counter: group) {
-        sb.append("\n\t\t").append(counter.getDisplayName()).append("=")
-          .append(counter.getValue());
-      }
-    }
-    return sb.toString();
-  }
-
-  /**
-   * Increments multiple counters by their amounts in another Counters
-   * instance.
-   * @param other the other Counters instance
-   */
-  public synchronized void incrAllCounters(AbstractCounters<C, G> other) {
-    for(G right : other) {
-      String groupName = right.getName();
-      G left = (isFrameworkGroup(groupName) ? fgroups : groups).get(groupName);
-      if (left == null) {
-        left = addGroup(groupName, right.getDisplayName());
-      }
-      left.incrAllCounters(right);
-    }
-  }
-
-  @Override
-  @SuppressWarnings("unchecked")
-  public boolean equals(Object genericRight) {
-    if (genericRight instanceof AbstractCounters<?, ?>) {
-      return Iterators.elementsEqual(iterator(),
-          ((AbstractCounters<C, G>)genericRight).iterator());
-    }
-    return false;
-  }
-
-  @Override
-  public int hashCode() {
-    return groups.hashCode();
-  }
-
-  /**
-   * Set the "writeAllCounters" option to true or false
-   * @param send  if true all counters would be serialized, otherwise only
-   *              framework counters would be serialized in
-   *              {@link #write(DataOutput)}
-   */
-  @InterfaceAudience.Private
-  public void setWriteAllCounters(boolean send) {
-    writeAllCounters = send;
-  }
-
-  /**
-   * Get the "writeAllCounters" option
-   * @return true of all counters would serialized
-   */
-  @InterfaceAudience.Private
-  public boolean getWriteAllCounters() {
-    return writeAllCounters;
-  }
-
-  @InterfaceAudience.Private
-  public Limits limits() {
-    return limits;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-common/src/main/java/org/apache/tez/common/counters/CounterGroup.java
----------------------------------------------------------------------
diff --git a/tez-common/src/main/java/org/apache/tez/common/counters/CounterGroup.java b/tez-common/src/main/java/org/apache/tez/common/counters/CounterGroup.java
deleted file mode 100644
index bc7986d..0000000
--- a/tez-common/src/main/java/org/apache/tez/common/counters/CounterGroup.java
+++ /dev/null
@@ -1,32 +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.common.counters;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-
-/**
- * A group of {@link TezCounter}s that logically belong together. Typically,
- * it is an {@link Enum} subclass and the counters are the values.
- */
-@InterfaceAudience.Public
-@InterfaceStability.Stable
-public interface CounterGroup extends CounterGroupBase<TezCounter> {
-  // essentially a typedef so user doesn't have to use generic syntax
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-common/src/main/java/org/apache/tez/common/counters/CounterGroupBase.java
----------------------------------------------------------------------
diff --git a/tez-common/src/main/java/org/apache/tez/common/counters/CounterGroupBase.java b/tez-common/src/main/java/org/apache/tez/common/counters/CounterGroupBase.java
deleted file mode 100644
index 3b702ba..0000000
--- a/tez-common/src/main/java/org/apache/tez/common/counters/CounterGroupBase.java
+++ /dev/null
@@ -1,108 +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.common.counters;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceAudience.Private;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.io.Writable;
-
-/**
- * The common counter group interface.
- *
- * @param <T> type of the counter for the group
- */
-@InterfaceAudience.Public
-@InterfaceStability.Evolving
-public interface CounterGroupBase<T extends TezCounter>
-    extends Writable, Iterable<T> {
-
-  /**
-   * Get the internal name of the group
-   * @return the internal name
-   */
-  String getName();
-
-  /**
-   * Get the display name of the group.
-   * @return the human readable name
-   */
-  String getDisplayName();
-
-  /**
-   * Set the display name of the group
-   * @param displayName of the group
-   */
-  void setDisplayName(String displayName);
-
-  /** Add a counter to this group.
-   * @param counter to add
-   */
-  void addCounter(T counter);
-
-  /**
-   * Add a counter to this group
-   * @param name  of the counter
-   * @param displayName of the counter
-   * @param value of the counter
-   * @return the counter
-   */
-  T addCounter(String name, String displayName, long value);
-
-  /**
-   * Find a counter in the group.
-   * @param counterName the name of the counter
-   * @param displayName the display name of the counter
-   * @return the counter that was found or added
-   */
-  T findCounter(String counterName, String displayName);
-
-  /**
-   * Find a counter in the group
-   * @param counterName the name of the counter
-   * @param create create the counter if not found if true
-   * @return the counter that was found or added or null if create is false
-   */
-  T findCounter(String counterName, boolean create);
-
-  /**
-   * Find a counter in the group.
-   * @param counterName the name of the counter
-   * @return the counter that was found or added
-   */
-  T findCounter(String counterName);
-
-  /**
-   * @return the number of counters in this group.
-   */
-  int size();
-
-  /**
-   * Increment all counters by a group of counters
-   * @param rightGroup  the group to be added to this group
-   */
-  void incrAllCounters(CounterGroupBase<T> rightGroup);
-  
-  @Private
-  /**
-   * Exposes the underlying group type if a facade.
-   * @return the underlying object that this object is wrapping up.
-   */
-  CounterGroupBase<T> getUnderlyingGroup();
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-common/src/main/java/org/apache/tez/common/counters/CounterGroupFactory.java
----------------------------------------------------------------------
diff --git a/tez-common/src/main/java/org/apache/tez/common/counters/CounterGroupFactory.java b/tez-common/src/main/java/org/apache/tez/common/counters/CounterGroupFactory.java
deleted file mode 100644
index 45da0dd..0000000
--- a/tez-common/src/main/java/org/apache/tez/common/counters/CounterGroupFactory.java
+++ /dev/null
@@ -1,180 +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.common.counters;
-
-import java.util.List;
-import java.util.Map;
-
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-
-/**
- * An abstract class to provide common implementation of the
- * group factory in both mapred and mapreduce packages.
- *
- * @param <C> type of the counter
- * @param <G> type of the group
- */
-@InterfaceAudience.Private
-public abstract class CounterGroupFactory<C extends TezCounter,
-                                          G extends CounterGroupBase<C>> {
-
-  public interface FrameworkGroupFactory<F> {
-    F newGroup(String name);
-  }
-
-  // Integer mapping (for serialization) for framework groups
-  private static final Map<String, Integer> s2i = Maps.newHashMap();
-  private static final List<String> i2s = Lists.newArrayList();
-  private static final int VERSION = 1;
-  private static final String FS_GROUP_NAME = FileSystemCounter.class.getName();
-
-  private final Map<String, FrameworkGroupFactory<G>> fmap = Maps.newHashMap();
-  {
-    // Add builtin counter class here and the version when changed.
-    addFrameworkGroup(TaskCounter.class);
-    addFrameworkGroup(JobCounter.class);
-    addFrameworkGroup(DAGCounter.class);
-  }
-
-  // Initialize the framework counter group mapping
-  private synchronized <T extends Enum<T>>
-  void addFrameworkGroup(final Class<T> cls) {
-    updateFrameworkGroupMapping(cls);
-    fmap.put(cls.getName(), newFrameworkGroupFactory(cls));
-  }
-
-  // Update static mappings (c2i, i2s) of framework groups
-  private static synchronized void updateFrameworkGroupMapping(Class<?> cls) {
-    String name = cls.getName();
-    Integer i = s2i.get(name);
-    if (i != null) return;
-    i2s.add(name);
-    s2i.put(name, i2s.size() - 1);
-  }
-
-  /**
-   * Required override to return a new framework group factory
-   * @param <T> type of the counter enum class
-   * @param cls the counter enum class
-   * @return a new framework group factory
-   */
-  protected abstract <T extends Enum<T>>
-  FrameworkGroupFactory<G> newFrameworkGroupFactory(Class<T> cls);
-
-  /**
-   * Create a new counter group
-   * @param name of the group
-   * @param limits the counters limits policy object
-   * @return a new counter group
-   */
-  public G newGroup(String name, Limits limits) {
-    return newGroup(name, ResourceBundles.getCounterGroupName(name, name),
-                    limits);
-  }
-
-  /**
-   * Create a new counter group
-   * @param name of the group
-   * @param displayName of the group
-   * @param limits the counters limits policy object
-   * @return a new counter group
-   */
-  public G newGroup(String name, String displayName, Limits limits) {
-    FrameworkGroupFactory<G> gf = fmap.get(name);
-    if (gf != null) return gf.newGroup(name);
-    if (name.equals(FS_GROUP_NAME)) {
-      return newFileSystemGroup();
-    } else if (s2i.get(name) != null) {
-      return newFrameworkGroup(s2i.get(name));
-    }
-    return newGenericGroup(name, displayName, limits);
-  }
-
-  /**
-   * Create a new framework group
-   * @param id of the group
-   * @return a new framework group
-   */
-  public G newFrameworkGroup(int id) {
-    String name;
-    synchronized(CounterGroupFactory.class) {
-      if (id < 0 || id >= i2s.size()) throwBadFrameGroupIdException(id);
-      name = i2s.get(id); // should not throw here.
-    }
-    FrameworkGroupFactory<G> gf = fmap.get(name);
-    if (gf == null) throwBadFrameGroupIdException(id);
-    return gf.newGroup(name);
-  }
-
-  /**
-   * Get the id of a framework group
-   * @param name of the group
-   * @return the framework group id
-   */
-  public static synchronized int getFrameworkGroupId(String name) {
-    Integer i = s2i.get(name);
-    if (i == null) throwBadFrameworkGroupNameException(name);
-    return i;
-  }
-
-  /**
-   * @return the counter factory version
-   */
-  public int version() {
-    return VERSION;
-  }
-
-  /**
-   * Check whether a group name is a name of a framework group (including
-   * the filesystem group).
-   *
-   * @param name  to check
-   * @return true for framework group names
-   */
-  public static synchronized boolean isFrameworkGroup(String name) {
-    return s2i.get(name) != null || name.equals(FS_GROUP_NAME);
-  }
-
-  private static void throwBadFrameGroupIdException(int id) {
-    throw new IllegalArgumentException("bad framework group id: "+ id);
-  }
-
-  private static void throwBadFrameworkGroupNameException(String name) {
-    throw new IllegalArgumentException("bad framework group name: "+ name);
-  }
-
-  /**
-   * Abstract factory method to create a generic (vs framework) counter group
-   * @param name  of the group
-   * @param displayName of the group
-   * @param limits limits of the counters
-   * @return a new generic counter group
-   */
-  protected abstract G newGenericGroup(String name, String displayName,
-                                       Limits limits);
-
-  /**
-   * Abstract factory method to create a file system counter group
-   * @return a new file system counter group
-   */
-  protected abstract G newFileSystemGroup();
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-common/src/main/java/org/apache/tez/common/counters/DAGCounter.java
----------------------------------------------------------------------
diff --git a/tez-common/src/main/java/org/apache/tez/common/counters/DAGCounter.java b/tez-common/src/main/java/org/apache/tez/common/counters/DAGCounter.java
deleted file mode 100644
index 3598572..0000000
--- a/tez-common/src/main/java/org/apache/tez/common/counters/DAGCounter.java
+++ /dev/null
@@ -1,39 +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.common.counters;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-
-// Per-job counters
-@InterfaceAudience.Public
-@InterfaceStability.Evolving
-public enum DAGCounter {
-  NUM_FAILED_TASKS, 
-  NUM_KILLED_TASKS,
-  TOTAL_LAUNCHED_TASKS,
-  OTHER_LOCAL_TASKS,
-  DATA_LOCAL_TASKS,
-  RACK_LOCAL_TASKS,
-  SLOTS_MILLIS_TASKS,
-  FALLOW_SLOTS_MILLIS_TASKS,
-  TOTAL_LAUNCHED_UBERTASKS,
-  NUM_UBER_SUBTASKS,
-  NUM_FAILED_UBERTASKS
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-common/src/main/java/org/apache/tez/common/counters/FileSystemCounter.java
----------------------------------------------------------------------
diff --git a/tez-common/src/main/java/org/apache/tez/common/counters/FileSystemCounter.java b/tez-common/src/main/java/org/apache/tez/common/counters/FileSystemCounter.java
deleted file mode 100644
index 08f4c5d..0000000
--- a/tez-common/src/main/java/org/apache/tez/common/counters/FileSystemCounter.java
+++ /dev/null
@@ -1,30 +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.common.counters;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-
-@InterfaceAudience.Private
-public enum FileSystemCounter {
-  BYTES_READ,
-  BYTES_WRITTEN,
-  READ_OPS,
-  LARGE_READ_OPS,
-  WRITE_OPS,
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-common/src/main/java/org/apache/tez/common/counters/FileSystemCounterGroup.java
----------------------------------------------------------------------
diff --git a/tez-common/src/main/java/org/apache/tez/common/counters/FileSystemCounterGroup.java b/tez-common/src/main/java/org/apache/tez/common/counters/FileSystemCounterGroup.java
deleted file mode 100644
index d4b167a..0000000
--- a/tez-common/src/main/java/org/apache/tez/common/counters/FileSystemCounterGroup.java
+++ /dev/null
@@ -1,327 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tez.common.counters;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.util.Arrays;
-import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.ConcurrentSkipListMap;
-import java.util.Iterator;
-import java.util.Locale;
-import java.util.Map;
-
-import com.google.common.base.Joiner;
-import static com.google.common.base.Preconditions.*;
-import com.google.common.collect.AbstractIterator;
-import com.google.common.collect.Iterators;
-import com.google.common.collect.Maps;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.io.WritableUtils;
-
-/**
- * An abstract class to provide common implementation of the filesystem
- * counter group in both mapred and mapreduce packages.
- *
- * @param <C> the type of the Counter for the group
- */
-@InterfaceAudience.Private
-public abstract class FileSystemCounterGroup<C extends TezCounter>
-    implements CounterGroupBase<C> {
-
-  static final int MAX_NUM_SCHEMES = 100; // intern/sanity check
-  static final ConcurrentMap<String, String> schemes = Maps.newConcurrentMap();
-
-  // C[] would need Array.newInstance which requires a Class<C> reference.
-  // Just a few local casts probably worth not having to carry it around.
-  private final Map<String, Object[]> map =
-    new ConcurrentSkipListMap<String, Object[]>();
-  private String displayName;
-
-  private static final Joiner NAME_JOINER = Joiner.on('_');
-  private static final Joiner DISP_JOINER = Joiner.on(": ");
-
-  @InterfaceAudience.Private
-  public static class FSCounter extends AbstractCounter {
-    final String scheme;
-    final FileSystemCounter key;
-    private long value;
-
-    public FSCounter(String scheme, FileSystemCounter ref) {
-      this.scheme = scheme;
-      key = ref;
-    }
-
-    @Override
-    public String getName() {
-      return NAME_JOINER.join(scheme, key.name());
-    }
-
-    @Override
-    public String getDisplayName() {
-      return DISP_JOINER.join(scheme, localizeCounterName(key.name()));
-    }
-
-    protected String localizeCounterName(String counterName) {
-      return ResourceBundles.getCounterName(FileSystemCounter.class.getName(),
-                                            counterName, counterName);
-    }
-
-    @Override
-    public long getValue() {
-      return value;
-    }
-
-    @Override
-    public void setValue(long value) {
-      this.value = value;
-    }
-
-    @Override
-    public void increment(long incr) {
-      value += incr;
-    }
-
-    @Override
-    public void write(DataOutput out) throws IOException {
-      assert false : "shouldn't be called";
-    }
-
-    @Override
-    public void readFields(DataInput in) throws IOException {
-      assert false : "shouldn't be called";
-    }
-
-    @Override
-    public TezCounter getUnderlyingCounter() {
-      return this;
-    }
-  }
-
-  @Override
-  public String getName() {
-    return FileSystemCounter.class.getName();
-  }
-
-  @Override
-  public String getDisplayName() {
-    if (displayName == null) {
-      displayName = ResourceBundles.getCounterGroupName(getName(),
-          "File System Counters");
-    }
-    return displayName;
-  }
-
-  @Override
-  public void setDisplayName(String displayName) {
-    this.displayName = displayName;
-  }
-
-  @Override
-  public void addCounter(C counter) {
-    C ours;
-    if (counter instanceof FileSystemCounterGroup.FSCounter) {
-      FSCounter c = (FSCounter) counter;
-      ours = findCounter(c.scheme, c.key);
-    }
-    else {
-      ours = findCounter(counter.getName());
-    }
-    ours.setValue(counter.getValue());
-  }
-
-  @Override
-  public C addCounter(String name, String displayName, long value) {
-    C counter = findCounter(name);
-    counter.setValue(value);
-    return counter;
-  }
-
-  // Parse generic counter name into [scheme, key]
-  private String[] parseCounterName(String counterName) {
-    int schemeEnd = counterName.indexOf('_');
-    if (schemeEnd < 0) {
-      throw new IllegalArgumentException("bad fs counter name");
-    }
-    return new String[]{counterName.substring(0, schemeEnd),
-                        counterName.substring(schemeEnd + 1)};
-  }
-
-  @Override
-  public C findCounter(String counterName, String displayName) {
-    return findCounter(counterName);
-  }
-
-  @Override
-  public C findCounter(String counterName, boolean create) {
-    try {
-      String[] pair = parseCounterName(counterName);
-      return findCounter(pair[0], FileSystemCounter.valueOf(pair[1]));
-    }
-    catch (Exception e) {
-      if (create) throw new IllegalArgumentException(e);
-      return null;
-    }
-  }
-
-  @Override
-  public C findCounter(String counterName) {
-    return findCounter(counterName, true);
-  }
-
-  @SuppressWarnings("unchecked")
-  public synchronized C findCounter(String scheme, FileSystemCounter key) {
-    final String canonicalScheme = checkScheme(scheme);
-    Object[] counters = map.get(canonicalScheme);
-    int ord = key.ordinal();
-    if (counters == null) {
-      counters = new Object[FileSystemCounter.values().length];
-      map.put(canonicalScheme, counters);
-      counters[ord] = newCounter(canonicalScheme, key);
-    }
-    else if (counters[ord] == null) {
-      counters[ord] = newCounter(canonicalScheme, key);
-    }
-    return (C) counters[ord];
-  }
-
-  private String checkScheme(String scheme) {
-    String fixed = scheme.toUpperCase(Locale.US);
-    String interned = schemes.putIfAbsent(fixed, fixed);
-    if (schemes.size() > MAX_NUM_SCHEMES) {
-      // mistakes or abuses
-      throw new IllegalArgumentException("too many schemes? "+ schemes.size() +
-                                         " when process scheme: "+ scheme);
-    }
-    return interned == null ? fixed : interned;
-  }
-
-  /**
-   * Abstract factory method to create a file system counter
-   * @param scheme of the file system
-   * @param key the enum of the file system counter
-   * @return a new file system counter
-   */
-  protected abstract C newCounter(String scheme, FileSystemCounter key);
-
-  @Override
-  public int size() {
-    int n = 0;
-    for (Object[] counters : map.values()) {
-      n += numSetCounters(counters);
-    }
-    return n;
-  }
-
-  @Override
-  @SuppressWarnings("unchecked")
-  public void incrAllCounters(CounterGroupBase<C> other) {
-    if (checkNotNull(other.getUnderlyingGroup(), "other group")
-        instanceof FileSystemCounterGroup<?>) {
-      for (TezCounter counter : other) {
-        FSCounter c = (FSCounter) ((TezCounter)counter).getUnderlyingCounter();
-        findCounter(c.scheme, c.key) .increment(counter.getValue());
-      }
-    }
-  }
-
-  /**
-   * FileSystemGroup ::= #scheme (scheme #counter (key value)*)*
-   */
-  @Override
-  public void write(DataOutput out) throws IOException {
-    WritableUtils.writeVInt(out, map.size()); // #scheme
-    for (Map.Entry<String, Object[]> entry : map.entrySet()) {
-      WritableUtils.writeString(out, entry.getKey()); // scheme
-      // #counter for the above scheme
-      WritableUtils.writeVInt(out, numSetCounters(entry.getValue()));
-      for (Object counter : entry.getValue()) {
-        if (counter == null) continue;
-        @SuppressWarnings("unchecked")
-        FSCounter c = (FSCounter) ((TezCounter)counter).getUnderlyingCounter();
-        WritableUtils.writeVInt(out, c.key.ordinal());  // key
-        WritableUtils.writeVLong(out, c.getValue());    // value
-      }
-    }
-  }
-
-  private int numSetCounters(Object[] counters) {
-    int n = 0;
-    for (Object counter : counters) if (counter != null) ++n;
-    return n;
-  }
-
-  @Override
-  public void readFields(DataInput in) throws IOException {
-    int numSchemes = WritableUtils.readVInt(in);    // #scheme
-    FileSystemCounter[] enums = FileSystemCounter.values();
-    for (int i = 0; i < numSchemes; ++i) {
-      String scheme = WritableUtils.readString(in); // scheme
-      int numCounters = WritableUtils.readVInt(in); // #counter
-      for (int j = 0; j < numCounters; ++j) {
-        findCounter(scheme, enums[WritableUtils.readVInt(in)])  // key
-            .setValue(WritableUtils.readVLong(in)); // value
-      }
-    }
-  }
-
-  @Override
-  public Iterator<C> iterator() {
-    return new AbstractIterator<C>() {
-      Iterator<Object[]> it = map.values().iterator();
-      Object[] counters = it.hasNext() ? it.next() : null;
-      int i = 0;
-      @Override
-      protected C computeNext() {
-        while (counters != null) {
-          while (i < counters.length) {
-            @SuppressWarnings("unchecked")
-            C counter = (C) counters[i++];
-            if (counter != null) return counter;
-          }
-          i = 0;
-          counters = it.hasNext() ? it.next() : null;
-        }
-        return endOfData();
-      }
-    };
-  }
-
-  @Override
-  public synchronized boolean equals(Object genericRight) {
-    if (genericRight instanceof CounterGroupBase<?>) {
-      @SuppressWarnings("unchecked")
-      CounterGroupBase<C> right = (CounterGroupBase<C>) genericRight;
-      return Iterators.elementsEqual(iterator(), right.iterator());
-    }
-    return false;
-  }
-
-  @Override
-  public synchronized int hashCode() {
-    // need to be deep as counters is an array
-    int hash = FileSystemCounter.class.hashCode();
-    for (Object[] counters : map.values()) {
-      if (counters != null) hash ^= Arrays.hashCode(counters);
-    }
-    return hash;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-common/src/main/java/org/apache/tez/common/counters/FrameworkCounterGroup.java
----------------------------------------------------------------------
diff --git a/tez-common/src/main/java/org/apache/tez/common/counters/FrameworkCounterGroup.java b/tez-common/src/main/java/org/apache/tez/common/counters/FrameworkCounterGroup.java
deleted file mode 100644
index 42fb636..0000000
--- a/tez-common/src/main/java/org/apache/tez/common/counters/FrameworkCounterGroup.java
+++ /dev/null
@@ -1,275 +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.common.counters;
-
-import static com.google.common.base.Preconditions.checkNotNull;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.util.Arrays;
-import java.util.Iterator;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.io.WritableUtils;
-
-import com.google.common.collect.AbstractIterator;
-import com.google.common.collect.Iterators;
-
-/**
- * An abstract class to provide common implementation for the framework
- * counter group in both mapred and mapreduce packages.
- *
- * @param <T> type of the counter enum class
- * @param <C> type of the counter
- */
-@InterfaceAudience.Private
-public abstract class FrameworkCounterGroup<T extends Enum<T>,
-    C extends TezCounter> implements CounterGroupBase<C> {
-  private static final Log LOG = LogFactory.getLog(FrameworkCounterGroup.class);
-  
-  private final Class<T> enumClass; // for Enum.valueOf
-  private final Object[] counters;  // local casts are OK and save a class ref
-  private String displayName = null;
-
-  /**
-   * A counter facade for framework counters.
-   * Use old (which extends new) interface to make compatibility easier.
-   */
-  @InterfaceAudience.Private
-  public static class FrameworkCounter<T extends Enum<T>> extends AbstractCounter {
-    final T key;
-    final String groupName;
-    private long value;
-
-    public FrameworkCounter(T ref, String groupName) {
-      key = ref;
-      this.groupName = groupName;
-    }
-
-    @Override
-    public String getName() {
-      return key.name();
-    }
-
-    @Override
-    public String getDisplayName() {
-      return ResourceBundles.getCounterName(groupName, getName(), getName());
-    }
-
-    @Override
-    public long getValue() {
-      return value;
-    }
-
-    @Override
-    public void setValue(long value) {
-      this.value = value;
-    }
-
-    @Override
-    public void increment(long incr) {
-      value += incr;
-    }
-
-    @Override
-    public void write(DataOutput out) throws IOException {
-      assert false : "shouldn't be called";
-    }
-
-    @Override
-    public void readFields(DataInput in) throws IOException {
-      assert false : "shouldn't be called";
-    }
-
-    @Override
-    public TezCounter getUnderlyingCounter() {
-      return this;
-    }
-  }
-
-  @SuppressWarnings("unchecked")
-  public FrameworkCounterGroup(Class<T> enumClass) {
-    this.enumClass = enumClass;
-    T[] enums = enumClass.getEnumConstants();
-    counters = new Object[enums.length];
-  }
-
-  @Override
-  public String getName() {
-    return enumClass.getName();
-  }
-
-  @Override
-  public String getDisplayName() {
-    if (displayName == null) {
-      displayName = ResourceBundles.getCounterGroupName(getName(), getName());
-    }
-    return displayName;
-  }
-
-  @Override
-  public void setDisplayName(String displayName) {
-    this.displayName = displayName;
-  }
-
-  private T valueOf(String name) {
-    return Enum.valueOf(enumClass, name);
-  }
-
-  @Override
-  public void addCounter(C counter) {
-    C ours = findCounter(counter.getName());
-    ours.setValue(counter.getValue());
-  }
-
-  @Override
-  public C addCounter(String name, String displayName, long value) {
-    C counter = findCounter(name);
-    counter.setValue(value);
-    return counter;
-  }
-
-  @Override
-  public C findCounter(String counterName, String displayName) {
-    return findCounter(counterName);
-  }
-
-  @Override
-  public C findCounter(String counterName, boolean create) {
-    try {
-      return findCounter(valueOf(counterName));
-    }
-    catch (Exception e) {
-      if (create) throw new IllegalArgumentException(e);
-      return null;
-    }
-  }
-
-  @Override
-  public C findCounter(String counterName) {
-    return findCounter(valueOf(counterName));
-  }
-
-  @SuppressWarnings("unchecked")
-  private C findCounter(T key) {
-    int i = key.ordinal();
-    if (counters[i] == null) {
-      counters[i] = newCounter(key);
-    }
-    return (C) counters[i];
-  }
-
-  /**
-   * Abstract factory method for new framework counter
-   * @param key for the enum value of a counter
-   * @return a new counter for the key
-   */
-  protected abstract C newCounter(T key);
-
-  @Override
-  public int size() {
-    int n = 0;
-    for (int i = 0; i < counters.length; ++i) {
-      if (counters[i] != null) ++n;
-    }
-    return n;
-  }
-
-  @Override
-  @SuppressWarnings("unchecked")
-  public void incrAllCounters(CounterGroupBase<C> other) {
-    if (checkNotNull(other, "other counter group")
-        instanceof FrameworkCounterGroup<?, ?>) {
-      for (TezCounter counter : other) {
-        findCounter(((FrameworkCounter) counter).key.name())
-            .increment(counter.getValue());
-      }
-    }
-  }
-
-  /**
-   * FrameworkGroup ::= #counter (key value)*
-   */
-  @Override
-  @SuppressWarnings("unchecked")
-  public void write(DataOutput out) throws IOException {
-    WritableUtils.writeVInt(out, size());
-    for (int i = 0; i < counters.length; ++i) {
-      TezCounter counter = (C) counters[i];
-      if (counter != null) {
-        WritableUtils.writeVInt(out, i);
-        WritableUtils.writeVLong(out, counter.getValue());
-      }
-    }
-  }
-
-  @Override
-  public void readFields(DataInput in) throws IOException {
-    clear();
-    int len = WritableUtils.readVInt(in);
-    T[] enums = enumClass.getEnumConstants();
-    for (int i = 0; i < len; ++i) {
-      int ord = WritableUtils.readVInt(in);
-      TezCounter counter = newCounter(enums[ord]);
-      counter.setValue(WritableUtils.readVLong(in));
-      counters[ord] = counter;
-    }
-  }
-
-  private void clear() {
-    for (int i = 0; i < counters.length; ++i) {
-      counters[i] = null;
-    }
-  }
-
-  @Override
-  public Iterator<C> iterator() {
-    return new AbstractIterator<C>() {
-      int i = 0;
-      @Override
-      protected C computeNext() {
-        while (i < counters.length) {
-          @SuppressWarnings("unchecked")
-          C counter = (C) counters[i++];
-          if (counter != null) return counter;
-        }
-        return endOfData();
-      }
-    };
-  }
-
-  @Override
-  public boolean equals(Object genericRight) {
-    if (genericRight instanceof CounterGroupBase<?>) {
-      @SuppressWarnings("unchecked")
-      CounterGroupBase<C> right = (CounterGroupBase<C>) genericRight;
-      return Iterators.elementsEqual(iterator(), right.iterator());
-    }
-    return false;
-  }
-
-  @Override
-  public synchronized int hashCode() {
-    // need to be deep as counters is an array
-    return Arrays.deepHashCode(new Object[]{enumClass, counters, displayName});
-  }
-}


[33/50] [abbrv] Rename tez-engine-api to tez-runtime-api and tez-engine is split into 2: - tez-engine-library for user-visible Input/Output/Processor implementations - tez-engine-internals for framework internals

Posted by ss...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/Fetcher.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/Fetcher.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/Fetcher.java
new file mode 100644
index 0000000..f5d1802
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/Fetcher.java
@@ -0,0 +1,624 @@
+/**
+ * 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.library.common.shuffle.impl;
+
+import java.io.DataInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.net.HttpURLConnection;
+import java.net.MalformedURLException;
+import java.net.URL;
+import java.net.URLConnection;
+import java.security.GeneralSecurityException;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import javax.crypto.SecretKey;
+import javax.net.ssl.HttpsURLConnection;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.io.compress.CodecPool;
+import org.apache.hadoop.io.compress.CompressionCodec;
+import org.apache.hadoop.io.compress.Decompressor;
+import org.apache.hadoop.io.compress.DefaultCodec;
+import org.apache.hadoop.security.ssl.SSLFactory;
+import org.apache.hadoop.util.ReflectionUtils;
+import org.apache.tez.common.TezJobConfig;
+import org.apache.tez.common.counters.TezCounter;
+import org.apache.tez.runtime.api.TezInputContext;
+import org.apache.tez.runtime.library.common.ConfigUtils;
+import org.apache.tez.runtime.library.common.InputAttemptIdentifier;
+import org.apache.tez.runtime.library.common.security.SecureShuffleUtils;
+import org.apache.tez.runtime.library.common.shuffle.impl.MapOutput.Type;
+import org.apache.tez.runtime.library.common.sort.impl.IFileInputStream;
+
+import com.google.common.annotations.VisibleForTesting;
+
+class Fetcher extends Thread {
+  
+  private static final Log LOG = LogFactory.getLog(Fetcher.class);
+  
+  /** Basic/unit connection timeout (in milliseconds) */
+  private final static int UNIT_CONNECT_TIMEOUT = 60 * 1000;
+
+  private static enum ShuffleErrors{IO_ERROR, WRONG_LENGTH, BAD_ID, WRONG_MAP,
+                                    CONNECTION, WRONG_REDUCE}
+  
+  private final static String SHUFFLE_ERR_GRP_NAME = "Shuffle Errors";
+  private final TezCounter connectionErrs;
+  private final TezCounter ioErrs;
+  private final TezCounter wrongLengthErrs;
+  private final TezCounter badIdErrs;
+  private final TezCounter wrongMapErrs;
+  private final TezCounter wrongReduceErrs;
+  private final MergeManager merger;
+  private final ShuffleScheduler scheduler;
+  private final ShuffleClientMetrics metrics;
+  private final Shuffle shuffle;
+  private final int id;
+  private static int nextId = 0;
+  
+  private final int connectionTimeout;
+  private final int readTimeout;
+  
+  // Decompression of map-outputs
+  private final CompressionCodec codec;
+  private final Decompressor decompressor;
+  private final SecretKey jobTokenSecret;
+
+  private volatile boolean stopped = false;
+
+  private Configuration job;
+
+  private static boolean sslShuffle;
+  private static SSLFactory sslFactory;
+
+  public Fetcher(Configuration job, 
+      ShuffleScheduler scheduler, MergeManager merger,
+      ShuffleClientMetrics metrics,
+      Shuffle shuffle, SecretKey jobTokenSecret, TezInputContext inputContext) throws IOException {
+    this.job = job;
+    this.scheduler = scheduler;
+    this.merger = merger;
+    this.metrics = metrics;
+    this.shuffle = shuffle;
+    this.id = ++nextId;
+    this.jobTokenSecret = jobTokenSecret;
+    ioErrs = inputContext.getCounters().findCounter(SHUFFLE_ERR_GRP_NAME,
+        ShuffleErrors.IO_ERROR.toString());
+    wrongLengthErrs = inputContext.getCounters().findCounter(SHUFFLE_ERR_GRP_NAME,
+        ShuffleErrors.WRONG_LENGTH.toString());
+    badIdErrs = inputContext.getCounters().findCounter(SHUFFLE_ERR_GRP_NAME,
+        ShuffleErrors.BAD_ID.toString());
+    wrongMapErrs = inputContext.getCounters().findCounter(SHUFFLE_ERR_GRP_NAME,
+        ShuffleErrors.WRONG_MAP.toString());
+    connectionErrs = inputContext.getCounters().findCounter(SHUFFLE_ERR_GRP_NAME,
+        ShuffleErrors.CONNECTION.toString());
+    wrongReduceErrs = inputContext.getCounters().findCounter(SHUFFLE_ERR_GRP_NAME,
+        ShuffleErrors.WRONG_REDUCE.toString());
+
+    if (ConfigUtils.isIntermediateInputCompressed(job)) {
+      Class<? extends CompressionCodec> codecClass =
+          ConfigUtils.getIntermediateInputCompressorClass(job, DefaultCodec.class);
+      codec = ReflectionUtils.newInstance(codecClass, job);
+      decompressor = CodecPool.getDecompressor(codec);
+    } else {
+      codec = null;
+      decompressor = null;
+    }
+
+    this.connectionTimeout = 
+        job.getInt(TezJobConfig.TEZ_RUNTIME_SHUFFLE_CONNECT_TIMEOUT,
+            TezJobConfig.DEFAULT_TEZ_RUNTIME_SHUFFLE_STALLED_COPY_TIMEOUT);
+    this.readTimeout = 
+        job.getInt(TezJobConfig.TEZ_RUNTIME_SHUFFLE_READ_TIMEOUT, 
+            TezJobConfig.DEFAULT_TEZ_RUNTIME_SHUFFLE_READ_TIMEOUT);
+
+    setName("fetcher#" + id);
+    setDaemon(true);
+
+    synchronized (Fetcher.class) {
+      sslShuffle = job.getBoolean(TezJobConfig.TEZ_RUNTIME_SHUFFLE_ENABLE_SSL,
+          TezJobConfig.DEFAULT_TEZ_RUNTIME_SHUFFLE_ENABLE_SSL);
+      if (sslShuffle && sslFactory == null) {
+        sslFactory = new SSLFactory(SSLFactory.Mode.CLIENT, job);
+        try {
+          sslFactory.init();
+        } catch (Exception ex) {
+          sslFactory.destroy();
+          throw new RuntimeException(ex);
+        }
+      }
+    }
+  }
+  
+  public void run() {
+    try {
+      while (!stopped && !Thread.currentThread().isInterrupted()) {
+        MapHost host = null;
+        try {
+          // If merge is on, block
+          merger.waitForInMemoryMerge();
+
+          // Get a host to shuffle from
+          host = scheduler.getHost();
+          metrics.threadBusy();
+
+          // Shuffle
+          copyFromHost(host);
+        } finally {
+          if (host != null) {
+            scheduler.freeHost(host);
+            metrics.threadFree();            
+          }
+        }
+      }
+    } catch (InterruptedException ie) {
+      return;
+    } catch (Throwable t) {
+      shuffle.reportException(t);
+    }
+  }
+
+  public void shutDown() throws InterruptedException {
+    this.stopped = true;
+    interrupt();
+    try {
+      join(5000);
+    } catch (InterruptedException ie) {
+      LOG.warn("Got interrupt while joining " + getName(), ie);
+    }
+    if (sslFactory != null) {
+      sslFactory.destroy();
+    }
+  }
+
+  @VisibleForTesting
+  protected HttpURLConnection openConnection(URL url) throws IOException {
+    HttpURLConnection conn = (HttpURLConnection) url.openConnection();
+    if (sslShuffle) {
+      HttpsURLConnection httpsConn = (HttpsURLConnection) conn;
+      try {
+        httpsConn.setSSLSocketFactory(sslFactory.createSSLSocketFactory());
+      } catch (GeneralSecurityException ex) {
+        throw new IOException(ex);
+      }
+      httpsConn.setHostnameVerifier(sslFactory.getHostnameVerifier());
+    }
+    return conn;
+  }
+  
+  /**
+   * The crux of the matter...
+   * 
+   * @param host {@link MapHost} from which we need to  
+   *              shuffle available map-outputs.
+   */
+  @VisibleForTesting
+  protected void copyFromHost(MapHost host) throws IOException {
+    // Get completed maps on 'host'
+    List<InputAttemptIdentifier> srcAttempts = scheduler.getMapsForHost(host);
+    
+    // Sanity check to catch hosts with only 'OBSOLETE' maps, 
+    // especially at the tail of large jobs
+    if (srcAttempts.size() == 0) {
+      return;
+    }
+    
+    if(LOG.isDebugEnabled()) {
+      LOG.debug("Fetcher " + id + " going to fetch from " + host + " for: "
+        + srcAttempts);
+    }
+    
+    // List of maps to be fetched yet
+    Set<InputAttemptIdentifier> remaining = new HashSet<InputAttemptIdentifier>(srcAttempts);
+    
+    // Construct the url and connect
+    DataInputStream input;
+    boolean connectSucceeded = false;
+    
+    try {
+      URL url = getMapOutputURL(host, srcAttempts);
+      HttpURLConnection connection = openConnection(url);
+      
+      // generate hash of the url
+      String msgToEncode = SecureShuffleUtils.buildMsgFrom(url);
+      String encHash = SecureShuffleUtils.hashFromString(msgToEncode, jobTokenSecret);
+      
+      // put url hash into http header
+      connection.addRequestProperty(
+          SecureShuffleUtils.HTTP_HEADER_URL_HASH, encHash);
+      // set the read timeout
+      connection.setReadTimeout(readTimeout);
+      // put shuffle version into http header
+      connection.addRequestProperty(ShuffleHeader.HTTP_HEADER_NAME,
+          ShuffleHeader.DEFAULT_HTTP_HEADER_NAME);
+      connection.addRequestProperty(ShuffleHeader.HTTP_HEADER_VERSION,
+          ShuffleHeader.DEFAULT_HTTP_HEADER_VERSION);
+      connect(connection, connectionTimeout);
+      connectSucceeded = true;
+      input = new DataInputStream(connection.getInputStream());
+
+      // Validate response code
+      int rc = connection.getResponseCode();
+      if (rc != HttpURLConnection.HTTP_OK) {
+        throw new IOException(
+            "Got invalid response code " + rc + " from " + url +
+            ": " + connection.getResponseMessage());
+      }
+      // get the shuffle version
+      if (!ShuffleHeader.DEFAULT_HTTP_HEADER_NAME.equals(
+          connection.getHeaderField(ShuffleHeader.HTTP_HEADER_NAME))
+          || !ShuffleHeader.DEFAULT_HTTP_HEADER_VERSION.equals(
+              connection.getHeaderField(ShuffleHeader.HTTP_HEADER_VERSION))) {
+        throw new IOException("Incompatible shuffle response version");
+      }
+      // get the replyHash which is HMac of the encHash we sent to the server
+      String replyHash = connection.getHeaderField(SecureShuffleUtils.HTTP_HEADER_REPLY_URL_HASH);
+      if(replyHash==null) {
+        throw new IOException("security validation of TT Map output failed");
+      }
+      LOG.debug("url="+msgToEncode+";encHash="+encHash+";replyHash="+replyHash);
+      // verify that replyHash is HMac of encHash
+      SecureShuffleUtils.verifyReply(replyHash, encHash, jobTokenSecret);
+      LOG.info("for url="+msgToEncode+" sent hash and receievd reply");
+    } catch (IOException ie) {
+      ioErrs.increment(1);
+      LOG.warn("Failed to connect to " + host + " with " + remaining.size() + 
+               " map outputs", ie);
+
+      // If connect did not succeed, just mark all the maps as failed,
+      // indirectly penalizing the host
+      if (!connectSucceeded) {
+        for(InputAttemptIdentifier left: remaining) {
+          scheduler.copyFailed(left, host, connectSucceeded);
+        }
+      } else {
+        // If we got a read error at this stage, it implies there was a problem
+        // with the first map, typically lost map. So, penalize only that map
+        // and add the rest
+        InputAttemptIdentifier firstMap = srcAttempts.get(0);
+        scheduler.copyFailed(firstMap, host, connectSucceeded);
+      }
+      
+      // Add back all the remaining maps, WITHOUT marking them as failed
+      for(InputAttemptIdentifier left: remaining) {
+        // TODO Should the first one be skipped ?
+        scheduler.putBackKnownMapOutput(host, left);
+      }
+      
+      return;
+    }
+    
+    try {
+      // Loop through available map-outputs and fetch them
+      // On any error, faildTasks is not null and we exit
+      // after putting back the remaining maps to the 
+      // yet_to_be_fetched list and marking the failed tasks.
+      InputAttemptIdentifier[] failedTasks = null;
+      while (!remaining.isEmpty() && failedTasks == null) {
+        failedTasks = copyMapOutput(host, input, remaining);
+      }
+      
+      if(failedTasks != null && failedTasks.length > 0) {
+        LOG.warn("copyMapOutput failed for tasks "+Arrays.toString(failedTasks));
+        for(InputAttemptIdentifier left: failedTasks) {
+          scheduler.copyFailed(left, host, true);
+        }
+      }
+      
+      IOUtils.cleanup(LOG, input);
+      
+      // Sanity check
+      if (failedTasks == null && !remaining.isEmpty()) {
+        throw new IOException("server didn't return all expected map outputs: "
+            + remaining.size() + " left.");
+      }
+    } finally {
+      for (InputAttemptIdentifier left : remaining) {
+        scheduler.putBackKnownMapOutput(host, left);
+      }
+    }
+  }
+  
+  private static InputAttemptIdentifier[] EMPTY_ATTEMPT_ID_ARRAY = new InputAttemptIdentifier[0];
+  
+  private InputAttemptIdentifier[] copyMapOutput(MapHost host,
+                                DataInputStream input,
+                                Set<InputAttemptIdentifier> remaining) {
+    MapOutput mapOutput = null;
+    InputAttemptIdentifier srcAttemptId = null;
+    long decompressedLength = -1;
+    long compressedLength = -1;
+    
+    try {
+      long startTime = System.currentTimeMillis();
+      int forReduce = -1;
+      //Read the shuffle header
+      try {
+        ShuffleHeader header = new ShuffleHeader();
+        header.readFields(input);
+        String pathComponent = header.mapId;
+        srcAttemptId = scheduler.getIdentifierForPathComponent(pathComponent);
+        compressedLength = header.compressedLength;
+        decompressedLength = header.uncompressedLength;
+        forReduce = header.forReduce;
+      } catch (IllegalArgumentException e) {
+        badIdErrs.increment(1);
+        LOG.warn("Invalid map id ", e);
+        //Don't know which one was bad, so consider all of them as bad
+        return remaining.toArray(new InputAttemptIdentifier[remaining.size()]);
+      }
+
+ 
+      // Do some basic sanity verification
+      if (!verifySanity(compressedLength, decompressedLength, forReduce,
+          remaining, srcAttemptId)) {
+        return new InputAttemptIdentifier[] {srcAttemptId};
+      }
+      
+      if(LOG.isDebugEnabled()) {
+        LOG.debug("header: " + srcAttemptId + ", len: " + compressedLength + 
+            ", decomp len: " + decompressedLength);
+      }
+      
+      // Get the location for the map output - either in-memory or on-disk
+      mapOutput = merger.reserve(srcAttemptId, decompressedLength, id);
+      
+      // Check if we can shuffle *now* ...
+      if (mapOutput.getType() == Type.WAIT) {
+        LOG.info("fetcher#" + id + " - MergerManager returned Status.WAIT ...");
+        //Not an error but wait to process data.
+        return EMPTY_ATTEMPT_ID_ARRAY;
+      } 
+      
+      // Go!
+      LOG.info("fetcher#" + id + " about to shuffle output of map " + 
+               mapOutput.getAttemptIdentifier() + " decomp: " +
+               decompressedLength + " len: " + compressedLength + " to " +
+               mapOutput.getType());
+      if (mapOutput.getType() == Type.MEMORY) {
+        shuffleToMemory(host, mapOutput, input, 
+                        (int) decompressedLength, (int) compressedLength);
+      } else {
+        shuffleToDisk(host, mapOutput, input, compressedLength);
+      }
+      
+      // Inform the shuffle scheduler
+      long endTime = System.currentTimeMillis();
+      scheduler.copySucceeded(srcAttemptId, host, compressedLength, 
+                              endTime - startTime, mapOutput);
+      // Note successful shuffle
+      remaining.remove(srcAttemptId);
+      metrics.successFetch();
+      return null;
+    } catch (IOException ioe) {
+      ioErrs.increment(1);
+      if (srcAttemptId == null || mapOutput == null) {
+        LOG.info("fetcher#" + id + " failed to read map header" + 
+                 srcAttemptId + " decomp: " + 
+                 decompressedLength + ", " + compressedLength, ioe);
+        if(srcAttemptId == null) {
+          return remaining.toArray(new InputAttemptIdentifier[remaining.size()]);
+        } else {
+          return new InputAttemptIdentifier[] {srcAttemptId};
+        }
+      }
+      
+      LOG.warn("Failed to shuffle output of " + srcAttemptId + 
+               " from " + host.getHostName(), ioe); 
+
+      // Inform the shuffle-scheduler
+      mapOutput.abort();
+      metrics.failedFetch();
+      return new InputAttemptIdentifier[] {srcAttemptId};
+    }
+
+  }
+  
+  /**
+   * Do some basic verification on the input received -- Being defensive
+   * @param compressedLength
+   * @param decompressedLength
+   * @param forReduce
+   * @param remaining
+   * @param mapId
+   * @return true/false, based on if the verification succeeded or not
+   */
+  private boolean verifySanity(long compressedLength, long decompressedLength,
+      int forReduce, Set<InputAttemptIdentifier> remaining, InputAttemptIdentifier srcAttemptId) {
+    if (compressedLength < 0 || decompressedLength < 0) {
+      wrongLengthErrs.increment(1);
+      LOG.warn(getName() + " invalid lengths in map output header: id: " +
+          srcAttemptId + " len: " + compressedLength + ", decomp len: " + 
+               decompressedLength);
+      return false;
+    }
+    
+    int reduceStartId = shuffle.getReduceStartId();
+    int reduceRange = shuffle.getReduceRange();
+    if (forReduce < reduceStartId || forReduce >= reduceStartId+reduceRange) {
+      wrongReduceErrs.increment(1);
+      LOG.warn(getName() + " data for the wrong reduce map: " +
+               srcAttemptId + " len: " + compressedLength + " decomp len: " +
+               decompressedLength + " for reduce " + forReduce);
+      return false;
+    }
+
+    // Sanity check
+    if (!remaining.contains(srcAttemptId)) {
+      wrongMapErrs.increment(1);
+      LOG.warn("Invalid map-output! Received output for " + srcAttemptId);
+      return false;
+    }
+    
+    return true;
+  }
+
+  /**
+   * Create the map-output-url. This will contain all the map ids
+   * separated by commas
+   * @param host
+   * @param maps
+   * @return
+   * @throws MalformedURLException
+   */
+  private URL getMapOutputURL(MapHost host, List<InputAttemptIdentifier> srcAttempts
+                              )  throws MalformedURLException {
+    // Get the base url
+    StringBuffer url = new StringBuffer(host.getBaseUrl());
+    
+    boolean first = true;
+    for (InputAttemptIdentifier mapId : srcAttempts) {
+      if (!first) {
+        url.append(",");
+      }
+      url.append(mapId.getPathComponent());
+      first = false;
+    }
+   
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("MapOutput URL for " + host + " -> " + url.toString());
+    }
+    return new URL(url.toString());
+  }
+  
+  /** 
+   * The connection establishment is attempted multiple times and is given up 
+   * only on the last failure. Instead of connecting with a timeout of 
+   * X, we try connecting with a timeout of x < X but multiple times. 
+   */
+  private void connect(URLConnection connection, int connectionTimeout)
+  throws IOException {
+    int unit = 0;
+    if (connectionTimeout < 0) {
+      throw new IOException("Invalid timeout "
+                            + "[timeout = " + connectionTimeout + " ms]");
+    } else if (connectionTimeout > 0) {
+      unit = Math.min(UNIT_CONNECT_TIMEOUT, connectionTimeout);
+    }
+    // set the connect timeout to the unit-connect-timeout
+    connection.setConnectTimeout(unit);
+    while (true) {
+      try {
+        connection.connect();
+        break;
+      } catch (IOException ioe) {
+        // update the total remaining connect-timeout
+        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) {
+          throw ioe;
+        }
+
+        // reset the connect timeout for the last try
+        if (connectionTimeout < unit) {
+          unit = connectionTimeout;
+          // reset the connect time out for the final connect
+          connection.setConnectTimeout(unit);
+        }
+      }
+    }
+  }
+
+  private void shuffleToMemory(MapHost host, MapOutput mapOutput, 
+                               InputStream input, 
+                               int decompressedLength, 
+                               int compressedLength) throws IOException {    
+    IFileInputStream checksumIn = 
+      new IFileInputStream(input, compressedLength, job);
+
+    input = checksumIn;       
+  
+    // Are map-outputs compressed?
+    if (codec != null) {
+      decompressor.reset();
+      input = codec.createInputStream(input, decompressor);
+    }
+  
+    // Copy map-output into an in-memory buffer
+    byte[] shuffleData = mapOutput.getMemory();
+    
+    try {
+      IOUtils.readFully(input, shuffleData, 0, shuffleData.length);
+      metrics.inputBytes(shuffleData.length);
+      LOG.info("Read " + shuffleData.length + " bytes from map-output for " +
+               mapOutput.getAttemptIdentifier());
+    } catch (IOException ioe) {      
+      // Close the streams
+      IOUtils.cleanup(LOG, input);
+
+      // Re-throw
+      throw ioe;
+    }
+
+  }
+  
+  private void shuffleToDisk(MapHost host, MapOutput mapOutput, 
+                             InputStream input, 
+                             long compressedLength) 
+  throws IOException {
+    // Copy data to local-disk
+    OutputStream output = mapOutput.getDisk();
+    long bytesLeft = compressedLength;
+    try {
+      final int BYTES_TO_READ = 64 * 1024;
+      byte[] buf = new byte[BYTES_TO_READ];
+      while (bytesLeft > 0) {
+        int n = input.read(buf, 0, (int) Math.min(bytesLeft, BYTES_TO_READ));
+        if (n < 0) {
+          throw new IOException("read past end of stream reading " + 
+                                mapOutput.getAttemptIdentifier());
+        }
+        output.write(buf, 0, n);
+        bytesLeft -= n;
+        metrics.inputBytes(n);
+      }
+
+      LOG.info("Read " + (compressedLength - bytesLeft) + 
+               " bytes from map-output for " +
+               mapOutput.getAttemptIdentifier());
+
+      output.close();
+    } catch (IOException ioe) {
+      // Close the streams
+      IOUtils.cleanup(LOG, input, output);
+
+      // Re-throw
+      throw ioe;
+    }
+
+    // Sanity check
+    if (bytesLeft != 0) {
+      throw new IOException("Incomplete map output received for " +
+                            mapOutput.getAttemptIdentifier() + " from " +
+                            host.getHostName() + " (" + 
+                            bytesLeft + " bytes missing of " + 
+                            compressedLength + ")"
+      );
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/InMemoryReader.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/InMemoryReader.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/InMemoryReader.java
new file mode 100644
index 0000000..ae95268
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/InMemoryReader.java
@@ -0,0 +1,156 @@
+/**
+ * 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.library.common.shuffle.impl;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.io.DataInputBuffer;
+import org.apache.tez.runtime.library.common.InputAttemptIdentifier;
+import org.apache.tez.runtime.library.common.sort.impl.IFile;
+import org.apache.tez.runtime.library.common.sort.impl.IFile.Reader;
+
+/**
+ * <code>IFile.InMemoryReader</code> to read map-outputs present in-memory.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class InMemoryReader extends Reader {
+  private final InputAttemptIdentifier taskAttemptId;
+  private final MergeManager merger;
+  DataInputBuffer memDataIn = new DataInputBuffer();
+  private int start;
+  private int length;
+  private int prevKeyPos;
+
+  public InMemoryReader(MergeManager merger, InputAttemptIdentifier taskAttemptId,
+                        byte[] data, int start, int length)
+  throws IOException {
+    super(null, null, length - start, null, null);
+    this.merger = merger;
+    this.taskAttemptId = taskAttemptId;
+
+    buffer = data;
+    bufferSize = (int)fileLength;
+    memDataIn.reset(buffer, start, length);
+    this.start = start;
+    this.length = length;
+  }
+
+  @Override
+  public void reset(int offset) {
+    memDataIn.reset(buffer, start + offset, length);
+    bytesRead = offset;
+    eof = false;
+  }
+
+  @Override
+  public long getPosition() throws IOException {
+    // InMemoryReader does not initialize streams like Reader, so in.getPos()
+    // would not work. Instead, return the number of uncompressed bytes read,
+    // which will be correct since in-memory data is not compressed.
+    return bytesRead;
+  }
+  
+  @Override
+  public long getLength() { 
+    return fileLength;
+  }
+  
+  private void dumpOnError() {
+    File dumpFile = new File("../output/" + taskAttemptId + ".dump");
+    System.err.println("Dumping corrupt map-output of " + taskAttemptId + 
+                       " to " + dumpFile.getAbsolutePath());
+    try {
+      FileOutputStream fos = new FileOutputStream(dumpFile);
+      fos.write(buffer, 0, bufferSize);
+      fos.close();
+    } catch (IOException ioe) {
+      System.err.println("Failed to dump map-output of " + taskAttemptId);
+    }
+  }
+  
+  public KeyState readRawKey(DataInputBuffer key) throws IOException {
+    try {
+      if (!positionToNextRecord(memDataIn)) {
+        return KeyState.NO_KEY;
+      }
+      // Setup the key
+      int pos = memDataIn.getPosition();
+      byte[] data = memDataIn.getData();      
+      if(currentKeyLength == IFile.RLE_MARKER) {
+        key.reset(data, prevKeyPos, prevKeyLength);
+        currentKeyLength = prevKeyLength;
+        return KeyState.SAME_KEY;
+      }      
+      key.reset(data, pos, currentKeyLength);
+      prevKeyPos = pos;
+      // Position for the next value
+      long skipped = memDataIn.skip(currentKeyLength);
+      if (skipped != currentKeyLength) {
+        throw new IOException("Rec# " + recNo + 
+            ": Failed to skip past key of length: " + 
+            currentKeyLength);
+      }
+
+      // Record the byte
+      bytesRead += currentKeyLength;
+      return KeyState.NEW_KEY;
+    } catch (IOException ioe) {
+      dumpOnError();
+      throw ioe;
+    }
+  }
+  
+  public void nextRawValue(DataInputBuffer value) throws IOException {
+    try {
+      int pos = memDataIn.getPosition();
+      byte[] data = memDataIn.getData();
+      value.reset(data, pos, currentValueLength);
+
+      // Position for the next record
+      long skipped = memDataIn.skip(currentValueLength);
+      if (skipped != currentValueLength) {
+        throw new IOException("Rec# " + recNo + 
+            ": Failed to skip past value of length: " + 
+            currentValueLength);
+      }
+      // Record the byte
+      bytesRead += currentValueLength;
+
+      ++recNo;
+    } catch (IOException ioe) {
+      dumpOnError();
+      throw ioe;
+    }
+  }
+    
+  public void close() {
+    // Release
+    dataIn = null;
+    buffer = null;
+      // Inform the MergeManager
+    if (merger != null) {
+      merger.unreserve(bufferSize);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/InMemoryWriter.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/InMemoryWriter.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/InMemoryWriter.java
new file mode 100644
index 0000000..f81b28e
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/InMemoryWriter.java
@@ -0,0 +1,100 @@
+/**
+ * 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.library.common.shuffle.impl;
+
+import java.io.DataOutputStream;
+import java.io.IOException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.io.BoundedByteArrayOutputStream;
+import org.apache.hadoop.io.DataInputBuffer;
+import org.apache.hadoop.io.WritableUtils;
+import org.apache.tez.runtime.library.common.sort.impl.IFile;
+import org.apache.tez.runtime.library.common.sort.impl.IFileOutputStream;
+import org.apache.tez.runtime.library.common.sort.impl.IFile.Writer;
+
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class InMemoryWriter extends Writer {
+  private static final Log LOG = LogFactory.getLog(InMemoryWriter.class);
+
+  private DataOutputStream out;
+
+  public InMemoryWriter(BoundedByteArrayOutputStream arrayStream) {
+    super(null);
+    this.out =
+      new DataOutputStream(new IFileOutputStream(arrayStream));
+  }
+
+  public void append(Object key, Object value) throws IOException {
+    throw new UnsupportedOperationException
+    ("InMemoryWriter.append(K key, V value");
+  }
+
+  public void append(DataInputBuffer key, DataInputBuffer value)
+  throws IOException {
+    int keyLength = key.getLength() - key.getPosition();
+    if (keyLength < 0) {
+      throw new IOException("Negative key-length not allowed: " + keyLength +
+                            " for " + key);
+    }
+
+    boolean sameKey = (key == IFile.REPEAT_KEY);
+
+    int valueLength = value.getLength() - value.getPosition();
+    if (valueLength < 0) {
+      throw new IOException("Negative value-length not allowed: " +
+                            valueLength + " for " + value);
+    }
+
+    if(sameKey) {
+      WritableUtils.writeVInt(out, IFile.RLE_MARKER);
+      WritableUtils.writeVInt(out, valueLength);
+      out.write(value.getData(), value.getPosition(), valueLength);
+    } else {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("InMemWriter.append" +
+            " key.data=" + key.getData() +
+            " key.pos=" + key.getPosition() +
+            " key.len=" +key.getLength() +
+            " val.data=" + value.getData() +
+            " val.pos=" + value.getPosition() +
+            " val.len=" + value.getLength());
+      }
+      WritableUtils.writeVInt(out, keyLength);
+      WritableUtils.writeVInt(out, valueLength);
+      out.write(key.getData(), key.getPosition(), keyLength);
+      out.write(value.getData(), value.getPosition(), valueLength);
+    }
+
+  }
+
+  public void close() throws IOException {
+    // Write EOF_MARKER for key/value length
+    WritableUtils.writeVInt(out, IFile.EOF_MARKER);
+    WritableUtils.writeVInt(out, IFile.EOF_MARKER);
+
+    // Close the stream
+    out.close();
+    out = null;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/MapHost.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/MapHost.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/MapHost.java
new file mode 100644
index 0000000..b8be657
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/MapHost.java
@@ -0,0 +1,124 @@
+/**
+ * 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.library.common.shuffle.impl;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.tez.runtime.library.common.InputAttemptIdentifier;
+
+@Private
+class MapHost {
+  
+  public static enum State {
+    IDLE,               // No map outputs available
+    BUSY,               // Map outputs are being fetched
+    PENDING,            // Known map outputs which need to be fetched
+    PENALIZED           // Host penalized due to shuffle failures
+  }
+  
+  private State state = State.IDLE;
+  private final String hostName;
+  private final int partitionId;
+  private final String baseUrl;
+  private final String identifier;
+  // Tracks attempt IDs
+  private List<InputAttemptIdentifier> maps = new ArrayList<InputAttemptIdentifier>();
+  
+  public MapHost(int partitionId, String hostName, String baseUrl) {
+    this.partitionId = partitionId;
+    this.hostName = hostName;
+    this.baseUrl = baseUrl;
+    this.identifier = createIdentifier(hostName, partitionId);
+  }
+  
+  public static String createIdentifier(String hostName, int partitionId) {
+    return hostName + ":" + Integer.toString(partitionId);
+  }
+  
+  public String getIdentifier() {
+    return identifier;
+  }
+  
+  public int getPartitionId() {
+    return partitionId;
+  }
+
+  public State getState() {
+    return state;
+  }
+
+  public String getHostName() {
+    return hostName;
+  }
+
+  public String getBaseUrl() {
+    return baseUrl;
+  }
+
+  public synchronized void addKnownMap(InputAttemptIdentifier srcAttempt) {
+    maps.add(srcAttempt);
+    if (state == State.IDLE) {
+      state = State.PENDING;
+    }
+  }
+
+  public synchronized List<InputAttemptIdentifier> getAndClearKnownMaps() {
+    List<InputAttemptIdentifier> currentKnownMaps = maps;
+    maps = new ArrayList<InputAttemptIdentifier>();
+    return currentKnownMaps;
+  }
+  
+  public synchronized void markBusy() {
+    state = State.BUSY;
+  }
+  
+  public synchronized void markPenalized() {
+    state = State.PENALIZED;
+  }
+  
+  public synchronized int getNumKnownMapOutputs() {
+    return maps.size();
+  }
+
+  /**
+   * Called when the node is done with its penalty or done copying.
+   * @return the host's new state
+   */
+  public synchronized State markAvailable() {
+    if (maps.isEmpty()) {
+      state = State.IDLE;
+    } else {
+      state = State.PENDING;
+    }
+    return state;
+  }
+  
+  @Override
+  public String toString() {
+    return hostName;
+  }
+  
+  /**
+   * Mark the host as penalized
+   */
+  public synchronized void penalize() {
+    state = State.PENALIZED;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/MapOutput.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/MapOutput.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/MapOutput.java
new file mode 100644
index 0000000..9f673a0
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/MapOutput.java
@@ -0,0 +1,227 @@
+/**
+ * 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.library.common.shuffle.impl;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.Comparator;
+import java.util.concurrent.atomic.AtomicInteger;
+
+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.io.BoundedByteArrayOutputStream;
+import org.apache.tez.runtime.library.common.InputAttemptIdentifier;
+import org.apache.tez.runtime.library.common.task.local.output.TezTaskOutputFiles;
+
+
+class MapOutput {
+  private static final Log LOG = LogFactory.getLog(MapOutput.class);
+  private static AtomicInteger ID = new AtomicInteger(0);
+  
+  public static enum Type {
+    WAIT,
+    MEMORY,
+    DISK
+  }
+  
+  private InputAttemptIdentifier attemptIdentifier;
+  private final int id;
+  
+  private final MergeManager merger;
+  
+  private final long size;
+  
+  private final byte[] memory;
+  private BoundedByteArrayOutputStream byteStream;
+  
+  private final FileSystem localFS;
+  private final Path tmpOutputPath;
+  private final Path outputPath;
+  private final OutputStream disk; 
+  
+  private final Type type;
+  
+  private final boolean primaryMapOutput;
+  
+  MapOutput(InputAttemptIdentifier attemptIdentifier, MergeManager merger, long size, 
+            Configuration conf, LocalDirAllocator localDirAllocator,
+            int fetcher, boolean primaryMapOutput, 
+            TezTaskOutputFiles mapOutputFile)
+         throws IOException {
+    this.id = ID.incrementAndGet();
+    this.attemptIdentifier = attemptIdentifier;
+    this.merger = merger;
+
+    type = Type.DISK;
+
+    memory = null;
+    byteStream = null;
+
+    this.size = size;
+    
+    this.localFS = FileSystem.getLocal(conf);
+    outputPath =
+      mapOutputFile.getInputFileForWrite(this.attemptIdentifier.getInputIdentifier().getSrcTaskIndex(), size);
+    tmpOutputPath = outputPath.suffix(String.valueOf(fetcher));
+
+    disk = localFS.create(tmpOutputPath);
+    
+    this.primaryMapOutput = primaryMapOutput;
+  }
+  
+  MapOutput(InputAttemptIdentifier attemptIdentifier, MergeManager merger, int size, 
+            boolean primaryMapOutput) {
+    this.id = ID.incrementAndGet();
+    this.attemptIdentifier = attemptIdentifier;
+    this.merger = merger;
+
+    type = Type.MEMORY;
+    byteStream = new BoundedByteArrayOutputStream(size);
+    memory = byteStream.getBuffer();
+
+    this.size = size;
+    
+    localFS = null;
+    disk = null;
+    outputPath = null;
+    tmpOutputPath = null;
+    
+    this.primaryMapOutput = primaryMapOutput;
+  }
+
+  public MapOutput(InputAttemptIdentifier attemptIdentifier) {
+    this.id = ID.incrementAndGet();
+    this.attemptIdentifier = attemptIdentifier;
+
+    type = Type.WAIT;
+    merger = null;
+    memory = null;
+    byteStream = null;
+    
+    size = -1;
+    
+    localFS = null;
+    disk = null;
+    outputPath = null;
+    tmpOutputPath = null;
+
+    this.primaryMapOutput = false;
+}
+  
+  public boolean isPrimaryMapOutput() {
+    return primaryMapOutput;
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (obj instanceof MapOutput) {
+      return id == ((MapOutput)obj).id;
+    }
+    return false;
+  }
+
+  @Override
+  public int hashCode() {
+    return id;
+  }
+
+  public Path getOutputPath() {
+    return outputPath;
+  }
+
+  public byte[] getMemory() {
+    return memory;
+  }
+
+  public BoundedByteArrayOutputStream getArrayStream() {
+    return byteStream;
+  }
+  
+  public OutputStream getDisk() {
+    return disk;
+  }
+
+  public InputAttemptIdentifier getAttemptIdentifier() {
+    return this.attemptIdentifier;
+  }
+
+  public Type getType() {
+    return type;
+  }
+
+  public long getSize() {
+    return size;
+  }
+
+  public void commit() throws IOException {
+    if (type == Type.MEMORY) {
+      merger.closeInMemoryFile(this);
+    } else if (type == Type.DISK) {
+      localFS.rename(tmpOutputPath, outputPath);
+      merger.closeOnDiskFile(outputPath);
+    } else {
+      throw new IOException("Cannot commit MapOutput of type WAIT!");
+    }
+  }
+  
+  public void abort() {
+    if (type == Type.MEMORY) {
+      merger.unreserve(memory.length);
+    } else if (type == Type.DISK) {
+      try {
+        localFS.delete(tmpOutputPath, false);
+      } catch (IOException ie) {
+        LOG.info("failure to clean up " + tmpOutputPath, ie);
+      }
+    } else {
+      throw new IllegalArgumentException
+                   ("Cannot commit MapOutput with of type WAIT!");
+    }
+  }
+  
+  public String toString() {
+    return "MapOutput( AttemptIdentifier: " + attemptIdentifier + ", Type: " + type + ")";
+  }
+  
+  public static class MapOutputComparator 
+  implements Comparator<MapOutput> {
+    public int compare(MapOutput o1, MapOutput o2) {
+      if (o1.id == o2.id) { 
+        return 0;
+      }
+      
+      if (o1.size < o2.size) {
+        return -1;
+      } else if (o1.size > o2.size) {
+        return 1;
+      }
+      
+      if (o1.id < o2.id) {
+        return -1;
+      } else {
+        return 1;
+      
+      }
+    }
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/MergeManager.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/MergeManager.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/MergeManager.java
new file mode 100644
index 0000000..0abe530
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/MergeManager.java
@@ -0,0 +1,782 @@
+/**
+ * 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.library.common.shuffle.impl;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Set;
+import java.util.TreeSet;
+
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.ChecksumFileSystem;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.LocalDirAllocator;
+import org.apache.hadoop.fs.LocalFileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.DataInputBuffer;
+import org.apache.hadoop.io.RawComparator;
+import org.apache.hadoop.io.compress.CompressionCodec;
+import org.apache.hadoop.io.compress.DefaultCodec;
+import org.apache.hadoop.util.Progressable;
+import org.apache.hadoop.util.ReflectionUtils;
+import org.apache.tez.common.TezJobConfig;
+import org.apache.tez.common.counters.TezCounter;
+import org.apache.tez.runtime.api.TezInputContext;
+import org.apache.tez.runtime.library.common.ConfigUtils;
+import org.apache.tez.runtime.library.common.Constants;
+import org.apache.tez.runtime.library.common.InputAttemptIdentifier;
+import org.apache.tez.runtime.library.common.combine.Combiner;
+import org.apache.tez.runtime.library.common.sort.impl.IFile;
+import org.apache.tez.runtime.library.common.sort.impl.TezMerger;
+import org.apache.tez.runtime.library.common.sort.impl.TezRawKeyValueIterator;
+import org.apache.tez.runtime.library.common.sort.impl.IFile.Writer;
+import org.apache.tez.runtime.library.common.sort.impl.TezMerger.Segment;
+import org.apache.tez.runtime.library.common.task.local.output.TezTaskOutputFiles;
+import org.apache.tez.runtime.library.hadoop.compat.NullProgressable;
+
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+@SuppressWarnings(value={"rawtypes"})
+public class MergeManager {
+  
+  private static final Log LOG = LogFactory.getLog(MergeManager.class);
+
+  private final Configuration conf;
+  private final FileSystem localFS;
+  private final FileSystem rfs;
+  private final LocalDirAllocator localDirAllocator;
+  
+  private final  TezTaskOutputFiles mapOutputFile;
+  private final Progressable nullProgressable = new NullProgressable();
+  private final Combiner combiner;  
+  
+  Set<MapOutput> inMemoryMergedMapOutputs = 
+    new TreeSet<MapOutput>(new MapOutput.MapOutputComparator());
+  private final IntermediateMemoryToMemoryMerger memToMemMerger;
+
+  Set<MapOutput> inMemoryMapOutputs = 
+    new TreeSet<MapOutput>(new MapOutput.MapOutputComparator());
+  private final InMemoryMerger inMemoryMerger;
+  
+  Set<Path> onDiskMapOutputs = new TreeSet<Path>();
+  private final OnDiskMerger onDiskMerger;
+  
+  private final long memoryLimit;
+  private long usedMemory;
+  private long commitMemory;
+  private final long maxSingleShuffleLimit;
+  
+  private final int memToMemMergeOutputsThreshold; 
+  private final long mergeThreshold;
+  
+  private final int ioSortFactor;
+
+  private final ExceptionReporter exceptionReporter;
+  
+  private final TezInputContext inputContext;
+
+  private final TezCounter spilledRecordsCounter;
+
+  private final TezCounter reduceCombineInputCounter;
+
+  private final TezCounter mergedMapOutputsCounter;
+  
+  private final CompressionCodec codec;
+  
+  private volatile boolean finalMergeComplete = false;
+
+  public MergeManager(Configuration conf, 
+                      FileSystem localFS,
+                      LocalDirAllocator localDirAllocator,  
+                      TezInputContext inputContext,
+                      Combiner combiner,
+                      TezCounter spilledRecordsCounter,
+                      TezCounter reduceCombineInputCounter,
+                      TezCounter mergedMapOutputsCounter,
+                      ExceptionReporter exceptionReporter) {
+    this.inputContext = inputContext;
+    this.conf = conf;
+    this.localDirAllocator = localDirAllocator;
+    this.exceptionReporter = exceptionReporter;
+    
+    this.combiner = combiner;
+
+    this.reduceCombineInputCounter = reduceCombineInputCounter;
+    this.spilledRecordsCounter = spilledRecordsCounter;
+    this.mergedMapOutputsCounter = mergedMapOutputsCounter;
+    this.mapOutputFile = new TezTaskOutputFiles(conf, inputContext.getUniqueIdentifier());
+    
+    this.localFS = localFS;
+    this.rfs = ((LocalFileSystem)localFS).getRaw();
+
+    if (ConfigUtils.isIntermediateInputCompressed(conf)) {
+      Class<? extends CompressionCodec> codecClass =
+          ConfigUtils.getIntermediateInputCompressorClass(conf, DefaultCodec.class);
+      codec = ReflectionUtils.newInstance(codecClass, conf);
+    } else {
+      codec = null;
+    }
+
+    final float maxInMemCopyUse =
+      conf.getFloat(
+          TezJobConfig.TEZ_RUNTIME_SHUFFLE_INPUT_BUFFER_PERCENT, 
+          TezJobConfig.DEFAULT_TEZ_RUNTIME_SHUFFLE_INPUT_BUFFER_PERCENT);
+    if (maxInMemCopyUse > 1.0 || maxInMemCopyUse < 0.0) {
+      throw new IllegalArgumentException("Invalid value for " +
+          TezJobConfig.TEZ_RUNTIME_SHUFFLE_INPUT_BUFFER_PERCENT + ": " +
+          maxInMemCopyUse);
+    }
+
+    // Allow unit tests to fix Runtime memory
+    this.memoryLimit = 
+      (long)(conf.getLong(Constants.TEZ_RUNTIME_TASK_MEMORY,
+          Math.min(Runtime.getRuntime().maxMemory(), Integer.MAX_VALUE))
+        * maxInMemCopyUse);
+ 
+    this.ioSortFactor = 
+        conf.getInt(
+            TezJobConfig.TEZ_RUNTIME_IO_SORT_FACTOR, 
+            TezJobConfig.DEFAULT_TEZ_RUNTIME_IO_SORT_FACTOR);
+
+    final float singleShuffleMemoryLimitPercent =
+        conf.getFloat(
+            TezJobConfig.TEZ_RUNTIME_SHUFFLE_MEMORY_LIMIT_PERCENT,
+            TezJobConfig.DEFAULT_TEZ_RUNTIME_SHUFFLE_MEMORY_LIMIT_PERCENT);
+    if (singleShuffleMemoryLimitPercent <= 0.0f
+        || singleShuffleMemoryLimitPercent > 1.0f) {
+      throw new IllegalArgumentException("Invalid value for "
+          + TezJobConfig.TEZ_RUNTIME_SHUFFLE_MEMORY_LIMIT_PERCENT + ": "
+          + singleShuffleMemoryLimitPercent);
+    }
+
+    this.maxSingleShuffleLimit = 
+      (long)(memoryLimit * singleShuffleMemoryLimitPercent);
+    this.memToMemMergeOutputsThreshold = 
+            conf.getInt(
+                TezJobConfig.TEZ_RUNTIME_SHUFFLE_MEMTOMEM_SEGMENTS, 
+                ioSortFactor);
+    this.mergeThreshold = 
+        (long)(this.memoryLimit * 
+               conf.getFloat(
+                   TezJobConfig.TEZ_RUNTIME_SHUFFLE_MERGE_PERCENT, 
+                   TezJobConfig.DEFAULT_TEZ_RUNTIME_SHUFFLE_MERGE_PERCENT));
+    LOG.info("MergerManager: memoryLimit=" + memoryLimit + ", " +
+             "maxSingleShuffleLimit=" + maxSingleShuffleLimit + ", " +
+             "mergeThreshold=" + mergeThreshold + ", " + 
+             "ioSortFactor=" + ioSortFactor + ", " +
+             "memToMemMergeOutputsThreshold=" + memToMemMergeOutputsThreshold);
+
+    if (this.maxSingleShuffleLimit >= this.mergeThreshold) {
+      throw new RuntimeException("Invlaid configuration: "
+          + "maxSingleShuffleLimit should be less than mergeThreshold"
+          + "maxSingleShuffleLimit: " + this.maxSingleShuffleLimit
+          + "mergeThreshold: " + this.mergeThreshold);
+    }
+
+    boolean allowMemToMemMerge = 
+      conf.getBoolean(
+          TezJobConfig.TEZ_RUNTIME_SHUFFLE_ENABLE_MEMTOMEM, 
+          TezJobConfig.DEFAULT_TEZ_RUNTIME_SHUFFLE_ENABLE_MEMTOMEM);
+    if (allowMemToMemMerge) {
+      this.memToMemMerger = 
+        new IntermediateMemoryToMemoryMerger(this,
+                                             memToMemMergeOutputsThreshold);
+      this.memToMemMerger.start();
+    } else {
+      this.memToMemMerger = null;
+    }
+    
+    this.inMemoryMerger = new InMemoryMerger(this);
+    this.inMemoryMerger.start();
+    
+    this.onDiskMerger = new OnDiskMerger(this);
+    this.onDiskMerger.start();
+  }
+
+  public void waitForInMemoryMerge() throws InterruptedException {
+    inMemoryMerger.waitForMerge();
+  }
+  
+  private boolean canShuffleToMemory(long requestedSize) {
+    return (requestedSize < maxSingleShuffleLimit); 
+  }
+
+  final private MapOutput stallShuffle = new MapOutput(null);
+
+  public synchronized MapOutput reserve(InputAttemptIdentifier srcAttemptIdentifier, 
+                                             long requestedSize,
+                                             int fetcher
+                                             ) throws IOException {
+    if (!canShuffleToMemory(requestedSize)) {
+      LOG.info(srcAttemptIdentifier + ": Shuffling to disk since " + requestedSize + 
+               " is greater than maxSingleShuffleLimit (" + 
+               maxSingleShuffleLimit + ")");
+      return new MapOutput(srcAttemptIdentifier, this, requestedSize, conf, 
+                                localDirAllocator, fetcher, true,
+                                mapOutputFile);
+    }
+    
+    // Stall shuffle if we are above the memory limit
+
+    // It is possible that all threads could just be stalling and not make
+    // progress at all. This could happen when:
+    //
+    // requested size is causing the used memory to go above limit &&
+    // requested size < singleShuffleLimit &&
+    // current used size < mergeThreshold (merge will not get triggered)
+    //
+    // To avoid this from happening, we allow exactly one thread to go past
+    // the memory limit. We check (usedMemory > memoryLimit) and not
+    // (usedMemory + requestedSize > memoryLimit). When this thread is done
+    // fetching, this will automatically trigger a merge thereby unlocking
+    // all the stalled threads
+    
+    if (usedMemory > memoryLimit) {
+      LOG.debug(srcAttemptIdentifier + ": Stalling shuffle since usedMemory (" + usedMemory
+          + ") is greater than memoryLimit (" + memoryLimit + ")." + 
+          " CommitMemory is (" + commitMemory + ")"); 
+      return stallShuffle;
+    }
+    
+    // Allow the in-memory shuffle to progress
+    LOG.debug(srcAttemptIdentifier + ": Proceeding with shuffle since usedMemory ("
+        + usedMemory + ") is lesser than memoryLimit (" + memoryLimit + ")."
+        + "CommitMemory is (" + commitMemory + ")"); 
+    return unconditionalReserve(srcAttemptIdentifier, requestedSize, true);
+  }
+  
+  /**
+   * Unconditional Reserve is used by the Memory-to-Memory thread
+   * @return
+   */
+  private synchronized MapOutput unconditionalReserve(
+      InputAttemptIdentifier srcAttemptIdentifier, long requestedSize, boolean primaryMapOutput) {
+    usedMemory += requestedSize;
+    return new MapOutput(srcAttemptIdentifier, this, (int)requestedSize, 
+        primaryMapOutput);
+  }
+  
+  synchronized void unreserve(long size) {
+    commitMemory -= size;
+    usedMemory -= size;
+  }
+
+  public synchronized void closeInMemoryFile(MapOutput mapOutput) { 
+    inMemoryMapOutputs.add(mapOutput);
+    LOG.info("closeInMemoryFile -> map-output of size: " + mapOutput.getSize()
+        + ", inMemoryMapOutputs.size() -> " + inMemoryMapOutputs.size()
+        + ", commitMemory -> " + commitMemory + ", usedMemory ->" + usedMemory);
+
+    commitMemory+= mapOutput.getSize();
+
+    synchronized (inMemoryMerger) {
+      // Can hang if mergeThreshold is really low.
+      if (!inMemoryMerger.isInProgress() && commitMemory >= mergeThreshold) {
+        LOG.info("Starting inMemoryMerger's merge since commitMemory=" +
+            commitMemory + " > mergeThreshold=" + mergeThreshold + 
+            ". Current usedMemory=" + usedMemory);
+        inMemoryMapOutputs.addAll(inMemoryMergedMapOutputs);
+        inMemoryMergedMapOutputs.clear();
+        inMemoryMerger.startMerge(inMemoryMapOutputs);
+      } 
+    }
+    
+    if (memToMemMerger != null) {
+      synchronized (memToMemMerger) {
+        if (!memToMemMerger.isInProgress() && 
+            inMemoryMapOutputs.size() >= memToMemMergeOutputsThreshold) {
+          memToMemMerger.startMerge(inMemoryMapOutputs);
+        }
+      }
+    }
+  }
+  
+  
+  public synchronized void closeInMemoryMergedFile(MapOutput mapOutput) {
+    inMemoryMergedMapOutputs.add(mapOutput);
+    LOG.info("closeInMemoryMergedFile -> size: " + mapOutput.getSize() + 
+             ", inMemoryMergedMapOutputs.size() -> " + 
+             inMemoryMergedMapOutputs.size());
+  }
+  
+  public synchronized void closeOnDiskFile(Path file) {
+    onDiskMapOutputs.add(file);
+    
+    synchronized (onDiskMerger) {
+      if (!onDiskMerger.isInProgress() && 
+          onDiskMapOutputs.size() >= (2 * ioSortFactor - 1)) {
+        onDiskMerger.startMerge(onDiskMapOutputs);
+      }
+    }
+  }
+
+  /**
+   * Should <b>only</b> be used after the Shuffle phaze is complete, otherwise can
+   * return an invalid state since a merge may not be in progress dur to
+   * inadequate inputs
+   * 
+   * @return true if the merge process is complete, otherwise false
+   */
+  @Private
+  public boolean isMergeComplete() {
+    return finalMergeComplete;
+  }
+  
+  public TezRawKeyValueIterator close() throws Throwable {
+    // Wait for on-going merges to complete
+    if (memToMemMerger != null) { 
+      memToMemMerger.close();
+    }
+    inMemoryMerger.close();
+    onDiskMerger.close();
+    
+    List<MapOutput> memory = 
+      new ArrayList<MapOutput>(inMemoryMergedMapOutputs);
+    memory.addAll(inMemoryMapOutputs);
+    List<Path> disk = new ArrayList<Path>(onDiskMapOutputs);
+    TezRawKeyValueIterator kvIter = finalMerge(conf, rfs, memory, disk);
+    this.finalMergeComplete = true;
+    return kvIter;
+  }
+   
+  void runCombineProcessor(TezRawKeyValueIterator kvIter, Writer writer)
+      throws IOException, InterruptedException {
+    combiner.combine(kvIter, writer);
+  }
+
+  private class IntermediateMemoryToMemoryMerger 
+  extends MergeThread<MapOutput> {
+    
+    public IntermediateMemoryToMemoryMerger(MergeManager manager, 
+                                            int mergeFactor) {
+      super(manager, mergeFactor, exceptionReporter);
+      setName("InMemoryMerger - Thread to do in-memory merge of in-memory " +
+      		    "shuffled map-outputs");
+      setDaemon(true);
+    }
+
+    @Override
+    public void merge(List<MapOutput> inputs) throws IOException {
+      if (inputs == null || inputs.size() == 0) {
+        return;
+      }
+
+      InputAttemptIdentifier dummyMapId = inputs.get(0).getAttemptIdentifier(); 
+      List<Segment> inMemorySegments = new ArrayList<Segment>();
+      long mergeOutputSize = 
+        createInMemorySegments(inputs, inMemorySegments, 0);
+      int noInMemorySegments = inMemorySegments.size();
+      
+      MapOutput mergedMapOutputs = 
+        unconditionalReserve(dummyMapId, mergeOutputSize, false);
+      
+      Writer writer = 
+        new InMemoryWriter(mergedMapOutputs.getArrayStream());
+      
+      LOG.info("Initiating Memory-to-Memory merge with " + noInMemorySegments +
+               " segments of total-size: " + mergeOutputSize);
+
+      TezRawKeyValueIterator rIter = 
+        TezMerger.merge(conf, rfs,
+                       ConfigUtils.getIntermediateInputKeyClass(conf),
+                       ConfigUtils.getIntermediateInputValueClass(conf),
+                       inMemorySegments, inMemorySegments.size(),
+                       new Path(inputContext.getUniqueIdentifier()),
+                       (RawComparator)ConfigUtils.getIntermediateInputKeyComparator(conf),
+                       nullProgressable, null, null, null);
+      TezMerger.writeFile(rIter, writer, nullProgressable, conf);
+      writer.close();
+
+      LOG.info(inputContext.getUniqueIdentifier() +  
+               " Memory-to-Memory merge of the " + noInMemorySegments +
+               " files in-memory complete.");
+
+      // Note the output of the merge
+      closeInMemoryMergedFile(mergedMapOutputs);
+    }
+  }
+  
+  private class InMemoryMerger extends MergeThread<MapOutput> {
+    
+    public InMemoryMerger(MergeManager manager) {
+      super(manager, Integer.MAX_VALUE, exceptionReporter);
+      setName
+      ("InMemoryMerger - Thread to merge in-memory shuffled map-outputs");
+      setDaemon(true);
+    }
+    
+    @Override
+    public void merge(List<MapOutput> inputs) throws IOException, InterruptedException {
+      if (inputs == null || inputs.size() == 0) {
+        return;
+      }
+      
+      //name this output file same as the name of the first file that is 
+      //there in the current list of inmem files (this is guaranteed to
+      //be absent on the disk currently. So we don't overwrite a prev. 
+      //created spill). Also we need to create the output file now since
+      //it is not guaranteed that this file will be present after merge
+      //is called (we delete empty files as soon as we see them
+      //in the merge method)
+
+      //figure out the mapId 
+      InputAttemptIdentifier srcTaskIdentifier = inputs.get(0).getAttemptIdentifier();
+
+      List<Segment> inMemorySegments = new ArrayList<Segment>();
+      long mergeOutputSize = 
+        createInMemorySegments(inputs, inMemorySegments,0);
+      int noInMemorySegments = inMemorySegments.size();
+
+      Path outputPath = mapOutputFile.getInputFileForWrite(
+          srcTaskIdentifier.getInputIdentifier().getSrcTaskIndex(),
+          mergeOutputSize).suffix(Constants.MERGED_OUTPUT_PREFIX);
+
+      Writer writer = null;
+      try {
+        writer =
+            new Writer(conf, rfs, outputPath,
+                (Class)ConfigUtils.getIntermediateInputKeyClass(conf),
+                (Class)ConfigUtils.getIntermediateInputValueClass(conf),
+                codec, null);
+
+        TezRawKeyValueIterator rIter = null;
+        LOG.info("Initiating in-memory merge with " + noInMemorySegments + 
+            " segments...");
+
+        rIter = TezMerger.merge(conf, rfs,
+            (Class)ConfigUtils.getIntermediateInputKeyClass(conf),
+            (Class)ConfigUtils.getIntermediateInputValueClass(conf),
+            inMemorySegments, inMemorySegments.size(),
+            new Path(inputContext.getUniqueIdentifier()),
+            (RawComparator)ConfigUtils.getIntermediateInputKeyComparator(conf),
+            nullProgressable, spilledRecordsCounter, null, null);
+
+        if (null == combiner) {
+          TezMerger.writeFile(rIter, writer, nullProgressable, conf);
+        } else {
+          runCombineProcessor(rIter, writer);
+        }
+        writer.close();
+        writer = null;
+
+        LOG.info(inputContext.getUniqueIdentifier() +  
+            " Merge of the " + noInMemorySegments +
+            " files in-memory complete." +
+            " Local file is " + outputPath + " of size " + 
+            localFS.getFileStatus(outputPath).getLen());
+      } catch (IOException e) { 
+        //make sure that we delete the ondisk file that we created 
+        //earlier when we invoked cloneFileAttributes
+        localFS.delete(outputPath, true);
+        throw e;
+      } finally {
+        if (writer != null) {
+          writer.close();
+        }
+      }
+
+      // Note the output of the merge
+      closeOnDiskFile(outputPath);
+    }
+
+  }
+  
+  private class OnDiskMerger extends MergeThread<Path> {
+    
+    public OnDiskMerger(MergeManager manager) {
+      super(manager, Integer.MAX_VALUE, exceptionReporter);
+      setName("OnDiskMerger - Thread to merge on-disk map-outputs");
+      setDaemon(true);
+    }
+    
+    @Override
+    public void merge(List<Path> inputs) throws IOException {
+      // sanity check
+      if (inputs == null || inputs.isEmpty()) {
+        LOG.info("No ondisk files to merge...");
+        return;
+      }
+      
+      long approxOutputSize = 0;
+      int bytesPerSum = 
+        conf.getInt("io.bytes.per.checksum", 512);
+      
+      LOG.info("OnDiskMerger: We have  " + inputs.size() + 
+               " map outputs on disk. Triggering merge...");
+      
+      // 1. Prepare the list of files to be merged. 
+      for (Path file : inputs) {
+        approxOutputSize += localFS.getFileStatus(file).getLen();
+      }
+
+      // add the checksum length
+      approxOutputSize += 
+        ChecksumFileSystem.getChecksumLength(approxOutputSize, bytesPerSum);
+
+      // 2. Start the on-disk merge process
+      Path outputPath = 
+        localDirAllocator.getLocalPathForWrite(inputs.get(0).toString(), 
+            approxOutputSize, conf).suffix(Constants.MERGED_OUTPUT_PREFIX);
+      Writer writer = 
+        new Writer(conf, rfs, outputPath, 
+                        (Class)ConfigUtils.getIntermediateInputKeyClass(conf), 
+                        (Class)ConfigUtils.getIntermediateInputValueClass(conf),
+                        codec, null);
+      TezRawKeyValueIterator iter  = null;
+      Path tmpDir = new Path(inputContext.getUniqueIdentifier());
+      try {
+        iter = TezMerger.merge(conf, rfs,
+                            (Class)ConfigUtils.getIntermediateInputKeyClass(conf), 
+                            (Class)ConfigUtils.getIntermediateInputValueClass(conf),
+                            codec, inputs.toArray(new Path[inputs.size()]), 
+                            true, ioSortFactor, tmpDir, 
+                            (RawComparator)ConfigUtils.getIntermediateInputKeyComparator(conf), 
+                            nullProgressable, spilledRecordsCounter, null, 
+                            mergedMapOutputsCounter, null);
+
+        TezMerger.writeFile(iter, writer, nullProgressable, conf);
+        writer.close();
+      } catch (IOException e) {
+        localFS.delete(outputPath, true);
+        throw e;
+      }
+
+      closeOnDiskFile(outputPath);
+
+      LOG.info(inputContext.getUniqueIdentifier() +
+          " Finished merging " + inputs.size() + 
+          " map output files on disk of total-size " + 
+          approxOutputSize + "." + 
+          " Local output file is " + outputPath + " of size " +
+          localFS.getFileStatus(outputPath).getLen());
+    }
+  }
+  
+  private long createInMemorySegments(List<MapOutput> inMemoryMapOutputs,
+                                      List<Segment> inMemorySegments, 
+                                      long leaveBytes
+                                      ) throws IOException {
+    long totalSize = 0L;
+    // We could use fullSize could come from the RamManager, but files can be
+    // closed but not yet present in inMemoryMapOutputs
+    long fullSize = 0L;
+    for (MapOutput mo : inMemoryMapOutputs) {
+      fullSize += mo.getMemory().length;
+    }
+    while(fullSize > leaveBytes) {
+      MapOutput mo = inMemoryMapOutputs.remove(0);
+      byte[] data = mo.getMemory();
+      long size = data.length;
+      totalSize += size;
+      fullSize -= size;
+      IFile.Reader reader = new InMemoryReader(MergeManager.this, 
+                                                   mo.getAttemptIdentifier(),
+                                                   data, 0, (int)size);
+      inMemorySegments.add(new Segment(reader, true, 
+                                            (mo.isPrimaryMapOutput() ? 
+                                            mergedMapOutputsCounter : null)));
+    }
+    return totalSize;
+  }
+
+  class RawKVIteratorReader extends IFile.Reader {
+
+    private final TezRawKeyValueIterator kvIter;
+
+    public RawKVIteratorReader(TezRawKeyValueIterator kvIter, long size)
+        throws IOException {
+      super(null, null, size, null, spilledRecordsCounter);
+      this.kvIter = kvIter;
+    }
+    public boolean nextRawKey(DataInputBuffer key) throws IOException {
+      if (kvIter.next()) {
+        final DataInputBuffer kb = kvIter.getKey();
+        final int kp = kb.getPosition();
+        final int klen = kb.getLength() - kp;
+        key.reset(kb.getData(), kp, klen);
+        bytesRead += klen;
+        return true;
+      }
+      return false;
+    }
+    public void nextRawValue(DataInputBuffer value) throws IOException {
+      final DataInputBuffer vb = kvIter.getValue();
+      final int vp = vb.getPosition();
+      final int vlen = vb.getLength() - vp;
+      value.reset(vb.getData(), vp, vlen);
+      bytesRead += vlen;
+    }
+    public long getPosition() throws IOException {
+      return bytesRead;
+    }
+
+    public void close() throws IOException {
+      kvIter.close();
+    }
+  }
+
+  private TezRawKeyValueIterator finalMerge(Configuration job, FileSystem fs,
+                                       List<MapOutput> inMemoryMapOutputs,
+                                       List<Path> onDiskMapOutputs
+                                       ) throws IOException {
+    LOG.info("finalMerge called with " + 
+             inMemoryMapOutputs.size() + " in-memory map-outputs and " + 
+             onDiskMapOutputs.size() + " on-disk map-outputs");
+    
+    final float maxRedPer =
+      job.getFloat(
+          TezJobConfig.TEZ_RUNTIME_INPUT_BUFFER_PERCENT,
+          TezJobConfig.DEFAULT_TEZ_RUNTIME_INPUT_BUFFER_PERCENT);
+    if (maxRedPer > 1.0 || maxRedPer < 0.0) {
+      throw new IOException(TezJobConfig.TEZ_RUNTIME_INPUT_BUFFER_PERCENT +
+                            maxRedPer);
+    }
+    int maxInMemReduce = (int)Math.min(
+        Runtime.getRuntime().maxMemory() * maxRedPer, Integer.MAX_VALUE);
+    
+
+    // merge config params
+    Class keyClass = (Class)ConfigUtils.getIntermediateInputKeyClass(job);
+    Class valueClass = (Class)ConfigUtils.getIntermediateInputValueClass(job);
+    final Path tmpDir = new Path(inputContext.getUniqueIdentifier());
+    final RawComparator comparator =
+      (RawComparator)ConfigUtils.getIntermediateInputKeyComparator(job);
+
+    // segments required to vacate memory
+    List<Segment> memDiskSegments = new ArrayList<Segment>();
+    long inMemToDiskBytes = 0;
+    boolean mergePhaseFinished = false;
+    if (inMemoryMapOutputs.size() > 0) {
+      int srcTaskId = inMemoryMapOutputs.get(0).getAttemptIdentifier().getInputIdentifier().getSrcTaskIndex();
+      inMemToDiskBytes = createInMemorySegments(inMemoryMapOutputs, 
+                                                memDiskSegments,
+                                                maxInMemReduce);
+      final int numMemDiskSegments = memDiskSegments.size();
+      if (numMemDiskSegments > 0 &&
+            ioSortFactor > onDiskMapOutputs.size()) {
+        
+        // If we reach here, it implies that we have less than io.sort.factor
+        // disk segments and this will be incremented by 1 (result of the 
+        // memory segments merge). Since this total would still be 
+        // <= io.sort.factor, we will not do any more intermediate merges,
+        // the merge of all these disk segments would be directly fed to the
+        // reduce method
+        
+        mergePhaseFinished = true;
+        // must spill to disk, but can't retain in-mem for intermediate merge
+        final Path outputPath = 
+          mapOutputFile.getInputFileForWrite(srcTaskId,
+                                             inMemToDiskBytes).suffix(
+                                                 Constants.MERGED_OUTPUT_PREFIX);
+        final TezRawKeyValueIterator rIter = TezMerger.merge(job, fs,
+            keyClass, valueClass, memDiskSegments, numMemDiskSegments,
+            tmpDir, comparator, nullProgressable, spilledRecordsCounter, null, null);
+        final Writer writer = new Writer(job, fs, outputPath,
+            keyClass, valueClass, codec, null);
+        try {
+          TezMerger.writeFile(rIter, writer, nullProgressable, job);
+          // add to list of final disk outputs.
+          onDiskMapOutputs.add(outputPath);
+        } catch (IOException e) {
+          if (null != outputPath) {
+            try {
+              fs.delete(outputPath, true);
+            } catch (IOException ie) {
+              // NOTHING
+            }
+          }
+          throw e;
+        } finally {
+          if (null != writer) {
+            writer.close();
+          }
+        }
+        LOG.info("Merged " + numMemDiskSegments + " segments, " +
+                 inMemToDiskBytes + " bytes to disk to satisfy " +
+                 "reduce memory limit");
+        inMemToDiskBytes = 0;
+        memDiskSegments.clear();
+      } else if (inMemToDiskBytes != 0) {
+        LOG.info("Keeping " + numMemDiskSegments + " segments, " +
+                 inMemToDiskBytes + " bytes in memory for " +
+                 "intermediate, on-disk merge");
+      }
+    }
+
+    // segments on disk
+    List<Segment> diskSegments = new ArrayList<Segment>();
+    long onDiskBytes = inMemToDiskBytes;
+    Path[] onDisk = onDiskMapOutputs.toArray(new Path[onDiskMapOutputs.size()]);
+    for (Path file : onDisk) {
+      onDiskBytes += fs.getFileStatus(file).getLen();
+      LOG.debug("Disk file: " + file + " Length is " + 
+          fs.getFileStatus(file).getLen());
+      diskSegments.add(new Segment(job, fs, file, codec, false,
+                                         (file.toString().endsWith(
+                                             Constants.MERGED_OUTPUT_PREFIX) ?
+                                          null : mergedMapOutputsCounter)
+                                        ));
+    }
+    LOG.info("Merging " + onDisk.length + " files, " +
+             onDiskBytes + " bytes from disk");
+    Collections.sort(diskSegments, new Comparator<Segment>() {
+      public int compare(Segment o1, Segment o2) {
+        if (o1.getLength() == o2.getLength()) {
+          return 0;
+        }
+        return o1.getLength() < o2.getLength() ? -1 : 1;
+      }
+    });
+
+    // build final list of segments from merged backed by disk + in-mem
+    List<Segment> finalSegments = new ArrayList<Segment>();
+    long inMemBytes = createInMemorySegments(inMemoryMapOutputs, 
+                                             finalSegments, 0);
+    LOG.info("Merging " + finalSegments.size() + " segments, " +
+             inMemBytes + " bytes from memory into reduce");
+    if (0 != onDiskBytes) {
+      final int numInMemSegments = memDiskSegments.size();
+      diskSegments.addAll(0, memDiskSegments);
+      memDiskSegments.clear();
+      TezRawKeyValueIterator diskMerge = TezMerger.merge(
+          job, fs, keyClass, valueClass, diskSegments,
+          ioSortFactor, numInMemSegments, tmpDir, comparator,
+          nullProgressable, false, spilledRecordsCounter, null, null);
+      diskSegments.clear();
+      if (0 == finalSegments.size()) {
+        return diskMerge;
+      }
+      finalSegments.add(new Segment(
+            new RawKVIteratorReader(diskMerge, onDiskBytes), true));
+    }
+    return TezMerger.merge(job, fs, keyClass, valueClass,
+                 finalSegments, finalSegments.size(), tmpDir,
+                 comparator, nullProgressable, spilledRecordsCounter, null,
+                 null);
+  
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/MergeThread.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/MergeThread.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/MergeThread.java
new file mode 100644
index 0000000..d8a7722
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/MergeThread.java
@@ -0,0 +1,108 @@
+/**
+ * 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.library.common.shuffle.impl;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+abstract class MergeThread<T> extends Thread {
+  
+  private static final Log LOG = LogFactory.getLog(MergeThread.class);
+
+  private volatile boolean inProgress = false;
+  private List<T> inputs = new ArrayList<T>();
+  protected final MergeManager manager;
+  private final ExceptionReporter reporter;
+  private boolean closed = false;
+  private final int mergeFactor;
+  
+  public MergeThread(MergeManager manager, int mergeFactor,
+                     ExceptionReporter reporter) {
+    this.manager = manager;
+    this.mergeFactor = mergeFactor;
+    this.reporter = reporter;
+  }
+  
+  public synchronized void close() throws InterruptedException {
+    closed = true;
+    waitForMerge();
+    interrupt();
+  }
+
+  public synchronized boolean isInProgress() {
+    return inProgress;
+  }
+  
+  public synchronized void startMerge(Set<T> inputs) {
+    if (!closed) {
+      inProgress = true;
+      this.inputs = new ArrayList<T>();
+      Iterator<T> iter=inputs.iterator();
+      for (int ctr = 0; iter.hasNext() && ctr < mergeFactor; ++ctr) {
+        this.inputs.add(iter.next());
+        iter.remove();
+      }
+      LOG.info(getName() + ": Starting merge with " + this.inputs.size() + 
+               " segments, while ignoring " + inputs.size() + " segments");
+      notifyAll();
+    }
+  }
+
+  public synchronized void waitForMerge() throws InterruptedException {
+    while (inProgress) {
+      wait();
+    }
+  }
+
+  public void run() {
+    while (true) {
+      try {
+        // Wait for notification to start the merge...
+        synchronized (this) {
+          while (!inProgress) {
+            wait();
+          }
+        }
+
+        // Merge
+        merge(inputs);
+      } catch (InterruptedException ie) {
+        return;
+      } catch(Throwable t) {
+        reporter.reportException(t);
+        return;
+      } finally {
+        synchronized (this) {
+          // Clear inputs
+          inputs = null;
+          inProgress = false;        
+          notifyAll();
+        }
+      }
+    }
+  }
+
+  public abstract void merge(List<T> inputs) 
+      throws IOException, InterruptedException;
+}


[31/50] [abbrv] Rename tez-engine-api to tez-runtime-api and tez-engine is split into 2: - tez-engine-library for user-visible Input/Output/Processor implementations - tez-engine-internals for framework internals

Posted by ss...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/IFile.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/IFile.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/IFile.java
new file mode 100644
index 0000000..4ce82d5
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/IFile.java
@@ -0,0 +1,559 @@
+/**
+ * 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.library.common.sort.impl;
+
+import java.io.DataInput;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.DataInputBuffer;
+import org.apache.hadoop.io.DataOutputBuffer;
+import org.apache.hadoop.io.BufferUtils;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.io.WritableUtils;
+import org.apache.hadoop.io.compress.CodecPool;
+import org.apache.hadoop.io.compress.CompressionCodec;
+import org.apache.hadoop.io.compress.CompressionOutputStream;
+import org.apache.hadoop.io.compress.Compressor;
+import org.apache.hadoop.io.compress.Decompressor;
+import org.apache.hadoop.io.serializer.SerializationFactory;
+import org.apache.hadoop.io.serializer.Serializer;
+import org.apache.tez.common.counters.TezCounter;
+
+/**
+ * <code>IFile</code> is the simple <key-len, value-len, key, value> format
+ * for the intermediate map-outputs in Map-Reduce.
+ *
+ * There is a <code>Writer</code> to write out map-outputs in this format and 
+ * a <code>Reader</code> to read files of this format.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class IFile {
+  private static final Log LOG = LogFactory.getLog(IFile.class);
+  public static final int EOF_MARKER = -1; // End of File Marker
+  public static final int RLE_MARKER = -2; // Repeat same key marker
+  public static final DataInputBuffer REPEAT_KEY = new DataInputBuffer();
+    
+  /**
+   * <code>IFile.Writer</code> to write out intermediate map-outputs. 
+   */
+  @InterfaceAudience.Private
+  @InterfaceStability.Unstable
+  @SuppressWarnings({"unchecked", "rawtypes"})
+  public static class Writer {
+    FSDataOutputStream out;
+    boolean ownOutputStream = false;
+    long start = 0;
+    FSDataOutputStream rawOut;
+    AtomicBoolean closed = new AtomicBoolean(false);
+    
+    CompressionOutputStream compressedOut;
+    Compressor compressor;
+    boolean compressOutput = false;
+    
+    long decompressedBytesWritten = 0;
+    long compressedBytesWritten = 0;
+
+    // Count records written to disk
+    private long numRecordsWritten = 0;
+    private final TezCounter writtenRecordsCounter;
+
+    IFileOutputStream checksumOut;
+
+    Class keyClass;
+    Class valueClass;
+    Serializer keySerializer;
+    Serializer valueSerializer;
+    
+    DataOutputBuffer buffer = new DataOutputBuffer();
+    DataOutputBuffer previous = new DataOutputBuffer();
+    
+    // de-dup keys or not
+    private boolean rle = false;
+
+    public Writer(Configuration conf, FileSystem fs, Path file, 
+                  Class keyClass, Class valueClass,
+                  CompressionCodec codec,
+                  TezCounter writesCounter) throws IOException {
+      this(conf, fs.create(file), keyClass, valueClass, codec,
+           writesCounter);
+      ownOutputStream = true;
+    }
+    
+    protected Writer(TezCounter writesCounter) {
+      writtenRecordsCounter = writesCounter;
+    }
+
+    public Writer(Configuration conf, FSDataOutputStream out, 
+        Class keyClass, Class valueClass,
+        CompressionCodec codec, TezCounter writesCounter)
+        throws IOException {
+      this.writtenRecordsCounter = writesCounter;
+      this.checksumOut = new IFileOutputStream(out);
+      this.rawOut = out;
+      this.start = this.rawOut.getPos();
+      if (codec != null) {
+        this.compressor = CodecPool.getCompressor(codec);
+        if (this.compressor != null) {
+          this.compressor.reset();
+          this.compressedOut = codec.createOutputStream(checksumOut, compressor);
+          this.out = new FSDataOutputStream(this.compressedOut,  null);
+          this.compressOutput = true;
+        } else {
+          LOG.warn("Could not obtain compressor from CodecPool");
+          this.out = new FSDataOutputStream(checksumOut,null);
+        }
+      } else {
+        this.out = new FSDataOutputStream(checksumOut,null);
+      }
+      
+      this.keyClass = keyClass;
+      this.valueClass = valueClass;
+
+      if (keyClass != null) {
+        SerializationFactory serializationFactory = 
+          new SerializationFactory(conf);
+        this.keySerializer = serializationFactory.getSerializer(keyClass);
+        this.keySerializer.open(buffer);
+        this.valueSerializer = serializationFactory.getSerializer(valueClass);
+        this.valueSerializer.open(buffer);
+      }
+    }
+
+    public Writer(Configuration conf, FileSystem fs, Path file) 
+    throws IOException {
+      this(conf, fs, file, null, null, null, null);
+    }
+
+    public void close() throws IOException {
+      if (closed.getAndSet(true)) {
+        throw new IOException("Writer was already closed earlier");
+      }
+
+      // When IFile writer is created by BackupStore, we do not have
+      // Key and Value classes set. So, check before closing the
+      // serializers
+      if (keyClass != null) {
+        keySerializer.close();
+        valueSerializer.close();
+      }
+
+      // Write EOF_MARKER for key/value length
+      WritableUtils.writeVInt(out, EOF_MARKER);
+      WritableUtils.writeVInt(out, EOF_MARKER);
+      decompressedBytesWritten += 2 * WritableUtils.getVIntSize(EOF_MARKER);
+      
+      //Flush the stream
+      out.flush();
+  
+      if (compressOutput) {
+        // Flush
+        compressedOut.finish();
+        compressedOut.resetState();
+      }
+      
+      // Close the underlying stream iff we own it...
+      if (ownOutputStream) {
+        out.close();
+      }
+      else {
+        // Write the checksum
+        checksumOut.finish();
+      }
+
+      compressedBytesWritten = rawOut.getPos() - start;
+
+      if (compressOutput) {
+        // Return back the compressor
+        CodecPool.returnCompressor(compressor);
+        compressor = null;
+      }
+
+      out = null;
+      if(writtenRecordsCounter != null) {
+        writtenRecordsCounter.increment(numRecordsWritten);
+      }
+    }
+
+    public void append(Object key, Object value) throws IOException {
+      if (key.getClass() != keyClass)
+        throw new IOException("wrong key class: "+ key.getClass()
+                              +" is not "+ keyClass);
+      if (value.getClass() != valueClass)
+        throw new IOException("wrong value class: "+ value.getClass()
+                              +" is not "+ valueClass);
+      
+      boolean sameKey = false;
+
+      // Append the 'key'
+      keySerializer.serialize(key);
+      int keyLength = buffer.getLength();
+      if (keyLength < 0) {
+        throw new IOException("Negative key-length not allowed: " + keyLength + 
+                              " for " + key);
+      }     
+      
+      if(keyLength == previous.getLength()) {
+        sameKey = (BufferUtils.compare(previous, buffer) == 0);       
+      }
+      
+      if(!sameKey) {
+        BufferUtils.copy(buffer, previous);
+      }
+
+      // Append the 'value'
+      valueSerializer.serialize(value);
+      int valueLength = buffer.getLength() - keyLength;
+      if (valueLength < 0) {
+        throw new IOException("Negative value-length not allowed: " + 
+                              valueLength + " for " + value);
+      }
+      
+      if(sameKey) {        
+        WritableUtils.writeVInt(out, RLE_MARKER);                   // Same key as previous
+        WritableUtils.writeVInt(out, valueLength);                  // value length
+        out.write(buffer.getData(), keyLength, buffer.getLength()); // only the value
+        // Update bytes written
+        decompressedBytesWritten += 0 + valueLength + 
+                                    WritableUtils.getVIntSize(RLE_MARKER) + 
+                                    WritableUtils.getVIntSize(valueLength);
+      } else {        
+        // Write the record out        
+        WritableUtils.writeVInt(out, keyLength);                  // key length
+        WritableUtils.writeVInt(out, valueLength);                // value length
+        out.write(buffer.getData(), 0, buffer.getLength());       // data
+        // Update bytes written
+        decompressedBytesWritten += keyLength + valueLength + 
+                                    WritableUtils.getVIntSize(keyLength) + 
+                                    WritableUtils.getVIntSize(valueLength);
+      }
+
+      // Reset
+      buffer.reset();
+      
+      
+      ++numRecordsWritten;
+    }
+    
+    public void append(DataInputBuffer key, DataInputBuffer value)
+    throws IOException {
+      int keyLength = key.getLength() - key.getPosition();
+      if (keyLength < 0) {
+        throw new IOException("Negative key-length not allowed: " + keyLength + 
+                              " for " + key);
+      }
+      
+      int valueLength = value.getLength() - value.getPosition();
+      if (valueLength < 0) {
+        throw new IOException("Negative value-length not allowed: " + 
+                              valueLength + " for " + value);
+      }
+      
+      boolean sameKey = false;
+      
+      if(rle && keyLength == previous.getLength()) {
+        sameKey = (keyLength != 0) && (BufferUtils.compare(previous, key) == 0);        
+      }
+      
+      if(rle && sameKey) {
+        WritableUtils.writeVInt(out, RLE_MARKER);
+        WritableUtils.writeVInt(out, valueLength);        
+        out.write(value.getData(), value.getPosition(), valueLength);
+
+        // Update bytes written
+        decompressedBytesWritten += 0 + valueLength
+            + WritableUtils.getVIntSize(RLE_MARKER)
+            + WritableUtils.getVIntSize(valueLength);
+      } else {
+        WritableUtils.writeVInt(out, keyLength);
+        WritableUtils.writeVInt(out, valueLength);
+        out.write(key.getData(), key.getPosition(), keyLength);
+        out.write(value.getData(), value.getPosition(), valueLength);
+
+        // Update bytes written
+        decompressedBytesWritten += keyLength + valueLength
+            + WritableUtils.getVIntSize(keyLength)
+            + WritableUtils.getVIntSize(valueLength);
+                
+        BufferUtils.copy(key, previous);        
+      }
+      ++numRecordsWritten;
+    }
+    
+    // Required for mark/reset
+    public DataOutputStream getOutputStream () {
+      return out;
+    }
+    
+    // Required for mark/reset
+    public void updateCountersForExternalAppend(long length) {
+      ++numRecordsWritten;
+      decompressedBytesWritten += length;
+    }
+    
+    public long getRawLength() {
+      return decompressedBytesWritten;
+    }
+    
+    public long getCompressedLength() {
+      return compressedBytesWritten;
+    }
+    
+    public void setRLE(boolean rle) {
+      this.rle = rle;
+      previous.reset();
+    }
+
+  }
+
+  /**
+   * <code>IFile.Reader</code> to read intermediate map-outputs. 
+   */
+  @InterfaceAudience.Private
+  @InterfaceStability.Unstable
+  public static class Reader {
+    
+    public enum KeyState {NO_KEY, NEW_KEY, SAME_KEY};
+    
+    private static final int DEFAULT_BUFFER_SIZE = 128*1024;
+
+    // Count records read from disk
+    private long numRecordsRead = 0;
+    private final TezCounter readRecordsCounter;
+
+    final InputStream in;        // Possibly decompressed stream that we read
+    Decompressor decompressor;
+    public long bytesRead = 0;
+    protected final long fileLength;
+    protected boolean eof = false;
+    final IFileInputStream checksumIn;
+    
+    protected byte[] buffer = null;
+    protected int bufferSize = DEFAULT_BUFFER_SIZE;
+    protected DataInputStream dataIn;
+
+    protected int recNo = 1;
+    protected int prevKeyLength;
+    protected int currentKeyLength;
+    protected int currentValueLength;
+    byte keyBytes[] = new byte[0];
+    
+    
+    /**
+     * Construct an IFile Reader.
+     * 
+     * @param conf Configuration File 
+     * @param fs  FileSystem
+     * @param file Path of the file to be opened. This file should have
+     *             checksum bytes for the data at the end of the file.
+     * @param codec codec
+     * @param readsCounter Counter for records read from disk
+     * @throws IOException
+     */
+    public Reader(Configuration conf, FileSystem fs, Path file,
+                  CompressionCodec codec,
+                  TezCounter readsCounter) throws IOException {
+      this(conf, fs.open(file), 
+           fs.getFileStatus(file).getLen(),
+           codec, readsCounter);
+    }
+
+    /**
+     * Construct an IFile Reader.
+     * 
+     * @param conf Configuration File 
+     * @param in   The input stream
+     * @param length Length of the data in the stream, including the checksum
+     *               bytes.
+     * @param codec codec
+     * @param readsCounter Counter for records read from disk
+     * @throws IOException
+     */
+    public Reader(Configuration conf, InputStream in, long length, 
+                  CompressionCodec codec,
+                  TezCounter readsCounter) throws IOException {
+      readRecordsCounter = readsCounter;
+      checksumIn = new IFileInputStream(in,length, conf);
+      if (codec != null) {
+        decompressor = CodecPool.getDecompressor(codec);
+        if (decompressor != null) {
+          this.in = codec.createInputStream(checksumIn, decompressor);
+        } else {
+          LOG.warn("Could not obtain decompressor from CodecPool");
+          this.in = checksumIn;
+        }
+      } else {
+        this.in = checksumIn;
+      }
+      this.dataIn = new DataInputStream(this.in);
+      this.fileLength = length;
+      
+      if (conf != null) {
+        bufferSize = conf.getInt("io.file.buffer.size", DEFAULT_BUFFER_SIZE);
+      }
+    }
+    
+    public long getLength() { 
+      return fileLength - checksumIn.getSize();
+    }
+    
+    public long getPosition() throws IOException {    
+      return checksumIn.getPosition(); 
+    }
+    
+    /**
+     * Read upto len bytes into buf starting at offset off.
+     * 
+     * @param buf buffer 
+     * @param off offset
+     * @param len length of buffer
+     * @return the no. of bytes read
+     * @throws IOException
+     */
+    private int readData(byte[] buf, int off, int len) throws IOException {
+      int bytesRead = 0;
+      while (bytesRead < len) {
+        int n = IOUtils.wrappedReadForCompressedData(in, buf, off + bytesRead,
+            len - bytesRead);
+        if (n < 0) {
+          return bytesRead;
+        }
+        bytesRead += n;
+      }
+      return len;
+    }
+    
+    protected boolean positionToNextRecord(DataInput dIn) throws IOException {
+      // Sanity check
+      if (eof) {
+        throw new EOFException("Completed reading " + bytesRead);
+      }
+      
+      // Read key and value lengths
+      prevKeyLength = currentKeyLength;
+      currentKeyLength = WritableUtils.readVInt(dIn);
+      currentValueLength = WritableUtils.readVInt(dIn);
+      bytesRead += WritableUtils.getVIntSize(currentKeyLength) +
+                   WritableUtils.getVIntSize(currentValueLength);
+      
+      // Check for EOF
+      if (currentKeyLength == EOF_MARKER && currentValueLength == EOF_MARKER) {
+        eof = true;
+        return false;
+      }      
+      
+      // Sanity check
+      if (currentKeyLength != RLE_MARKER && currentKeyLength < 0) {
+        throw new IOException("Rec# " + recNo + ": Negative key-length: " + 
+                              currentKeyLength);
+      }
+      if (currentValueLength < 0) {
+        throw new IOException("Rec# " + recNo + ": Negative value-length: " + 
+                              currentValueLength);
+      }
+            
+      return true;
+    }
+    
+    public boolean nextRawKey(DataInputBuffer key) throws IOException {
+      return readRawKey(key) != KeyState.NO_KEY;
+    }
+    
+    public KeyState readRawKey(DataInputBuffer key) throws IOException {
+      if (!positionToNextRecord(dataIn)) {
+        return KeyState.NO_KEY;
+      }
+      if(currentKeyLength == RLE_MARKER) {
+        currentKeyLength = prevKeyLength;
+        // no data to read
+        key.reset(keyBytes, currentKeyLength);
+        return KeyState.SAME_KEY;
+      }
+      if (keyBytes.length < currentKeyLength) {
+        keyBytes = new byte[currentKeyLength << 1];
+      }
+      int i = readData(keyBytes, 0, currentKeyLength);
+      if (i != currentKeyLength) {
+        throw new IOException ("Asked for " + currentKeyLength + " Got: " + i);
+      }
+      key.reset(keyBytes, currentKeyLength);
+      bytesRead += currentKeyLength;
+      return KeyState.NEW_KEY;
+    }
+    
+    public void nextRawValue(DataInputBuffer value) throws IOException {
+      final byte[] valBytes = 
+        ((value.getData().length < currentValueLength) || (value.getData() == keyBytes))
+        ? new byte[currentValueLength << 1]
+        : value.getData();
+      int i = readData(valBytes, 0, currentValueLength);
+      if (i != currentValueLength) {
+        throw new IOException ("Asked for " + currentValueLength + " Got: " + i);
+      }
+      value.reset(valBytes, currentValueLength);
+      
+      // Record the bytes read
+      bytesRead += currentValueLength;
+
+      ++recNo;
+      ++numRecordsRead;
+    }
+    
+    public void close() throws IOException {
+      // Close the underlying stream
+      in.close();
+      
+      // Release the buffer
+      dataIn = null;
+      buffer = null;
+      if(readRecordsCounter != null) {
+        readRecordsCounter.increment(numRecordsRead);
+      }
+
+      // Return the decompressor
+      if (decompressor != null) {
+        decompressor.reset();
+        CodecPool.returnDecompressor(decompressor);
+        decompressor = null;
+      }
+    }
+    
+    public void reset(int offset) {
+      return;
+    }
+
+    public void disableChecksumValidation() {
+      checksumIn.disableChecksumValidation();
+    }
+
+  }    
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/IFileInputStream.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/IFileInputStream.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/IFileInputStream.java
new file mode 100644
index 0000000..e828c0b
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/IFileInputStream.java
@@ -0,0 +1,276 @@
+/**
+ * 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.library.common.sort.impl;
+
+import java.io.EOFException;
+import java.io.FileDescriptor;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.ChecksumException;
+import org.apache.hadoop.fs.HasFileDescriptor;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.io.ReadaheadPool;
+import org.apache.hadoop.io.ReadaheadPool.ReadaheadRequest;
+import org.apache.hadoop.util.DataChecksum;
+import org.apache.tez.common.TezJobConfig;
+/**
+ * A checksum input stream, used for IFiles.
+ * Used to validate the checksum of files created by {@link IFileOutputStream}. 
+*/
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class IFileInputStream extends InputStream {
+  
+  private final InputStream in; //The input stream to be verified for checksum.
+  private final FileDescriptor inFd; // the file descriptor, if it is known
+  private final long length; //The total length of the input file
+  private final long dataLength;
+  private DataChecksum sum;
+  private long currentOffset = 0;
+  private final byte b[] = new byte[1];
+  private byte csum[] = null;
+  private int checksumSize;
+  private byte[] buffer;
+  private int offset;
+
+  private ReadaheadRequest curReadahead = null;
+  private ReadaheadPool raPool = ReadaheadPool.getInstance();
+  private boolean readahead;
+  private int readaheadLength;
+
+  public static final Log LOG = LogFactory.getLog(IFileInputStream.class);
+
+  private boolean disableChecksumValidation = false;
+  
+  /**
+   * Create a checksum input stream that reads
+   * @param in The input stream to be verified for checksum.
+   * @param len The length of the input stream including checksum bytes.
+   */
+  public IFileInputStream(InputStream in, long len, Configuration conf) {
+    this.in = in;
+    this.inFd = getFileDescriptorIfAvail(in);
+    sum = DataChecksum.newDataChecksum(DataChecksum.Type.CRC32, 
+        Integer.MAX_VALUE);
+    checksumSize = sum.getChecksumSize();
+    buffer = new byte[4096];
+    offset = 0;
+    length = len;
+    dataLength = length - checksumSize;
+
+    conf = (conf != null) ? conf : new Configuration();
+    readahead = conf.getBoolean(TezJobConfig.TEZ_RUNTIME_IFILE_READAHEAD,
+        TezJobConfig.DEFAULT_TEZ_RUNTIME_IFILE_READAHEAD);
+    readaheadLength = conf.getInt(TezJobConfig.TEZ_RUNTIME_IFILE_READAHEAD_BYTES,
+        TezJobConfig.DEFAULT_TEZ_RUNTIME_IFILE_READAHEAD_BYTES);
+
+    doReadahead();
+  }
+
+  private static FileDescriptor getFileDescriptorIfAvail(InputStream in) {
+    FileDescriptor fd = null;
+    try {
+      if (in instanceof HasFileDescriptor) {
+        fd = ((HasFileDescriptor)in).getFileDescriptor();
+      } else if (in instanceof FileInputStream) {
+        fd = ((FileInputStream)in).getFD();
+      }
+    } catch (IOException e) {
+      LOG.info("Unable to determine FileDescriptor", e);
+    }
+    return fd;
+  }
+
+  /**
+   * Close the input stream. Note that we need to read to the end of the
+   * stream to validate the checksum.
+   */
+  @Override
+  public void close() throws IOException {
+
+    if (curReadahead != null) {
+      curReadahead.cancel();
+    }
+    if (currentOffset < dataLength) {
+      byte[] t = new byte[Math.min((int)
+            (Integer.MAX_VALUE & (dataLength - currentOffset)), 32 * 1024)];
+      while (currentOffset < dataLength) {
+        int n = read(t, 0, t.length);
+        if (0 == n) {
+          throw new EOFException("Could not validate checksum");
+        }
+      }
+    }
+    in.close();
+  }
+  
+  @Override
+  public long skip(long n) throws IOException {
+   throw new IOException("Skip not supported for IFileInputStream");
+  }
+  
+  public long getPosition() {
+    return (currentOffset >= dataLength) ? dataLength : currentOffset;
+  }
+  
+  public long getSize() {
+    return checksumSize;
+  }
+
+  private void checksum(byte[] b, int off, int len) {
+    if(len >= buffer.length) {
+      sum.update(buffer, 0, offset);
+      offset = 0;
+      sum.update(b, off, len);
+      return;
+    }
+    final int remaining = buffer.length - offset;
+    if(len > remaining) {
+      sum.update(buffer, 0, offset);
+      offset = 0;
+    }
+    /* now we should have len < buffer.length */
+    System.arraycopy(b, off, buffer, offset, len);
+    offset += len;
+  }
+  
+  /**
+   * Read bytes from the stream.
+   * At EOF, checksum is validated, but the checksum
+   * bytes are not passed back in the buffer. 
+   */
+  public int read(byte[] b, int off, int len) throws IOException {
+
+    if (currentOffset >= dataLength) {
+      return -1;
+    }
+
+    doReadahead();
+
+    return doRead(b,off,len);
+  }
+
+  private void doReadahead() {
+    if (raPool != null && inFd != null && readahead) {
+      curReadahead = raPool.readaheadStream(
+          "ifile", inFd,
+          currentOffset, readaheadLength, dataLength,
+          curReadahead);
+    }
+  }
+
+  /**
+   * Read bytes from the stream.
+   * At EOF, checksum is validated and sent back
+   * as the last four bytes of the buffer. The caller should handle
+   * these bytes appropriately
+   */
+  public int readWithChecksum(byte[] b, int off, int len) throws IOException {
+
+    if (currentOffset == length) {
+      return -1;
+    }
+    else if (currentOffset >= dataLength) {
+      // If the previous read drained off all the data, then just return
+      // the checksum now. Note that checksum validation would have 
+      // happened in the earlier read
+      int lenToCopy = (int) (checksumSize - (currentOffset - dataLength));
+      if (len < lenToCopy) {
+        lenToCopy = len;
+      }
+      System.arraycopy(csum, (int) (currentOffset - dataLength), b, off, 
+          lenToCopy);
+      currentOffset += lenToCopy;
+      return lenToCopy;
+    }
+
+    int bytesRead = doRead(b,off,len);
+
+    if (currentOffset == dataLength) {
+      if (len >= bytesRead + checksumSize) {
+        System.arraycopy(csum, 0, b, off + bytesRead, checksumSize);
+        bytesRead += checksumSize;
+        currentOffset += checksumSize;
+      }
+    }
+    return bytesRead;
+  }
+
+  private int doRead(byte[]b, int off, int len) throws IOException {
+    
+    // If we are trying to read past the end of data, just read
+    // the left over data
+    if (currentOffset + len > dataLength) {
+      len = (int) dataLength - (int)currentOffset;
+    }
+    
+    int bytesRead = in.read(b, off, len);
+
+    if (bytesRead < 0) {
+      throw new ChecksumException("Checksum Error", 0);
+    }
+
+    checksum(b, off, bytesRead);
+
+    currentOffset += bytesRead;
+
+    if (disableChecksumValidation) {
+      return bytesRead;
+    }
+    
+    if (currentOffset == dataLength) {
+      // The last four bytes are checksum. Strip them and verify
+      sum.update(buffer, 0, offset);
+      csum = new byte[checksumSize];
+      IOUtils.readFully(in, csum, 0, checksumSize);
+      if (!sum.compare(csum, 0)) {
+        throw new ChecksumException("Checksum Error", 0);
+      }
+    }
+    return bytesRead;
+  }
+
+
+  @Override
+  public int read() throws IOException {    
+    b[0] = 0;
+    int l = read(b,0,1);
+    if (l < 0)  return l;
+    
+    // Upgrade the b[0] to an int so as not to misinterpret the
+    // first bit of the byte as a sign bit
+    int result = 0xFF & b[0];
+    return result;
+  }
+
+  public byte[] getChecksum() {
+    return csum;
+  }
+
+  void disableChecksumValidation() {
+    disableChecksumValidation = true;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/IFileOutputStream.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/IFileOutputStream.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/IFileOutputStream.java
new file mode 100644
index 0000000..3198446
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/IFileOutputStream.java
@@ -0,0 +1,129 @@
+/**
+ * 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.library.common.sort.impl;
+
+import java.io.FilterOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.util.DataChecksum;
+/**
+ * A Checksum output stream.
+ * Checksum for the contents of the file is calculated and
+ * appended to the end of the file on close of the stream.
+ * Used for IFiles
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class IFileOutputStream extends FilterOutputStream {
+
+  /**
+   * The output stream to be checksummed.
+   */
+  private final DataChecksum sum;
+  private byte[] barray;
+  private byte[] buffer;
+  private int offset;
+  private boolean closed = false;
+  private boolean finished = false;
+
+  /**
+   * Create a checksum output stream that writes
+   * the bytes to the given stream.
+   * @param out
+   */
+  public IFileOutputStream(OutputStream out) {
+    super(out);
+    sum = DataChecksum.newDataChecksum(DataChecksum.Type.CRC32,
+        Integer.MAX_VALUE);
+    barray = new byte[sum.getChecksumSize()];
+    buffer = new byte[4096];
+    offset = 0;
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (closed) {
+      return;
+    }
+    closed = true;
+    finish();
+    out.close();
+  }
+
+  /**
+   * Finishes writing data to the output stream, by writing
+   * the checksum bytes to the end. The underlying stream is not closed.
+   * @throws IOException
+   */
+  public void finish() throws IOException {
+    if (finished) {
+      return;
+    }
+    finished = true;
+    sum.update(buffer, 0, offset);
+    sum.writeValue(barray, 0, false);
+    out.write (barray, 0, sum.getChecksumSize());
+    out.flush();
+  }
+
+  private void checksum(byte[] b, int off, int len) {
+    if(len >= buffer.length) {
+      sum.update(buffer, 0, offset);
+      offset = 0;
+      sum.update(b, off, len);
+      return;
+    }
+    final int remaining = buffer.length - offset;
+    if(len > remaining) {
+      sum.update(buffer, 0, offset);
+      offset = 0;
+    }
+    /*
+    // FIXME if needed re-enable this in debug mode
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("XXX checksum" +
+          " b=" + b + " off=" + off +
+          " buffer=" + " offset=" + offset +
+          " len=" + len);
+    }
+    */
+    /* now we should have len < buffer.length */
+    System.arraycopy(b, off, buffer, offset, len);
+    offset += len;
+  }
+
+  /**
+   * Write bytes to the stream.
+   */
+  @Override
+  public void write(byte[] b, int off, int len) throws IOException {
+    checksum(b, off, len);
+    out.write(b,off,len);
+  }
+
+  @Override
+  public void write(int b) throws IOException {
+    barray[0] = (byte) (b & 0xFF);
+    write(barray,0,1);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/PipelinedSorter.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/PipelinedSorter.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/PipelinedSorter.java
new file mode 100644
index 0000000..1b153ca
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/PipelinedSorter.java
@@ -0,0 +1,932 @@
+/**
+* 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.library.common.sort.impl;
+
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.nio.BufferOverflowException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.nio.IntBuffer;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.PriorityQueue;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+
+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.Path;
+import org.apache.hadoop.io.DataInputBuffer;
+import org.apache.hadoop.io.HashComparator;
+import org.apache.hadoop.io.RawComparator;
+import org.apache.hadoop.util.IndexedSortable;
+import org.apache.hadoop.util.IndexedSorter;
+import org.apache.hadoop.util.Progress;
+import org.apache.tez.common.TezJobConfig;
+import org.apache.tez.runtime.api.TezOutputContext;
+import org.apache.tez.runtime.library.common.ConfigUtils;
+import org.apache.tez.runtime.library.common.sort.impl.IFile.Writer;
+import org.apache.tez.runtime.library.common.sort.impl.TezMerger.Segment;
+
+@SuppressWarnings({"unchecked", "rawtypes"})
+public class PipelinedSorter extends ExternalSorter {
+  
+  private static final Log LOG = LogFactory.getLog(PipelinedSorter.class);
+  
+  /**
+   * The size of each record in the index file for the map-outputs.
+   */
+  public static final int MAP_OUTPUT_INDEX_RECORD_LENGTH = 24;
+
+  private final static int APPROX_HEADER_LENGTH = 150;
+    
+  int partitionBits;
+  
+  private static final int PARTITION = 0;        // partition offset in acct
+  private static final int KEYSTART = 1;         // key offset in acct
+  private static final int VALSTART = 2;         // val offset in acct
+  private static final int VALLEN = 3;           // val len in acct
+  private static final int NMETA = 4;            // num meta ints
+  private static final int METASIZE = NMETA * 4; // size in bytes
+
+  // spill accounting
+  volatile Throwable sortSpillException = null;
+
+  int numSpills = 0;
+  int minSpillsForCombine;
+  private HashComparator hasher;
+  // SortSpans  
+  private SortSpan span;
+  private ByteBuffer largeBuffer;
+  // Merger
+  private SpanMerger merger; 
+  private ExecutorService sortmaster;
+
+  final ArrayList<TezSpillRecord> indexCacheList =
+    new ArrayList<TezSpillRecord>();
+  private int totalIndexCacheMemory;
+  private int indexCacheMemoryLimit;
+
+  public void initialize(TezOutputContext outputContext, Configuration conf, int numOutputs) throws IOException {
+    super.initialize(outputContext, conf, numOutputs);
+    
+    partitionBits = bitcount(partitions)+1;
+   
+    //sanity checks
+    final float spillper =
+      this.conf.getFloat(
+          TezJobConfig.TEZ_RUNTIME_SORT_SPILL_PERCENT, 
+          TezJobConfig.DEFAULT_TEZ_RUNTIME_SORT_SPILL_PERCENT);
+    final int sortmb = 
+        this.conf.getInt(
+            TezJobConfig.TEZ_RUNTIME_IO_SORT_MB, 
+            TezJobConfig.DEFAULT_TEZ_RUNTIME_IO_SORT_MB);
+    indexCacheMemoryLimit = this.conf.getInt(TezJobConfig.TEZ_RUNTIME_INDEX_CACHE_MEMORY_LIMIT_BYTES,
+                                       TezJobConfig.DEFAULT_TEZ_RUNTIME_INDEX_CACHE_MEMORY_LIMIT_BYTES);
+    if (spillper > (float)1.0 || spillper <= (float)0.0) {
+      throw new IOException("Invalid \"" + TezJobConfig.TEZ_RUNTIME_SORT_SPILL_PERCENT +
+          "\": " + spillper);
+    }
+    if ((sortmb & 0x7FF) != sortmb) {
+      throw new IOException(
+          "Invalid \"" + TezJobConfig.TEZ_RUNTIME_IO_SORT_MB + "\": " + sortmb);
+    }
+    
+    // buffers and accounting
+    int maxMemUsage = sortmb << 20;
+    maxMemUsage -= maxMemUsage % METASIZE;
+    largeBuffer = ByteBuffer.allocate(maxMemUsage);
+    LOG.info(TezJobConfig.TEZ_RUNTIME_IO_SORT_MB + " = " + sortmb);
+    // TODO: configurable setting?
+    span = new SortSpan(largeBuffer, 1024*1024, 16);
+    merger = new SpanMerger(comparator);
+    final int sortThreads = 
+            this.conf.getInt(
+                TezJobConfig.TEZ_RUNTIME_SORT_THREADS, 
+                TezJobConfig.DEFAULT_TEZ_RUNTIME_SORT_THREADS);
+    sortmaster = Executors.newFixedThreadPool(sortThreads);
+
+    // k/v serialization    
+    if(comparator instanceof HashComparator) {
+      hasher = (HashComparator)comparator;
+      LOG.info("Using the HashComparator");
+    } else {
+      hasher = null;
+    }    
+    valSerializer.open(span.out);
+    keySerializer.open(span.out);
+    minSpillsForCombine = this.conf.getInt(TezJobConfig.TEZ_RUNTIME_COMBINE_MIN_SPILLS, 3);
+  }
+
+  private int bitcount(int n) {
+    int bit = 0;
+    while(n!=0) {
+      bit++;
+      n >>= 1;
+    }
+    return bit;
+  }
+  
+  public void sort() throws IOException {
+    SortSpan newSpan = span.next();
+
+    if(newSpan == null) {
+      // sort in the same thread, do not wait for the thread pool
+      merger.add(span.sort(sorter, comparator));
+      spill();
+      int items = 1024*1024;
+      int perItem = 16;
+      if(span.length() != 0) {
+        items = span.length();
+        perItem = span.kvbuffer.limit()/items;
+        items = (largeBuffer.capacity())/(METASIZE+perItem);
+        if(items > 1024*1024) {
+            // our goal is to have 1M splits and sort early
+            items = 1024*1024;
+        }
+      }      
+      span = new SortSpan(largeBuffer, items, perItem);
+    } else {
+      // queue up the sort
+      SortTask task = new SortTask(span, sorter, comparator);
+      Future<SpanIterator> future = sortmaster.submit(task);
+      merger.add(future);
+      span = newSpan;
+    }
+    valSerializer.open(span.out);
+    keySerializer.open(span.out);
+  }
+
+  @Override
+  public void write(Object key, Object value) 
+      throws IOException {
+    collect(
+        key, value, partitioner.getPartition(key, value, partitions));
+  }
+
+  /**
+   * Serialize the key, value to intermediate storage.
+   * When this method returns, kvindex must refer to sufficient unused
+   * storage to store one METADATA.
+   */
+  synchronized void collect(Object key, Object value, final int partition
+                                   ) throws IOException {
+    if (key.getClass() != keyClass) {
+      throw new IOException("Type mismatch in key from map: expected "
+                            + keyClass.getName() + ", received "
+                            + key.getClass().getName());
+    }
+    if (value.getClass() != valClass) {
+      throw new IOException("Type mismatch in value from map: expected "
+                            + valClass.getName() + ", received "
+                            + value.getClass().getName());
+    }
+    if (partition < 0 || partition >= partitions) {
+      throw new IOException("Illegal partition for " + key + " (" +
+          partition + ")");
+    }
+    if(span.kvmeta.remaining() < METASIZE) {
+      this.sort();
+    }
+    int keystart = span.kvbuffer.position();
+    int valstart = -1;
+    int valend = -1;
+    try { 
+      keySerializer.serialize(key);
+      valstart = span.kvbuffer.position();      
+      valSerializer.serialize(value);
+      valend = span.kvbuffer.position();
+    } catch(BufferOverflowException overflow) {
+      // restore limit
+      span.kvbuffer.position(keystart);
+      this.sort();
+      // try again
+      this.collect(key, value, partition);
+      return;
+    }
+
+    int prefix = 0;
+
+    if(hasher != null) {
+      prefix = hasher.getHashCode(key);
+    }
+
+    prefix = (partition << (32 - partitionBits)) | (prefix >>> partitionBits);
+
+    /* maintain order as in PARTITION, KEYSTART, VALSTART, VALLEN */
+    span.kvmeta.put(prefix);
+    span.kvmeta.put(keystart);
+    span.kvmeta.put(valstart);
+    span.kvmeta.put(valend - valstart);
+    if((valstart - keystart) > span.keymax) {
+      span.keymax = (valstart - keystart);
+    }
+    if((valend - valstart) > span.valmax) {
+      span.valmax = (valend - valstart);
+    }
+    mapOutputRecordCounter.increment(1);
+    mapOutputByteCounter.increment(valend - keystart);
+  }
+
+  public void spill() throws IOException { 
+    // create spill file
+    final long size = largeBuffer.capacity() + 
+      (partitions * APPROX_HEADER_LENGTH);
+    final TezSpillRecord spillRec = new TezSpillRecord(partitions);
+    final Path filename =
+      mapOutputFile.getSpillFileForWrite(numSpills, size);    
+    FSDataOutputStream out = rfs.create(filename, true, 4096);
+
+    try {
+      merger.ready(); // wait for all the future results from sort threads
+      LOG.info("Spilling to " + filename.toString());
+      for (int i = 0; i < partitions; ++i) {
+        TezRawKeyValueIterator kvIter = merger.filter(i);
+        //write merged output to disk
+        long segmentStart = out.getPos();
+        Writer writer =
+          new Writer(conf, out, keyClass, valClass, codec,
+              spilledRecordsCounter);
+        writer.setRLE(merger.needsRLE());
+        if (combiner == null) {
+          while(kvIter.next()) {
+            writer.append(kvIter.getKey(), kvIter.getValue());
+          }
+        } else {          
+          runCombineProcessor(kvIter, writer);
+        }
+        //close
+        writer.close();
+
+        // record offsets
+        final TezIndexRecord rec = 
+            new TezIndexRecord(
+                segmentStart, 
+                writer.getRawLength(), 
+                writer.getCompressedLength());
+        spillRec.putIndex(rec, i);
+      }
+
+      Path indexFilename =
+        mapOutputFile.getSpillIndexFileForWrite(numSpills, partitions
+            * MAP_OUTPUT_INDEX_RECORD_LENGTH);
+      // TODO: cache
+      spillRec.writeToFile(indexFilename, conf);
+      ++numSpills;
+    } catch(InterruptedException ie) {
+      // TODO:the combiner has been interrupted
+    } finally {
+      out.close();
+    }
+  }
+
+  @Override
+  public void flush() throws IOException {
+    final String uniqueIdentifier = outputContext.getUniqueIdentifier();
+    Path finalOutputFile =
+        mapOutputFile.getOutputFileForWrite(0); //TODO
+    Path finalIndexFile =
+        mapOutputFile.getOutputIndexFileForWrite(0); //TODO
+
+    LOG.info("Starting flush of map output");
+    span.end();
+    merger.add(span.sort(sorter, comparator));
+    spill();
+    sortmaster.shutdown();
+
+    largeBuffer = null;
+
+    if(numSpills == 1) {
+      // someday be able to pass this directly to shuffle
+      // without writing to disk
+      final Path filename =
+          mapOutputFile.getSpillFile(0);
+      Path indexFilename =
+              mapOutputFile.getSpillIndexFile(0);
+      sameVolRename(filename, finalOutputFile);
+      sameVolRename(indexFilename, finalIndexFile);
+      return;
+    }
+    
+    //The output stream for the final single output file
+    FSDataOutputStream finalOut = rfs.create(finalOutputFile, true, 4096);
+
+    TezMerger.considerFinalMergeForProgress();
+
+    final TezSpillRecord spillRec = new TezSpillRecord(partitions);
+    final ArrayList<TezSpillRecord> indexCacheList = new ArrayList<TezSpillRecord>();
+
+    for(int i = 0; i < numSpills; i++) {
+      // TODO: build this cache before
+      Path indexFilename = mapOutputFile.getSpillIndexFile(i);
+      TezSpillRecord spillIndex = new TezSpillRecord(indexFilename, conf);
+      indexCacheList.add(spillIndex);
+    }
+    
+    for (int parts = 0; parts < partitions; parts++) {
+      //create the segments to be merged
+      List<Segment> segmentList =
+          new ArrayList<Segment>(numSpills);
+      for(int i = 0; i < numSpills; i++) {
+        Path spillFilename = mapOutputFile.getSpillFile(i);
+        TezIndexRecord indexRecord = indexCacheList.get(i).getIndex(parts);
+
+        Segment s =
+            new Segment(conf, rfs, spillFilename, indexRecord.getStartOffset(),
+                             indexRecord.getPartLength(), codec, true);
+        segmentList.add(i, s);
+      }
+
+      int mergeFactor = 
+              this.conf.getInt(TezJobConfig.TEZ_RUNTIME_IO_SORT_FACTOR, 
+                  TezJobConfig.DEFAULT_TEZ_RUNTIME_IO_SORT_FACTOR);
+      // sort the segments only if there are intermediate merges
+      boolean sortSegments = segmentList.size() > mergeFactor;
+      //merge
+      TezRawKeyValueIterator kvIter = TezMerger.merge(conf, rfs,
+                     keyClass, valClass, codec,
+                     segmentList, mergeFactor,
+                     new Path(uniqueIdentifier),
+                     (RawComparator)ConfigUtils.getIntermediateOutputKeyComparator(conf), 
+                     nullProgressable, sortSegments,
+                     null, spilledRecordsCounter,
+                     null); // Not using any Progress in TezMerger. Should just work.
+
+      //write merged output to disk
+      long segmentStart = finalOut.getPos();
+      Writer writer =
+          new Writer(conf, finalOut, keyClass, valClass, codec,
+                           spilledRecordsCounter);
+      writer.setRLE(merger.needsRLE());
+      if (combiner == null || numSpills < minSpillsForCombine) {
+        TezMerger.writeFile(kvIter, writer, nullProgressable, conf);
+      } else {
+        runCombineProcessor(kvIter, writer);
+      }
+
+      //close
+      writer.close();
+
+      // record offsets
+      final TezIndexRecord rec = 
+          new TezIndexRecord(
+              segmentStart, 
+              writer.getRawLength(), 
+              writer.getCompressedLength());
+      spillRec.putIndex(rec, parts);
+    }
+
+    spillRec.writeToFile(finalIndexFile, conf);
+    finalOut.close();
+    for(int i = 0; i < numSpills; i++) {
+      Path indexFilename = mapOutputFile.getSpillIndexFile(i);
+      Path spillFilename = mapOutputFile.getSpillFile(i);
+      rfs.delete(indexFilename,true);
+      rfs.delete(spillFilename,true);
+    }
+  }
+
+  public void close() { }
+
+  private interface PartitionedRawKeyValueIterator extends TezRawKeyValueIterator {
+    int getPartition();
+  }
+
+  private class BufferStreamWrapper extends OutputStream 
+  {
+    private final ByteBuffer out;
+    public BufferStreamWrapper(ByteBuffer out) {
+      this.out = out;
+    }
+    
+    @Override
+    public void write(int b) throws IOException { out.put((byte)b); }
+    @Override
+    public void write(byte[] b) throws IOException { out.put(b); }
+    @Override
+    public void write(byte[] b, int off, int len) throws IOException { out.put(b, off, len); }
+  }
+
+  protected class InputByteBuffer extends DataInputBuffer {
+    private byte[] buffer = new byte[256]; 
+    private ByteBuffer wrapped = ByteBuffer.wrap(buffer);
+    private void resize(int length) {
+      if(length > buffer.length) {
+        buffer = new byte[length];
+        wrapped = ByteBuffer.wrap(buffer);
+      }
+      wrapped.limit(length);
+    }
+    public void reset(ByteBuffer b, int start, int length) {
+      resize(length);
+      b.position(start);
+      b.get(buffer, 0, length);
+      super.reset(buffer, 0, length);
+    }
+    // clone-ish function
+    public void reset(DataInputBuffer clone) {
+      byte[] data = clone.getData();
+      int start = clone.getPosition();
+      int length = clone.getLength();
+      resize(length);
+      System.arraycopy(data, start, buffer, 0, length);
+      super.reset(buffer, 0, length);
+    }
+  }
+
+  private class SortSpan  implements IndexedSortable {
+    final IntBuffer kvmeta;
+    final ByteBuffer kvbuffer;
+    final DataOutputStream out;    
+    private RawComparator comparator; 
+    final int imeta[] = new int[NMETA];
+    final int jmeta[] = new int[NMETA];
+    int keymax = 1;
+    int valmax = 1;
+    private int i,j;
+    private byte[] ki;
+    private byte[] kj;
+    private int index = 0;
+    private InputByteBuffer hay = new InputByteBuffer();
+    private long eq = 0;
+
+    public SortSpan(ByteBuffer source, int maxItems, int perItem) {
+      int capacity = source.remaining(); 
+      int metasize = METASIZE*maxItems;
+      int dataSize = maxItems * perItem;
+      if(capacity < (metasize+dataSize)) {
+        // try to allocate less meta space, because we have sample data
+        metasize = METASIZE*(capacity/(perItem+METASIZE));
+      }
+      ByteBuffer reserved = source.duplicate();
+      reserved.mark();
+      LOG.info("reserved.remaining() = "+reserved.remaining());
+      LOG.info("reserved.size = "+metasize);
+      reserved.position(metasize);
+      kvbuffer = reserved.slice();
+      reserved.flip();
+      reserved.limit(metasize);
+      kvmeta = reserved
+                .slice()
+                .order(ByteOrder.nativeOrder())
+               .asIntBuffer();
+      out = new DataOutputStream(
+              new BufferStreamWrapper(kvbuffer));
+    }
+
+    public SpanIterator sort(IndexedSorter sorter, RawComparator comparator) {
+    	this.comparator = comparator;
+      ki = new byte[keymax];
+      kj = new byte[keymax];
+      LOG.info("begin sorting Span"+index + " ("+length()+")");
+      if(length() > 1) {
+        sorter.sort(this, 0, length(), nullProgressable);
+      }
+      LOG.info("done sorting Span"+index);
+      return new SpanIterator(this);
+    }
+
+    int offsetFor(int i) {
+      return (i * NMETA);
+    }
+
+    public void swap(final int mi, final int mj) {
+      final int kvi = offsetFor(mi);
+      final int kvj = offsetFor(mj);
+
+      kvmeta.position(kvi); kvmeta.get(imeta);
+      kvmeta.position(kvj); kvmeta.get(jmeta);
+      kvmeta.position(kvj); kvmeta.put(imeta);
+      kvmeta.position(kvi); kvmeta.put(jmeta);
+
+      if(i == mi || j == mj) i = -1;
+      if(i == mi || j == mj) j = -1;
+    }
+
+    public int compare(final int mi, final int mj) {
+      final int kvi = offsetFor(mi);
+      final int kvj = offsetFor(mj);
+      final int kvip = kvmeta.get(kvi + PARTITION);
+      final int kvjp = kvmeta.get(kvj + PARTITION);
+      // sort by partition      
+      if (kvip != kvjp) {
+        return kvip - kvjp;
+      }
+      
+      final int istart = kvmeta.get(kvi + KEYSTART);
+      final int jstart = kvmeta.get(kvj + KEYSTART);
+      final int ilen   = kvmeta.get(kvi + VALSTART) - istart;
+      final int jlen   = kvmeta.get(kvj + VALSTART) - jstart;
+
+      kvbuffer.position(istart);
+      kvbuffer.get(ki, 0, ilen);
+      kvbuffer.position(jstart);
+      kvbuffer.get(kj, 0, jlen);
+      // sort by key
+      final int cmp = comparator.compare(ki, 0, ilen, kj, 0, jlen);
+      if(cmp == 0) eq++;
+      return cmp;
+    }
+
+    public SortSpan next() {
+      ByteBuffer remaining = end();
+      if(remaining != null) {
+        int items = length();
+        int perItem = kvbuffer.position()/items;
+        SortSpan newSpan = new SortSpan(remaining, items, perItem);
+        newSpan.index = index+1;
+        return newSpan;
+      }
+      return null;
+    }
+
+    public int length() {
+      return kvmeta.limit()/NMETA;
+    }
+
+    public ByteBuffer end() {
+      ByteBuffer remaining = kvbuffer.duplicate();
+      remaining.position(kvbuffer.position());
+      remaining = remaining.slice();
+      kvbuffer.limit(kvbuffer.position());
+      kvmeta.limit(kvmeta.position());
+      int items = length();
+      if(items == 0) {
+        return null;
+      }
+      int perItem = kvbuffer.position()/items;
+      LOG.info(String.format("Span%d.length = %d, perItem = %d", index, length(), perItem));
+      if(remaining.remaining() < NMETA+perItem) {
+        return null;
+      }
+      return remaining;
+    }
+
+    private int compareInternal(DataInputBuffer needle, int needlePart, int index) {
+      int cmp = 0;
+      int keystart;
+      int valstart;
+      int partition;
+      partition = kvmeta.get(span.offsetFor(index) + PARTITION);
+      if(partition != needlePart) {
+          cmp = (partition-needlePart);
+      } else {
+        keystart = kvmeta.get(span.offsetFor(index) + KEYSTART);
+        valstart = kvmeta.get(span.offsetFor(index) + VALSTART);
+        // hay is allocated ahead of time
+        hay.reset(kvbuffer, keystart, valstart - keystart);
+        cmp = comparator.compare(hay.getData(), 
+            hay.getPosition(), hay.getLength(),
+            needle.getData(), 
+            needle.getPosition(), needle.getLength());
+      }
+      return cmp;
+    }
+    
+    public long getEq() {
+      return eq;
+    }
+    
+    @Override
+    public String toString() {
+        return String.format("Span[%d,%d]", NMETA*kvmeta.capacity(), kvbuffer.limit());
+    }
+  }
+
+  private class SpanIterator implements PartitionedRawKeyValueIterator, Comparable<SpanIterator> {
+    private int kvindex = -1;
+    private int maxindex;
+    private IntBuffer kvmeta;
+    private ByteBuffer kvbuffer;
+    private SortSpan span;
+    private InputByteBuffer key = new InputByteBuffer();
+    private InputByteBuffer value = new InputByteBuffer();
+    private Progress progress = new Progress();
+
+    private final int minrun = (1 << 4);
+
+    public SpanIterator(SortSpan span) {
+      this.kvmeta = span.kvmeta;
+      this.kvbuffer = span.kvbuffer;
+      this.span = span;
+      this.maxindex = (kvmeta.limit()/NMETA) - 1;
+    }
+
+    public DataInputBuffer getKey() throws IOException {
+      final int keystart = kvmeta.get(span.offsetFor(kvindex) + KEYSTART);
+      final int valstart = kvmeta.get(span.offsetFor(kvindex) + VALSTART);
+      key.reset(kvbuffer, keystart, valstart - keystart);
+      return key;
+    }
+
+    public DataInputBuffer getValue() throws IOException {
+      final int valstart = kvmeta.get(span.offsetFor(kvindex) + VALSTART);
+      final int vallen = kvmeta.get(span.offsetFor(kvindex) + VALLEN);
+      value.reset(kvbuffer, valstart, vallen);
+      return value;
+    }
+
+    public boolean next() throws IOException {
+      // caveat: since we use this as a comparable in the merger 
+      if(kvindex == maxindex) return false;
+      if(kvindex % 100 == 0) {
+          progress.set((kvindex-maxindex) / maxindex);
+      }
+      kvindex += 1;
+      return true;
+    }
+
+    public void close() throws IOException {
+    }
+
+    public Progress getProgress() { 
+      return progress;
+    }
+
+    public int getPartition() {
+      final int partition = kvmeta.get(span.offsetFor(kvindex) + PARTITION);
+      return partition;
+    }
+
+    public int size() {
+      return (maxindex - kvindex);
+    }
+
+    public int compareTo(SpanIterator other) {
+      try {
+        return span.compareInternal(other.getKey(), other.getPartition(), kvindex);
+      } catch(IOException ie) {
+        // since we're not reading off disk, how could getKey() throw exceptions?
+      }
+      return -1;
+    }
+    
+    @Override
+    public String toString() {
+        return String.format("SpanIterator<%d:%d> (span=%s)", kvindex, maxindex, span.toString());
+    }
+
+    /**
+     * bisect returns the next insertion point for a given raw key, skipping keys
+     * which are <= needle using a binary search instead of a linear comparison.
+     * This is massively efficient when long strings of identical keys occur.
+     * @param needle 
+     * @param needlePart
+     * @return
+     */
+    int bisect(DataInputBuffer needle, int needlePart) {
+      int start = kvindex;
+      int end = maxindex-1;
+      int mid = start;
+      int cmp = 0;
+
+      if(end - start < minrun) {
+        return 0;
+      }
+
+      if(span.compareInternal(needle, needlePart, start) > 0) {
+        return kvindex;
+      }
+      
+      // bail out early if we haven't got a min run 
+      if(span.compareInternal(needle, needlePart, start+minrun) > 0) {
+        return 0;
+      }
+
+      if(span.compareInternal(needle, needlePart, end) < 0) {
+        return end - kvindex;
+      }
+      
+      boolean found = false;
+      
+      // we sort 100k items, the max it can do is 20 loops, but break early
+      for(int i = 0; start < end && i < 16; i++) {
+        mid = start + (end - start)/2;
+        cmp = span.compareInternal(needle, needlePart, mid);
+        if(cmp == 0) {
+          start = mid;
+          found = true;
+        } else if(cmp < 0) {
+          start = mid; 
+          found = true;
+        }
+        if(cmp > 0) {
+          end = mid;
+        }
+      }
+
+      if(found) {
+        return start - kvindex;
+      }
+      return 0;
+    }
+  }
+
+  private class SortTask implements Callable<SpanIterator> {
+    private final SortSpan sortable;
+    private final IndexedSorter sorter;
+    private final RawComparator comparator;
+    
+    public SortTask(SortSpan sortable, 
+              IndexedSorter sorter, RawComparator comparator) {
+        this.sortable = sortable;
+        this.sorter = sorter;
+        this.comparator = comparator;
+    }
+
+    public SpanIterator call() {
+      return sortable.sort(sorter, comparator);
+    }
+  }
+
+  private class PartitionFilter implements TezRawKeyValueIterator {
+    private final PartitionedRawKeyValueIterator iter;
+    private int partition;
+    private boolean dirty = false;
+    public PartitionFilter(PartitionedRawKeyValueIterator iter) {
+      this.iter = iter;
+    }
+    public DataInputBuffer getKey() throws IOException { return iter.getKey(); }
+    public DataInputBuffer getValue() throws IOException { return iter.getValue(); }
+    public void close() throws IOException { }
+    public Progress getProgress() {
+      return new Progress();
+    }
+    public boolean next() throws IOException {
+      if(dirty || iter.next()) { 
+        int prefix = iter.getPartition();
+
+        if((prefix >>> (32 - partitionBits)) == partition) {
+          dirty = false; // we found what we were looking for, good
+          return true;
+        } else if(!dirty) {
+          dirty = true; // we did a lookahead and failed to find partition
+        }
+      }
+      return false;
+    }
+
+    public void reset(int partition) {
+      this.partition = partition;
+    }
+
+    public int getPartition() {
+      return this.partition;
+    }
+  }
+
+  private class SpanHeap extends java.util.PriorityQueue<SpanIterator> {
+    public SpanHeap() {
+      super(256);
+    }
+    /**
+     * {@link PriorityQueue}.poll() by a different name 
+     * @return
+     */
+    public SpanIterator pop() {
+      return this.poll();
+    }
+  }
+
+  private class SpanMerger implements PartitionedRawKeyValueIterator {
+    private final RawComparator comparator;
+    InputByteBuffer key = new InputByteBuffer();
+    InputByteBuffer value = new InputByteBuffer();
+    int partition;
+
+    private ArrayList< Future<SpanIterator>> futures = new ArrayList< Future<SpanIterator>>();
+
+    private SpanHeap heap = new SpanHeap();
+    private PartitionFilter partIter;
+
+    private int gallop = 0;
+    private SpanIterator horse;
+    private long total = 0;
+    private long count = 0;
+    private long eq = 0;
+    
+    public SpanMerger(RawComparator comparator) {
+      this.comparator = comparator;
+      partIter = new PartitionFilter(this);
+    }
+
+    public void add(SpanIterator iter) throws IOException{
+      if(iter.next()) {
+        heap.add(iter);
+      }
+    }
+
+    public void add(Future<SpanIterator> iter) throws IOException{
+      this.futures.add(iter);
+    }
+
+    public boolean ready() throws IOException, InterruptedException {
+      try {
+        SpanIterator iter = null;
+        while(this.futures.size() > 0) {
+          Future<SpanIterator> futureIter = this.futures.remove(0);
+          iter = futureIter.get();
+          this.add(iter);
+        }
+        
+        StringBuilder sb = new StringBuilder();
+        for(SpanIterator sp: heap) {
+            sb.append(sp.toString());
+            sb.append(",");
+            total += sp.span.length();
+            eq += sp.span.getEq();
+        }
+        LOG.info("Heap = " + sb.toString());
+        return true;
+      } catch(Exception e) {
+        LOG.info(e.toString());
+        return false;
+      }
+    }
+
+    private SpanIterator pop() throws IOException {
+      if(gallop > 0) {
+        gallop--;
+        return horse;
+      }
+      SpanIterator current = heap.pop();
+      SpanIterator next = heap.peek();
+      if(next != null && current != null &&
+        ((Object)horse) == ((Object)current)) {
+        // TODO: a better threshold check
+        gallop = current.bisect(next.getKey(), next.getPartition())-1;
+      }
+      horse = current;
+      return current;
+    }
+    
+    public boolean needsRLE() {
+      return (eq > 0.1 * total);
+    }
+    
+    private SpanIterator peek() throws IOException {
+    	if(gallop > 0) {
+            return horse;
+        }
+    	return heap.peek();
+    }
+
+    public boolean next() throws IOException {
+      SpanIterator current = pop();
+
+      if(current != null) {
+        // keep local copies, since add() will move it all out
+        key.reset(current.getKey());
+        value.reset(current.getValue());
+        partition = current.getPartition();
+        if(gallop <= 0) {
+          this.add(current);
+        } else {
+          // galloping
+          current.next();
+        }
+        return true;
+      }
+      return false;
+    }
+
+    public DataInputBuffer getKey() throws IOException { return key; }
+    public DataInputBuffer getValue() throws IOException { return value; }
+    public int getPartition() { return partition; }
+
+    public void close() throws IOException {
+    }
+
+    public Progress getProgress() {
+      // TODO
+      return new Progress();
+    }
+
+    public TezRawKeyValueIterator filter(int partition) {
+      partIter.reset(partition);
+      return partIter;
+    }
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/TezIndexRecord.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/TezIndexRecord.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/TezIndexRecord.java
new file mode 100644
index 0000000..95ae8eb
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/TezIndexRecord.java
@@ -0,0 +1,45 @@
+/**
+* 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.library.common.sort.impl;
+
+public class TezIndexRecord {
+  private long startOffset;
+  private long rawLength;
+  private long partLength;
+
+  public TezIndexRecord() { }
+
+  public TezIndexRecord(long startOffset, long rawLength, long partLength) {
+    this.startOffset = startOffset;
+    this.rawLength = rawLength;
+    this.partLength = partLength;
+  }
+
+  public long getStartOffset() {
+    return startOffset;
+  }
+
+  public long getRawLength() {
+    return rawLength;
+  }
+
+  public long getPartLength() {
+    return partLength;
+  }
+}


[30/50] [abbrv] Rename tez-engine-api to tez-runtime-api and tez-engine is split into 2: - tez-engine-library for user-visible Input/Output/Processor implementations - tez-engine-internals for framework internals

Posted by ss...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/TezMerger.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/TezMerger.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/TezMerger.java
new file mode 100644
index 0000000..bb4b4a2
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/TezMerger.java
@@ -0,0 +1,798 @@
+/**
+ * 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.library.common.sort.impl;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.ChecksumFileSystem;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.LocalDirAllocator;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.DataInputBuffer;
+import org.apache.hadoop.io.RawComparator;
+import org.apache.hadoop.io.compress.CompressionCodec;
+import org.apache.hadoop.util.PriorityQueue;
+import org.apache.hadoop.util.Progress;
+import org.apache.hadoop.util.Progressable;
+import org.apache.tez.common.TezJobConfig;
+import org.apache.tez.common.counters.TezCounter;
+import org.apache.tez.runtime.library.common.Constants;
+import org.apache.tez.runtime.library.common.sort.impl.IFile.Reader;
+import org.apache.tez.runtime.library.common.sort.impl.IFile.Writer;
+
+/**
+ * Merger is an utility class used by the Map and Reduce tasks for merging
+ * both their memory and disk segments
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+@SuppressWarnings({"unchecked", "rawtypes"})
+public class TezMerger {  
+  private static final Log LOG = LogFactory.getLog(TezMerger.class);
+
+  
+  // Local directories
+  private static LocalDirAllocator lDirAlloc = 
+    new LocalDirAllocator(TezJobConfig.LOCAL_DIRS);
+
+  public static
+  TezRawKeyValueIterator merge(Configuration conf, FileSystem fs,
+                            Class keyClass, Class valueClass, 
+                            CompressionCodec codec,
+                            Path[] inputs, boolean deleteInputs, 
+                            int mergeFactor, Path tmpDir,
+                            RawComparator comparator, Progressable reporter,
+                            TezCounter readsCounter,
+                            TezCounter writesCounter,
+                            Progress mergePhase)
+  throws IOException {
+    return 
+      new MergeQueue(conf, fs, inputs, deleteInputs, codec, comparator, 
+                           reporter, null).merge(keyClass, valueClass,
+                                           mergeFactor, tmpDir,
+                                           readsCounter, writesCounter, 
+                                           mergePhase);
+  }
+
+  public static 
+  TezRawKeyValueIterator merge(Configuration conf, FileSystem fs,
+                            Class keyClass, Class valueClass, 
+                            CompressionCodec codec,
+                            Path[] inputs, boolean deleteInputs, 
+                            int mergeFactor, Path tmpDir,
+                            RawComparator comparator,
+                            Progressable reporter,
+                            TezCounter readsCounter,
+                            TezCounter writesCounter,
+                            TezCounter mergedMapOutputsCounter,
+                            Progress mergePhase)
+  throws IOException {
+    return 
+      new MergeQueue(conf, fs, inputs, deleteInputs, codec, comparator, 
+                           reporter, mergedMapOutputsCounter).merge(
+                                           keyClass, valueClass,
+                                           mergeFactor, tmpDir,
+                                           readsCounter, writesCounter,
+                                           mergePhase);
+  }
+  
+  public static
+  TezRawKeyValueIterator merge(Configuration conf, FileSystem fs, 
+                            Class keyClass, Class valueClass, 
+                            List<Segment> segments, 
+                            int mergeFactor, Path tmpDir,
+                            RawComparator comparator, Progressable reporter,
+                            TezCounter readsCounter,
+                            TezCounter writesCounter,
+                            Progress mergePhase)
+      throws IOException {
+    return merge(conf, fs, keyClass, valueClass, segments, mergeFactor, tmpDir,
+                 comparator, reporter, false, readsCounter, writesCounter,
+                 mergePhase);
+  }
+
+  public static <K extends Object, V extends Object>
+  TezRawKeyValueIterator merge(Configuration conf, FileSystem fs,
+                            Class keyClass, Class valueClass,
+                            List<Segment> segments,
+                            int mergeFactor, Path tmpDir,
+                            RawComparator comparator, Progressable reporter,
+                            boolean sortSegments,
+                            TezCounter readsCounter,
+                            TezCounter writesCounter,
+                            Progress mergePhase)
+      throws IOException {
+    return new MergeQueue(conf, fs, segments, comparator, reporter,
+                           sortSegments).merge(keyClass, valueClass,
+                                               mergeFactor, tmpDir,
+                                               readsCounter, writesCounter,
+                                               mergePhase);
+  }
+
+  public static <K extends Object, V extends Object>
+  TezRawKeyValueIterator merge(Configuration conf, FileSystem fs,
+                            Class keyClass, Class valueClass,
+                            CompressionCodec codec,
+                            List<Segment> segments,
+                            int mergeFactor, Path tmpDir,
+                            RawComparator comparator, Progressable reporter,
+                            boolean sortSegments,
+                            TezCounter readsCounter,
+                            TezCounter writesCounter,
+                            Progress mergePhase)
+      throws IOException {
+    return new MergeQueue(conf, fs, segments, comparator, reporter,
+                           sortSegments, codec).merge(keyClass, valueClass,
+                                               mergeFactor, tmpDir,
+                                               readsCounter, writesCounter,
+                                               mergePhase);
+  }
+
+  public static <K extends Object, V extends Object>
+    TezRawKeyValueIterator merge(Configuration conf, FileSystem fs,
+                            Class keyClass, Class valueClass,
+                            List<Segment> segments,
+                            int mergeFactor, int inMemSegments, Path tmpDir,
+                            RawComparator comparator, Progressable reporter,
+                            boolean sortSegments,
+                            TezCounter readsCounter,
+                            TezCounter writesCounter,
+                            Progress mergePhase)
+      throws IOException {
+    return new MergeQueue(conf, fs, segments, comparator, reporter,
+                           sortSegments).merge(keyClass, valueClass,
+                                               mergeFactor, inMemSegments,
+                                               tmpDir,
+                                               readsCounter, writesCounter,
+                                               mergePhase);
+  }
+
+
+  static <K extends Object, V extends Object>
+  TezRawKeyValueIterator merge(Configuration conf, FileSystem fs,
+                          Class keyClass, Class valueClass,
+                          CompressionCodec codec,
+                          List<Segment> segments,
+                          int mergeFactor, int inMemSegments, Path tmpDir,
+                          RawComparator comparator, Progressable reporter,
+                          boolean sortSegments,
+                          TezCounter readsCounter,
+                          TezCounter writesCounter,
+                          Progress mergePhase)
+    throws IOException {
+  return new MergeQueue(conf, fs, segments, comparator, reporter,
+                         sortSegments, codec).merge(keyClass, valueClass,
+                                             mergeFactor, inMemSegments,
+                                             tmpDir,
+                                             readsCounter, writesCounter,
+                                             mergePhase);
+}
+
+  public static <K extends Object, V extends Object>
+  void writeFile(TezRawKeyValueIterator records, Writer writer, 
+                 Progressable progressable, Configuration conf) 
+  throws IOException {
+    long progressBar = 
+        conf.getLong(TezJobConfig.RECORDS_BEFORE_PROGRESS, 
+            TezJobConfig.DEFAULT_RECORDS_BEFORE_PROGRESS);
+    long recordCtr = 0;
+    while(records.next()) {
+      writer.append(records.getKey(), records.getValue());
+      
+      if (((recordCtr++) % progressBar) == 0) {
+        progressable.progress();
+      }
+    }
+}
+
+  @InterfaceAudience.Private
+  @InterfaceStability.Unstable
+  public static class Segment<K extends Object, V extends Object> {
+    Reader reader = null;
+    final DataInputBuffer key = new DataInputBuffer();
+    
+    Configuration conf = null;
+    FileSystem fs = null;
+    Path file = null;
+    boolean preserve = false;
+    CompressionCodec codec = null;
+    long segmentOffset = 0;
+    long segmentLength = -1;
+    
+    TezCounter mapOutputsCounter = null;
+
+    public Segment(Configuration conf, FileSystem fs, Path file,
+                   CompressionCodec codec, boolean preserve)
+    throws IOException {
+      this(conf, fs, file, codec, preserve, null);
+    }
+
+    public Segment(Configuration conf, FileSystem fs, Path file,
+                   CompressionCodec codec, boolean preserve,
+                   TezCounter mergedMapOutputsCounter)
+  throws IOException {
+      this(conf, fs, file, 0, fs.getFileStatus(file).getLen(), codec, preserve, 
+           mergedMapOutputsCounter);
+    }
+
+    public Segment(Configuration conf, FileSystem fs, Path file,
+                   long segmentOffset, long segmentLength,
+                   CompressionCodec codec,
+                   boolean preserve) throws IOException {
+      this(conf, fs, file, segmentOffset, segmentLength, codec, preserve, null);
+    }
+
+    public Segment(Configuration conf, FileSystem fs, Path file,
+        long segmentOffset, long segmentLength, CompressionCodec codec,
+        boolean preserve, TezCounter mergedMapOutputsCounter)
+    throws IOException {
+      this.conf = conf;
+      this.fs = fs;
+      this.file = file;
+      this.codec = codec;
+      this.preserve = preserve;
+
+      this.segmentOffset = segmentOffset;
+      this.segmentLength = segmentLength;
+      
+      this.mapOutputsCounter = mergedMapOutputsCounter;
+    }
+    
+    public Segment(Reader reader, boolean preserve) {
+      this(reader, preserve, null);
+    }
+    
+    public Segment(Reader reader, boolean preserve, 
+                   TezCounter mapOutputsCounter) {
+      this.reader = reader;
+      this.preserve = preserve;
+      
+      this.segmentLength = reader.getLength();
+      
+      this.mapOutputsCounter = mapOutputsCounter;
+    }
+
+    void init(TezCounter readsCounter) throws IOException {
+      if (reader == null) {
+        FSDataInputStream in = fs.open(file);
+        in.seek(segmentOffset);
+        reader = new Reader(conf, in, segmentLength, codec, readsCounter);
+      }
+      
+      if (mapOutputsCounter != null) {
+        mapOutputsCounter.increment(1);
+      }
+    }
+    
+    boolean inMemory() {
+      return fs == null;
+    }
+    
+    DataInputBuffer getKey() { return key; }
+
+    DataInputBuffer getValue(DataInputBuffer value) throws IOException {
+      nextRawValue(value);
+      return value;
+    }
+
+    public long getLength() { 
+      return (reader == null) ?
+        segmentLength : reader.getLength();
+    }
+    
+    boolean nextRawKey() throws IOException {
+      return reader.nextRawKey(key);
+    }
+
+    void nextRawValue(DataInputBuffer value) throws IOException {
+      reader.nextRawValue(value);
+    }
+
+    void closeReader() throws IOException {
+      if (reader != null) {
+        reader.close();
+        reader = null;
+      }
+    }
+    
+    void close() throws IOException {
+      closeReader();
+      if (!preserve && fs != null) {
+        fs.delete(file, false);
+      }
+    }
+
+    public long getPosition() throws IOException {
+      return reader.getPosition();
+    }
+
+    // This method is used by BackupStore to extract the 
+    // absolute position after a reset
+    long getActualPosition() throws IOException {
+      return segmentOffset + reader.getPosition();
+    }
+
+    Reader getReader() {
+      return reader;
+    }
+    
+    // This method is used by BackupStore to reinitialize the
+    // reader to start reading from a different segment offset
+    void reinitReader(int offset) throws IOException {
+      if (!inMemory()) {
+        closeReader();
+        segmentOffset = offset;
+        segmentLength = fs.getFileStatus(file).getLen() - segmentOffset;
+        init(null);
+      }
+    }
+  }
+  
+  // Boolean variable for including/considering final merge as part of sort
+  // phase or not. This is true in map task, false in reduce task. It is
+  // used in calculating mergeProgress.
+  static boolean includeFinalMerge = false;
+  
+  /**
+   * Sets the boolean variable includeFinalMerge to true. Called from
+   * map task before calling merge() so that final merge of map task
+   * is also considered as part of sort phase.
+   */
+  public static void considerFinalMergeForProgress() {
+    includeFinalMerge = true;
+  }
+  
+  private static class MergeQueue<K extends Object, V extends Object> 
+  extends PriorityQueue<Segment> implements TezRawKeyValueIterator {
+    Configuration conf;
+    FileSystem fs;
+    CompressionCodec codec;
+    
+    List<Segment> segments = new ArrayList<Segment>();
+    
+    RawComparator comparator;
+    
+    private long totalBytesProcessed;
+    private float progPerByte;
+    private Progress mergeProgress = new Progress();
+    
+    Progressable reporter;
+    
+    DataInputBuffer key;
+    final DataInputBuffer value = new DataInputBuffer();
+    final DataInputBuffer diskIFileValue = new DataInputBuffer();
+    
+    Segment minSegment;
+    Comparator<Segment> segmentComparator =   
+      new Comparator<Segment>() {
+      public int compare(Segment o1, Segment o2) {
+        if (o1.getLength() == o2.getLength()) {
+          return 0;
+        }
+
+        return o1.getLength() < o2.getLength() ? -1 : 1;
+      }
+    };
+
+    public MergeQueue(Configuration conf, FileSystem fs, 
+                      Path[] inputs, boolean deleteInputs, 
+                      CompressionCodec codec, RawComparator comparator,
+                      Progressable reporter, 
+                      TezCounter mergedMapOutputsCounter) 
+    throws IOException {
+      this.conf = conf;
+      this.fs = fs;
+      this.codec = codec;
+      this.comparator = comparator;
+      this.reporter = reporter;
+      
+      for (Path file : inputs) {
+        LOG.debug("MergeQ: adding: " + file);
+        segments.add(new Segment(conf, fs, file, codec, !deleteInputs, 
+                                       (file.toString().endsWith(
+                                           Constants.MERGED_OUTPUT_PREFIX) ? 
+                                        null : mergedMapOutputsCounter)));
+      }
+      
+      // Sort segments on file-lengths
+      Collections.sort(segments, segmentComparator); 
+    }
+    
+    public MergeQueue(Configuration conf, FileSystem fs, 
+        List<Segment> segments, RawComparator comparator,
+        Progressable reporter, boolean sortSegments) {
+      this.conf = conf;
+      this.fs = fs;
+      this.comparator = comparator;
+      this.segments = segments;
+      this.reporter = reporter;
+      if (sortSegments) {
+        Collections.sort(segments, segmentComparator);
+      }
+    }
+
+    public MergeQueue(Configuration conf, FileSystem fs,
+        List<Segment> segments, RawComparator comparator,
+        Progressable reporter, boolean sortSegments, CompressionCodec codec) {
+      this(conf, fs, segments, comparator, reporter, sortSegments);
+      this.codec = codec;
+    }
+
+    public void close() throws IOException {
+      Segment segment;
+      while((segment = pop()) != null) {
+        segment.close();
+      }
+    }
+
+    public DataInputBuffer getKey() throws IOException {
+      return key;
+    }
+
+    public DataInputBuffer getValue() throws IOException {
+      return value;
+    }
+
+    private void adjustPriorityQueue(Segment reader) throws IOException{
+      long startPos = reader.getPosition();
+      boolean hasNext = reader.nextRawKey();
+      long endPos = reader.getPosition();
+      totalBytesProcessed += endPos - startPos;
+      mergeProgress.set(totalBytesProcessed * progPerByte);
+      if (hasNext) {
+        adjustTop();
+      } else {
+        pop();
+        reader.close();
+      }
+    }
+
+    public boolean next() throws IOException {
+      if (size() == 0)
+        return false;
+
+      if (minSegment != null) {
+        //minSegment is non-null for all invocations of next except the first
+        //one. For the first invocation, the priority queue is ready for use
+        //but for the subsequent invocations, first adjust the queue 
+        adjustPriorityQueue(minSegment);
+        if (size() == 0) {
+          minSegment = null;
+          return false;
+        }
+      }
+      minSegment = top();
+      if (!minSegment.inMemory()) {
+        //When we load the value from an inmemory segment, we reset
+        //the "value" DIB in this class to the inmem segment's byte[].
+        //When we load the value bytes from disk, we shouldn't use
+        //the same byte[] since it would corrupt the data in the inmem
+        //segment. So we maintain an explicit DIB for value bytes
+        //obtained from disk, and if the current segment is a disk
+        //segment, we reset the "value" DIB to the byte[] in that (so 
+        //we reuse the disk segment DIB whenever we consider
+        //a disk segment).
+        value.reset(diskIFileValue.getData(), diskIFileValue.getLength());
+      }
+      long startPos = minSegment.getPosition();
+      key = minSegment.getKey();
+      minSegment.getValue(value);
+      long endPos = minSegment.getPosition();
+      totalBytesProcessed += endPos - startPos;
+      mergeProgress.set(totalBytesProcessed * progPerByte);
+      return true;
+    }
+
+    protected boolean lessThan(Object a, Object b) {
+      DataInputBuffer key1 = ((Segment)a).getKey();
+      DataInputBuffer key2 = ((Segment)b).getKey();
+      int s1 = key1.getPosition();
+      int l1 = key1.getLength() - s1;
+      int s2 = key2.getPosition();
+      int l2 = key2.getLength() - s2;
+
+      return comparator.compare(key1.getData(), s1, l1, key2.getData(), s2, l2) < 0;
+    }
+    
+    public TezRawKeyValueIterator merge(Class keyClass, Class valueClass,
+                                     int factor, Path tmpDir,
+                                     TezCounter readsCounter,
+                                     TezCounter writesCounter,
+                                     Progress mergePhase)
+        throws IOException {
+      return merge(keyClass, valueClass, factor, 0, tmpDir,
+                   readsCounter, writesCounter, mergePhase);
+    }
+
+    TezRawKeyValueIterator merge(Class keyClass, Class valueClass,
+                                     int factor, int inMem, Path tmpDir,
+                                     TezCounter readsCounter,
+                                     TezCounter writesCounter,
+                                     Progress mergePhase)
+        throws IOException {
+      LOG.info("Merging " + segments.size() + " sorted segments");
+
+      /*
+       * If there are inMemory segments, then they come first in the segments
+       * list and then the sorted disk segments. Otherwise(if there are only
+       * disk segments), then they are sorted segments if there are more than
+       * factor segments in the segments list.
+       */
+      int numSegments = segments.size();
+      int origFactor = factor;
+      int passNo = 1;
+      if (mergePhase != null) {
+        mergeProgress = mergePhase;
+      }
+
+      long totalBytes = computeBytesInMerges(factor, inMem);
+      if (totalBytes != 0) {
+        progPerByte = 1.0f / (float)totalBytes;
+      }
+      
+      //create the MergeStreams from the sorted map created in the constructor
+      //and dump the final output to a file
+      do {
+        //get the factor for this pass of merge. We assume in-memory segments
+        //are the first entries in the segment list and that the pass factor
+        //doesn't apply to them
+        factor = getPassFactor(factor, passNo, numSegments - inMem);
+        if (1 == passNo) {
+          factor += inMem;
+        }
+        List<Segment> segmentsToMerge =
+          new ArrayList<Segment>();
+        int segmentsConsidered = 0;
+        int numSegmentsToConsider = factor;
+        long startBytes = 0; // starting bytes of segments of this merge
+        while (true) {
+          //extract the smallest 'factor' number of segments  
+          //Call cleanup on the empty segments (no key/value data)
+          List<Segment> mStream = 
+            getSegmentDescriptors(numSegmentsToConsider);
+          for (Segment segment : mStream) {
+            // Initialize the segment at the last possible moment;
+            // this helps in ensuring we don't use buffers until we need them
+            segment.init(readsCounter);
+            long startPos = segment.getPosition();
+            boolean hasNext = segment.nextRawKey();
+            long endPos = segment.getPosition();
+            
+            if (hasNext) {
+              startBytes += endPos - startPos;
+              segmentsToMerge.add(segment);
+              segmentsConsidered++;
+            }
+            else {
+              segment.close();
+              numSegments--; //we ignore this segment for the merge
+            }
+          }
+          //if we have the desired number of segments
+          //or looked at all available segments, we break
+          if (segmentsConsidered == factor || 
+              segments.size() == 0) {
+            break;
+          }
+            
+          numSegmentsToConsider = factor - segmentsConsidered;
+        }
+        
+        //feed the streams to the priority queue
+        initialize(segmentsToMerge.size());
+        clear();
+        for (Segment segment : segmentsToMerge) {
+          put(segment);
+        }
+        
+        //if we have lesser number of segments remaining, then just return the
+        //iterator, else do another single level merge
+        if (numSegments <= factor) {
+          if (!includeFinalMerge) { // for reduce task
+
+            // Reset totalBytesProcessed and recalculate totalBytes from the
+            // remaining segments to track the progress of the final merge.
+            // Final merge is considered as the progress of the reducePhase,
+            // the 3rd phase of reduce task.
+            totalBytesProcessed = 0;
+            totalBytes = 0;
+            for (int i = 0; i < segmentsToMerge.size(); i++) {
+              totalBytes += segmentsToMerge.get(i).getLength();
+            }
+          }
+          if (totalBytes != 0) //being paranoid
+            progPerByte = 1.0f / (float)totalBytes;
+          
+          totalBytesProcessed += startBytes;         
+          if (totalBytes != 0)
+            mergeProgress.set(totalBytesProcessed * progPerByte);
+          else
+            mergeProgress.set(1.0f); // Last pass and no segments left - we're done
+          
+          LOG.info("Down to the last merge-pass, with " + numSegments + 
+                   " segments left of total size: " +
+                   (totalBytes - totalBytesProcessed) + " bytes");
+          return this;
+        } else {
+          LOG.info("Merging " + segmentsToMerge.size() + 
+                   " intermediate segments out of a total of " + 
+                   (segments.size()+segmentsToMerge.size()));
+          
+          long bytesProcessedInPrevMerges = totalBytesProcessed;
+          totalBytesProcessed += startBytes;
+
+          //we want to spread the creation of temp files on multiple disks if 
+          //available under the space constraints
+          long approxOutputSize = 0; 
+          for (Segment s : segmentsToMerge) {
+            approxOutputSize += s.getLength() + 
+                                ChecksumFileSystem.getApproxChkSumLength(
+                                s.getLength());
+          }
+          Path tmpFilename = 
+            new Path(tmpDir, "intermediate").suffix("." + passNo);
+
+          Path outputFile =  lDirAlloc.getLocalPathForWrite(
+                                              tmpFilename.toString(),
+                                              approxOutputSize, conf);
+
+          Writer writer = 
+            new Writer(conf, fs, outputFile, keyClass, valueClass, codec,
+                             writesCounter);
+          writeFile(this, writer, reporter, conf);
+          writer.close();
+          
+          //we finished one single level merge; now clean up the priority 
+          //queue
+          this.close();
+
+          // Add the newly create segment to the list of segments to be merged
+          Segment tempSegment = 
+            new Segment(conf, fs, outputFile, codec, false);
+
+          // Insert new merged segment into the sorted list
+          int pos = Collections.binarySearch(segments, tempSegment,
+                                             segmentComparator);
+          if (pos < 0) {
+            // binary search failed. So position to be inserted at is -pos-1
+            pos = -pos-1;
+          }
+          segments.add(pos, tempSegment);
+          numSegments = segments.size();
+          
+          // Subtract the difference between expected size of new segment and 
+          // actual size of new segment(Expected size of new segment is
+          // inputBytesOfThisMerge) from totalBytes. Expected size and actual
+          // size will match(almost) if combiner is not called in merge.
+          long inputBytesOfThisMerge = totalBytesProcessed -
+                                       bytesProcessedInPrevMerges;
+          totalBytes -= inputBytesOfThisMerge - tempSegment.getLength();
+          if (totalBytes != 0) {
+            progPerByte = 1.0f / (float)totalBytes;
+          }
+          
+          passNo++;
+        }
+        //we are worried about only the first pass merge factor. So reset the 
+        //factor to what it originally was
+        factor = origFactor;
+      } while(true);
+    }
+    
+    /**
+     * Determine the number of segments to merge in a given pass. Assuming more
+     * than factor segments, the first pass should attempt to bring the total
+     * number of segments - 1 to be divisible by the factor - 1 (each pass
+     * takes X segments and produces 1) to minimize the number of merges.
+     */
+    private int getPassFactor(int factor, int passNo, int numSegments) {
+      if (passNo > 1 || numSegments <= factor || factor == 1) 
+        return factor;
+      int mod = (numSegments - 1) % (factor - 1);
+      if (mod == 0)
+        return factor;
+      return mod + 1;
+    }
+    
+    /** Return (& remove) the requested number of segment descriptors from the
+     * sorted map.
+     */
+    private List<Segment> getSegmentDescriptors(int numDescriptors) {
+      if (numDescriptors > segments.size()) {
+        List<Segment> subList = new ArrayList<Segment>(segments);
+        segments.clear();
+        return subList;
+      }
+      
+      List<Segment> subList = 
+        new ArrayList<Segment>(segments.subList(0, numDescriptors));
+      for (int i=0; i < numDescriptors; ++i) {
+        segments.remove(0);
+      }
+      return subList;
+    }
+    
+    /**
+     * Compute expected size of input bytes to merges, will be used in
+     * calculating mergeProgress. This simulates the above merge() method and
+     * tries to obtain the number of bytes that are going to be merged in all
+     * merges(assuming that there is no combiner called while merging).
+     * @param factor mapreduce.task.io.sort.factor
+     * @param inMem  number of segments in memory to be merged
+     */
+    long computeBytesInMerges(int factor, int inMem) {
+      int numSegments = segments.size();
+      List<Long> segmentSizes = new ArrayList<Long>(numSegments);
+      long totalBytes = 0;
+      int n = numSegments - inMem;
+      // factor for 1st pass
+      int f = getPassFactor(factor, 1, n) + inMem;
+      n = numSegments;
+ 
+      for (int i = 0; i < numSegments; i++) {
+        // Not handling empty segments here assuming that it would not affect
+        // much in calculation of mergeProgress.
+        segmentSizes.add(segments.get(i).getLength());
+      }
+      
+      // If includeFinalMerge is true, allow the following while loop iterate
+      // for 1 more iteration. This is to include final merge as part of the
+      // computation of expected input bytes of merges
+      boolean considerFinalMerge = includeFinalMerge;
+      
+      while (n > f || considerFinalMerge) {
+        if (n <=f ) {
+          considerFinalMerge = false;
+        }
+        long mergedSize = 0;
+        f = Math.min(f, segmentSizes.size());
+        for (int j = 0; j < f; j++) {
+          mergedSize += segmentSizes.remove(0);
+        }
+        totalBytes += mergedSize;
+        
+        // insert new size into the sorted list
+        int pos = Collections.binarySearch(segmentSizes, mergedSize);
+        if (pos < 0) {
+          pos = -pos-1;
+        }
+        segmentSizes.add(pos, mergedSize);
+        
+        n -= (f-1);
+        f = factor;
+      }
+
+      return totalBytes;
+    }
+
+    public Progress getProgress() {
+      return mergeProgress;
+    }
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/TezRawKeyValueIterator.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/TezRawKeyValueIterator.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/TezRawKeyValueIterator.java
new file mode 100644
index 0000000..3a2c2bf
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/TezRawKeyValueIterator.java
@@ -0,0 +1,70 @@
+/**
+ * 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.library.common.sort.impl;
+
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.io.DataInputBuffer;
+import org.apache.hadoop.util.Progress;
+
+/**
+ * <code>TezRawKeyValueIterator</code> is an iterator used to iterate over
+ * the raw keys and values during sort/merge of intermediate data. 
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public interface TezRawKeyValueIterator {
+  /** 
+   * Gets the current raw key.
+   * 
+   * @return Gets the current raw key as a DataInputBuffer
+   * @throws IOException
+   */
+  DataInputBuffer getKey() throws IOException;
+  
+  /** 
+   * Gets the current raw value.
+   * 
+   * @return Gets the current raw value as a DataInputBuffer 
+   * @throws IOException
+   */
+  DataInputBuffer getValue() throws IOException;
+  
+  /** 
+   * Sets up the current key and value (for getKey and getValue).
+   * 
+   * @return <code>true</code> if there exists a key/value, 
+   *         <code>false</code> otherwise. 
+   * @throws IOException
+   */
+  boolean next() throws IOException;
+  
+  /** 
+   * Closes the iterator so that the underlying streams can be closed.
+   * 
+   * @throws IOException
+   */
+  void close() throws IOException;
+  
+  /** Gets the Progress object; this has a float (0.0 - 1.0) 
+   * indicating the bytes processed by the iterator so far
+   */
+  Progress getProgress();
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/TezSpillRecord.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/TezSpillRecord.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/TezSpillRecord.java
new file mode 100644
index 0000000..ab4142b
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/TezSpillRecord.java
@@ -0,0 +1,146 @@
+/**
+ * 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.library.common.sort.impl;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.LongBuffer;
+import java.util.zip.CheckedInputStream;
+import java.util.zip.CheckedOutputStream;
+import java.util.zip.Checksum;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.ChecksumException;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.util.PureJavaCrc32;
+import org.apache.tez.runtime.library.common.Constants;
+
+public class TezSpillRecord {
+
+  /** Backing store */
+  private final ByteBuffer buf;
+  /** View of backing storage as longs */
+  private final LongBuffer entries;
+
+  public TezSpillRecord(int numPartitions) {
+    buf = ByteBuffer.allocate(
+        numPartitions * Constants.MAP_OUTPUT_INDEX_RECORD_LENGTH);
+    entries = buf.asLongBuffer();
+  }
+
+  public TezSpillRecord(Path indexFileName, Configuration job) throws IOException {
+    this(indexFileName, job, null);
+  }
+
+  public TezSpillRecord(Path indexFileName, Configuration job, String expectedIndexOwner)
+    throws IOException {
+    this(indexFileName, job, new PureJavaCrc32(), expectedIndexOwner);
+  }
+
+  public TezSpillRecord(Path indexFileName, Configuration job, Checksum crc,
+                     String expectedIndexOwner)
+      throws IOException {
+
+    final FileSystem rfs = FileSystem.getLocal(job).getRaw();
+    final FSDataInputStream in = rfs.open(indexFileName);
+    try {
+      final long length = rfs.getFileStatus(indexFileName).getLen();
+      final int partitions = 
+          (int) length / Constants.MAP_OUTPUT_INDEX_RECORD_LENGTH;
+      final int size = partitions * Constants.MAP_OUTPUT_INDEX_RECORD_LENGTH;
+
+      buf = ByteBuffer.allocate(size);
+      if (crc != null) {
+        crc.reset();
+        CheckedInputStream chk = new CheckedInputStream(in, crc);
+        IOUtils.readFully(chk, buf.array(), 0, size);
+        if (chk.getChecksum().getValue() != in.readLong()) {
+          throw new ChecksumException("Checksum error reading spill index: " +
+                                indexFileName, -1);
+        }
+      } else {
+        IOUtils.readFully(in, buf.array(), 0, size);
+      }
+      entries = buf.asLongBuffer();
+    } finally {
+      in.close();
+    }
+  }
+
+  /**
+   * Return number of IndexRecord entries in this spill.
+   */
+  public int size() {
+    return entries.capacity() / (Constants.MAP_OUTPUT_INDEX_RECORD_LENGTH / 8);
+  }
+
+  /**
+   * Get spill offsets for given partition.
+   */
+  public TezIndexRecord getIndex(int partition) {
+    final int pos = partition * Constants.MAP_OUTPUT_INDEX_RECORD_LENGTH / 8;
+    return new TezIndexRecord(entries.get(pos), entries.get(pos + 1),
+                           entries.get(pos + 2));
+  }
+
+  /**
+   * Set spill offsets for given partition.
+   */
+  public void putIndex(TezIndexRecord rec, int partition) {
+    final int pos = partition * Constants.MAP_OUTPUT_INDEX_RECORD_LENGTH / 8;
+    entries.put(pos, rec.getStartOffset());
+    entries.put(pos + 1, rec.getRawLength());
+    entries.put(pos + 2, rec.getPartLength());
+  }
+
+  /**
+   * Write this spill record to the location provided.
+   */
+  public void writeToFile(Path loc, Configuration job)
+      throws IOException {
+    writeToFile(loc, job, new PureJavaCrc32());
+  }
+
+  public void writeToFile(Path loc, Configuration job, Checksum crc)
+      throws IOException {
+    final FileSystem rfs = FileSystem.getLocal(job).getRaw();
+    CheckedOutputStream chk = null;
+    final FSDataOutputStream out = rfs.create(loc);
+    try {
+      if (crc != null) {
+        crc.reset();
+        chk = new CheckedOutputStream(out, crc);
+        chk.write(buf.array());
+        out.writeLong(chk.getChecksum().getValue());
+      } else {
+        out.write(buf.array());
+      }
+    } finally {
+      if (chk != null) {
+        chk.close();
+      } else {
+        out.close();
+      }
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/dflt/DefaultSorter.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/dflt/DefaultSorter.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/dflt/DefaultSorter.java
new file mode 100644
index 0000000..1ff486f
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/dflt/DefaultSorter.java
@@ -0,0 +1,1108 @@
+/**
+* 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.library.common.sort.impl.dflt;
+
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.nio.IntBuffer;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+
+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.Path;
+import org.apache.hadoop.io.DataInputBuffer;
+import org.apache.hadoop.io.RawComparator;
+import org.apache.hadoop.util.IndexedSortable;
+import org.apache.hadoop.util.Progress;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.tez.common.TezJobConfig;
+import org.apache.tez.runtime.api.TezOutputContext;
+import org.apache.tez.runtime.library.common.ConfigUtils;
+import org.apache.tez.runtime.library.common.sort.impl.ExternalSorter;
+import org.apache.tez.runtime.library.common.sort.impl.IFile;
+import org.apache.tez.runtime.library.common.sort.impl.TezIndexRecord;
+import org.apache.tez.runtime.library.common.sort.impl.TezMerger;
+import org.apache.tez.runtime.library.common.sort.impl.TezRawKeyValueIterator;
+import org.apache.tez.runtime.library.common.sort.impl.TezSpillRecord;
+import org.apache.tez.runtime.library.common.sort.impl.IFile.Writer;
+import org.apache.tez.runtime.library.common.sort.impl.TezMerger.Segment;
+
+@SuppressWarnings({"unchecked", "rawtypes"})
+public class DefaultSorter extends ExternalSorter implements IndexedSortable {
+  
+  private static final Log LOG = LogFactory.getLog(DefaultSorter.class);
+
+  // TODO NEWTEZ Progress reporting to Tez framework. (making progress vs %complete)
+  
+  /**
+   * The size of each record in the index file for the map-outputs.
+   */
+  public static final int MAP_OUTPUT_INDEX_RECORD_LENGTH = 24;
+
+  private final static int APPROX_HEADER_LENGTH = 150;
+
+  // k/v accounting
+  IntBuffer kvmeta; // metadata overlay on backing store
+  int kvstart;            // marks origin of spill metadata
+  int kvend;              // marks end of spill metadata
+  int kvindex;            // marks end of fully serialized records
+
+  int equator;            // marks origin of meta/serialization
+  int bufstart;           // marks beginning of spill
+  int bufend;             // marks beginning of collectable
+  int bufmark;            // marks end of record
+  int bufindex;           // marks end of collected
+  int bufvoid;            // marks the point where we should stop
+                          // reading at the end of the buffer
+
+  byte[] kvbuffer;        // main output buffer
+  private final byte[] b0 = new byte[0];
+
+  protected static final int INDEX = 0;            // index offset in acct
+  protected static final int VALSTART = 1;         // val offset in acct
+  protected static final int KEYSTART = 2;         // key offset in acct
+  protected static final int PARTITION = 3;        // partition offset in acct
+  protected static final int NMETA = 4;            // num meta ints
+  protected static final int METASIZE = NMETA * 4; // size in bytes
+
+  // spill accounting
+  int maxRec;
+  int softLimit;
+  boolean spillInProgress;
+  int bufferRemaining;
+  volatile Throwable sortSpillException = null;
+
+  int numSpills = 0;
+  int minSpillsForCombine;
+  final ReentrantLock spillLock = new ReentrantLock();
+  final Condition spillDone = spillLock.newCondition();
+  final Condition spillReady = spillLock.newCondition();
+  final BlockingBuffer bb = new BlockingBuffer();
+  volatile boolean spillThreadRunning = false;
+  final SpillThread spillThread = new SpillThread();
+
+  final ArrayList<TezSpillRecord> indexCacheList =
+    new ArrayList<TezSpillRecord>();
+  private int totalIndexCacheMemory;
+  private int indexCacheMemoryLimit;
+
+  @Override
+  public void initialize(TezOutputContext outputContext, Configuration conf, int numOutputs) throws IOException { 
+    super.initialize(outputContext, conf, numOutputs);
+
+    // sanity checks
+    final float spillper = this.conf.getFloat(
+        TezJobConfig.TEZ_RUNTIME_SORT_SPILL_PERCENT,
+        TezJobConfig.DEFAULT_TEZ_RUNTIME_SORT_SPILL_PERCENT);
+    final int sortmb = this.conf.getInt(TezJobConfig.TEZ_RUNTIME_IO_SORT_MB,
+        TezJobConfig.DEFAULT_TEZ_RUNTIME_IO_SORT_MB);
+    if (spillper > (float) 1.0 || spillper <= (float) 0.0) {
+      throw new IOException("Invalid \""
+          + TezJobConfig.TEZ_RUNTIME_SORT_SPILL_PERCENT + "\": " + spillper);
+    }
+    if ((sortmb & 0x7FF) != sortmb) {
+      throw new IOException("Invalid \"" + TezJobConfig.TEZ_RUNTIME_IO_SORT_MB
+          + "\": " + sortmb);
+    }
+
+    indexCacheMemoryLimit = this.conf.getInt(TezJobConfig.TEZ_RUNTIME_INDEX_CACHE_MEMORY_LIMIT_BYTES,
+                                       TezJobConfig.DEFAULT_TEZ_RUNTIME_INDEX_CACHE_MEMORY_LIMIT_BYTES);
+
+    // buffers and accounting
+    int maxMemUsage = sortmb << 20;
+    maxMemUsage -= maxMemUsage % METASIZE;
+    kvbuffer = new byte[maxMemUsage];
+    bufvoid = kvbuffer.length;
+    kvmeta = ByteBuffer.wrap(kvbuffer)
+       .order(ByteOrder.nativeOrder())
+       .asIntBuffer();
+    setEquator(0);
+    bufstart = bufend = bufindex = equator;
+    kvstart = kvend = kvindex;
+
+    maxRec = kvmeta.capacity() / NMETA;
+    softLimit = (int)(kvbuffer.length * spillper);
+    bufferRemaining = softLimit;
+    if (LOG.isInfoEnabled()) {
+      LOG.info(TezJobConfig.TEZ_RUNTIME_IO_SORT_MB + ": " + sortmb);
+      LOG.info("soft limit at " + softLimit);
+      LOG.info("bufstart = " + bufstart + "; bufvoid = " + bufvoid);
+      LOG.info("kvstart = " + kvstart + "; length = " + maxRec);
+    }
+
+    // k/v serialization
+    valSerializer.open(bb);
+    keySerializer.open(bb);
+
+    spillInProgress = false;
+    minSpillsForCombine = this.conf.getInt(TezJobConfig.TEZ_RUNTIME_COMBINE_MIN_SPILLS, 3);
+    spillThread.setDaemon(true);
+    spillThread.setName("SpillThread");
+    spillLock.lock();
+    try {
+      spillThread.start();
+      while (!spillThreadRunning) {
+        spillDone.await();
+      }
+    } catch (InterruptedException e) {
+      throw new IOException("Spill thread failed to initialize", e);
+    } finally {
+      spillLock.unlock();
+    }
+    if (sortSpillException != null) {
+      throw new IOException("Spill thread failed to initialize",
+          sortSpillException);
+    }
+  }
+
+  @Override
+  public void write(Object key, Object value)
+      throws IOException {
+    collect(
+        key, value, partitioner.getPartition(key, value, partitions));
+  }
+
+  /**
+   * Serialize the key, value to intermediate storage.
+   * When this method returns, kvindex must refer to sufficient unused
+   * storage to store one METADATA.
+   */
+  synchronized void collect(Object key, Object value, final int partition
+                                   ) throws IOException {
+
+    if (key.getClass() != keyClass) {
+      throw new IOException("Type mismatch in key from map: expected "
+                            + keyClass.getName() + ", received "
+                            + key.getClass().getName());
+    }
+    if (value.getClass() != valClass) {
+      throw new IOException("Type mismatch in value from map: expected "
+                            + valClass.getName() + ", received "
+                            + value.getClass().getName());
+    }
+    if (partition < 0 || partition >= partitions) {
+      throw new IOException("Illegal partition for " + key + " (" +
+          partition + ")" + ", TotalPartitions: " + partitions);
+    }
+    checkSpillException();
+    bufferRemaining -= METASIZE;
+    if (bufferRemaining <= 0) {
+      // start spill if the thread is not running and the soft limit has been
+      // reached
+      spillLock.lock();
+      try {
+        do {
+          if (!spillInProgress) {
+            final int kvbidx = 4 * kvindex;
+            final int kvbend = 4 * kvend;
+            // serialized, unspilled bytes always lie between kvindex and
+            // bufindex, crossing the equator. Note that any void space
+            // created by a reset must be included in "used" bytes
+            final int bUsed = distanceTo(kvbidx, bufindex);
+            final boolean bufsoftlimit = bUsed >= softLimit;
+            if ((kvbend + METASIZE) % kvbuffer.length !=
+                equator - (equator % METASIZE)) {
+              // spill finished, reclaim space
+              resetSpill();
+              bufferRemaining = Math.min(
+                  distanceTo(bufindex, kvbidx) - 2 * METASIZE,
+                  softLimit - bUsed) - METASIZE;
+              continue;
+            } else if (bufsoftlimit && kvindex != kvend) {
+              // spill records, if any collected; check latter, as it may
+              // be possible for metadata alignment to hit spill pcnt
+              startSpill();
+              final int avgRec = (int)
+                (mapOutputByteCounter.getValue() /
+                mapOutputRecordCounter.getValue());
+              // leave at least half the split buffer for serialization data
+              // ensure that kvindex >= bufindex
+              final int distkvi = distanceTo(bufindex, kvbidx);
+              final int newPos = (bufindex +
+                Math.max(2 * METASIZE - 1,
+                        Math.min(distkvi / 2,
+                                 distkvi / (METASIZE + avgRec) * METASIZE)))
+                % kvbuffer.length;
+              setEquator(newPos);
+              bufmark = bufindex = newPos;
+              final int serBound = 4 * kvend;
+              // bytes remaining before the lock must be held and limits
+              // checked is the minimum of three arcs: the metadata space, the
+              // serialization space, and the soft limit
+              bufferRemaining = Math.min(
+                  // metadata max
+                  distanceTo(bufend, newPos),
+                  Math.min(
+                    // serialization max
+                    distanceTo(newPos, serBound),
+                    // soft limit
+                    softLimit)) - 2 * METASIZE;
+            }
+          }
+        } while (false);
+      } finally {
+        spillLock.unlock();
+      }
+    }
+
+    try {
+      // serialize key bytes into buffer
+      int keystart = bufindex;
+      keySerializer.serialize(key);
+      if (bufindex < keystart) {
+        // wrapped the key; must make contiguous
+        bb.shiftBufferedKey();
+        keystart = 0;
+      }
+      // serialize value bytes into buffer
+      final int valstart = bufindex;
+      valSerializer.serialize(value);
+      // It's possible for records to have zero length, i.e. the serializer
+      // will perform no writes. To ensure that the boundary conditions are
+      // checked and that the kvindex invariant is maintained, perform a
+      // zero-length write into the buffer. The logic monitoring this could be
+      // moved into collect, but this is cleaner and inexpensive. For now, it
+      // is acceptable.
+      bb.write(b0, 0, 0);
+
+      // the record must be marked after the preceding write, as the metadata
+      // for this record are not yet written
+      int valend = bb.markRecord();
+
+      mapOutputRecordCounter.increment(1);
+      mapOutputByteCounter.increment(
+          distanceTo(keystart, valend, bufvoid));
+
+      // write accounting info
+      kvmeta.put(kvindex + INDEX, kvindex);
+      kvmeta.put(kvindex + PARTITION, partition);
+      kvmeta.put(kvindex + KEYSTART, keystart);
+      kvmeta.put(kvindex + VALSTART, valstart);
+      // advance kvindex
+      kvindex = (kvindex - NMETA + kvmeta.capacity()) % kvmeta.capacity();
+    } catch (MapBufferTooSmallException e) {
+      LOG.info("Record too large for in-memory buffer: " + e.getMessage());
+      spillSingleRecord(key, value, partition);
+      mapOutputRecordCounter.increment(1);
+      return;
+    }
+  }
+
+  /**
+   * Set the point from which meta and serialization data expand. The meta
+   * indices are aligned with the buffer, so metadata never spans the ends of
+   * the circular buffer.
+   */
+  private void setEquator(int pos) {
+    equator = pos;
+    // set index prior to first entry, aligned at meta boundary
+    final int aligned = pos - (pos % METASIZE);
+    kvindex =
+      ((aligned - METASIZE + kvbuffer.length) % kvbuffer.length) / 4;
+    if (LOG.isInfoEnabled()) {
+      LOG.info("(EQUATOR) " + pos + " kvi " + kvindex +
+          "(" + (kvindex * 4) + ")");
+    }
+  }
+
+  /**
+   * The spill is complete, so set the buffer and meta indices to be equal to
+   * the new equator to free space for continuing collection. Note that when
+   * kvindex == kvend == kvstart, the buffer is empty.
+   */
+  private void resetSpill() {
+    final int e = equator;
+    bufstart = bufend = e;
+    final int aligned = e - (e % METASIZE);
+    // set start/end to point to first meta record
+    kvstart = kvend =
+      ((aligned - METASIZE + kvbuffer.length) % kvbuffer.length) / 4;
+    if (LOG.isInfoEnabled()) {
+      LOG.info("(RESET) equator " + e + " kv " + kvstart + "(" +
+        (kvstart * 4) + ")" + " kvi " + kvindex + "(" + (kvindex * 4) + ")");
+    }
+  }
+
+  /**
+   * Compute the distance in bytes between two indices in the serialization
+   * buffer.
+   * @see #distanceTo(int,int,int)
+   */
+  final int distanceTo(final int i, final int j) {
+    return distanceTo(i, j, kvbuffer.length);
+  }
+
+  /**
+   * Compute the distance between two indices in the circular buffer given the
+   * max distance.
+   */
+  int distanceTo(final int i, final int j, final int mod) {
+    return i <= j
+      ? j - i
+      : mod - i + j;
+  }
+
+  /**
+   * For the given meta position, return the dereferenced position in the
+   * integer array. Each meta block contains several integers describing
+   * record data in its serialized form, but the INDEX is not necessarily
+   * related to the proximate metadata. The index value at the referenced int
+   * position is the start offset of the associated metadata block. So the
+   * metadata INDEX at metapos may point to the metadata described by the
+   * metadata block at metapos + k, which contains information about that
+   * serialized record.
+   */
+  int offsetFor(int metapos) {
+    return kvmeta.get((metapos % maxRec) * NMETA + INDEX);
+  }
+
+  /**
+   * Compare logical range, st i, j MOD offset capacity.
+   * Compare by partition, then by key.
+   * @see IndexedSortable#compare
+   */
+  public int compare(final int mi, final int mj) {
+    final int kvi = offsetFor(mi);
+    final int kvj = offsetFor(mj);
+    final int kvip = kvmeta.get(kvi + PARTITION);
+    final int kvjp = kvmeta.get(kvj + PARTITION);
+    // sort by partition
+    if (kvip != kvjp) {
+      return kvip - kvjp;
+    }
+    // sort by key
+    return comparator.compare(kvbuffer,
+        kvmeta.get(kvi + KEYSTART),
+        kvmeta.get(kvi + VALSTART) - kvmeta.get(kvi + KEYSTART),
+        kvbuffer,
+        kvmeta.get(kvj + KEYSTART),
+        kvmeta.get(kvj + VALSTART) - kvmeta.get(kvj + KEYSTART));
+  }
+
+  /**
+   * Swap logical indices st i, j MOD offset capacity.
+   * @see IndexedSortable#swap
+   */
+  public void swap(final int mi, final int mj) {
+    final int kvi = (mi % maxRec) * NMETA + INDEX;
+    final int kvj = (mj % maxRec) * NMETA + INDEX;
+    int tmp = kvmeta.get(kvi);
+    kvmeta.put(kvi, kvmeta.get(kvj));
+    kvmeta.put(kvj, tmp);
+  }
+
+  /**
+   * Inner class managing the spill of serialized records to disk.
+   */
+  protected class BlockingBuffer extends DataOutputStream {
+
+    public BlockingBuffer() {
+      super(new Buffer());
+    }
+
+    /**
+     * Mark end of record. Note that this is required if the buffer is to
+     * cut the spill in the proper place.
+     */
+    public int markRecord() {
+      bufmark = bufindex;
+      return bufindex;
+    }
+
+    /**
+     * Set position from last mark to end of writable buffer, then rewrite
+     * the data between last mark and kvindex.
+     * This handles a special case where the key wraps around the buffer.
+     * If the key is to be passed to a RawComparator, then it must be
+     * contiguous in the buffer. This recopies the data in the buffer back
+     * into itself, but starting at the beginning of the buffer. Note that
+     * this method should <b>only</b> be called immediately after detecting
+     * this condition. To call it at any other time is undefined and would
+     * likely result in data loss or corruption.
+     * @see #markRecord()
+     */
+    protected void shiftBufferedKey() throws IOException {
+      // spillLock unnecessary; both kvend and kvindex are current
+      int headbytelen = bufvoid - bufmark;
+      bufvoid = bufmark;
+      final int kvbidx = 4 * kvindex;
+      final int kvbend = 4 * kvend;
+      final int avail =
+        Math.min(distanceTo(0, kvbidx), distanceTo(0, kvbend));
+      if (bufindex + headbytelen < avail) {
+        System.arraycopy(kvbuffer, 0, kvbuffer, headbytelen, bufindex);
+        System.arraycopy(kvbuffer, bufvoid, kvbuffer, 0, headbytelen);
+        bufindex += headbytelen;
+        bufferRemaining -= kvbuffer.length - bufvoid;
+      } else {
+        byte[] keytmp = new byte[bufindex];
+        System.arraycopy(kvbuffer, 0, keytmp, 0, bufindex);
+        bufindex = 0;
+        out.write(kvbuffer, bufmark, headbytelen);
+        out.write(keytmp);
+      }
+    }
+  }
+
+  public class Buffer extends OutputStream {
+    private final byte[] scratch = new byte[1];
+
+    @Override
+    public void write(int v)
+        throws IOException {
+      scratch[0] = (byte)v;
+      write(scratch, 0, 1);
+    }
+
+    /**
+     * Attempt to write a sequence of bytes to the collection buffer.
+     * This method will block if the spill thread is running and it
+     * cannot write.
+     * @throws MapBufferTooSmallException if record is too large to
+     *    deserialize into the collection buffer.
+     */
+    @Override
+    public void write(byte b[], int off, int len)
+        throws IOException {
+      // must always verify the invariant that at least METASIZE bytes are
+      // available beyond kvindex, even when len == 0
+      bufferRemaining -= len;
+      if (bufferRemaining <= 0) {
+        // writing these bytes could exhaust available buffer space or fill
+        // the buffer to soft limit. check if spill or blocking are necessary
+        boolean blockwrite = false;
+        spillLock.lock();
+        try {
+          do {
+            checkSpillException();
+
+            final int kvbidx = 4 * kvindex;
+            final int kvbend = 4 * kvend;
+            // ser distance to key index
+            final int distkvi = distanceTo(bufindex, kvbidx);
+            // ser distance to spill end index
+            final int distkve = distanceTo(bufindex, kvbend);
+
+            // if kvindex is closer than kvend, then a spill is neither in
+            // progress nor complete and reset since the lock was held. The
+            // write should block only if there is insufficient space to
+            // complete the current write, write the metadata for this record,
+            // and write the metadata for the next record. If kvend is closer,
+            // then the write should block if there is too little space for
+            // either the metadata or the current write. Note that collect
+            // ensures its metadata requirement with a zero-length write
+            blockwrite = distkvi <= distkve
+              ? distkvi <= len + 2 * METASIZE
+              : distkve <= len || distanceTo(bufend, kvbidx) < 2 * METASIZE;
+
+            if (!spillInProgress) {
+              if (blockwrite) {
+                if ((kvbend + METASIZE) % kvbuffer.length !=
+                    equator - (equator % METASIZE)) {
+                  // spill finished, reclaim space
+                  // need to use meta exclusively; zero-len rec & 100% spill
+                  // pcnt would fail
+                  resetSpill(); // resetSpill doesn't move bufindex, kvindex
+                  bufferRemaining = Math.min(
+                      distkvi - 2 * METASIZE,
+                      softLimit - distanceTo(kvbidx, bufindex)) - len;
+                  continue;
+                }
+                // we have records we can spill; only spill if blocked
+                if (kvindex != kvend) {
+                  startSpill();
+                  // Blocked on this write, waiting for the spill just
+                  // initiated to finish. Instead of repositioning the marker
+                  // and copying the partial record, we set the record start
+                  // to be the new equator
+                  setEquator(bufmark);
+                } else {
+                  // We have no buffered records, and this record is too large
+                  // to write into kvbuffer. We must spill it directly from
+                  // collect
+                  final int size = distanceTo(bufstart, bufindex) + len;
+                  setEquator(0);
+                  bufstart = bufend = bufindex = equator;
+                  kvstart = kvend = kvindex;
+                  bufvoid = kvbuffer.length;
+                  throw new MapBufferTooSmallException(size + " bytes");
+                }
+              }
+            }
+
+            if (blockwrite) {
+              // wait for spill
+              try {
+                while (spillInProgress) {
+                  spillDone.await();
+                }
+              } catch (InterruptedException e) {
+                  throw new IOException(
+                      "Buffer interrupted while waiting for the writer", e);
+              }
+            }
+          } while (blockwrite);
+        } finally {
+          spillLock.unlock();
+        }
+      }
+      // here, we know that we have sufficient space to write
+      if (bufindex + len > bufvoid) {
+        final int gaplen = bufvoid - bufindex;
+        System.arraycopy(b, off, kvbuffer, bufindex, gaplen);
+        len -= gaplen;
+        off += gaplen;
+        bufindex = 0;
+      }
+      System.arraycopy(b, off, kvbuffer, bufindex, len);
+      bufindex += len;
+    }
+  }
+
+  @Override
+  public void flush() throws IOException {
+    LOG.info("Starting flush of map output");
+    spillLock.lock();
+    try {
+      while (spillInProgress) {
+        spillDone.await();
+      }
+      checkSpillException();
+
+      final int kvbend = 4 * kvend;
+      if ((kvbend + METASIZE) % kvbuffer.length !=
+          equator - (equator % METASIZE)) {
+        // spill finished
+        resetSpill();
+      }
+      if (kvindex != kvend) {
+        kvend = (kvindex + NMETA) % kvmeta.capacity();
+        bufend = bufmark;
+        if (LOG.isInfoEnabled()) {
+          LOG.info("Sorting & Spilling map output");
+          LOG.info("bufstart = " + bufstart + "; bufend = " + bufmark +
+                   "; bufvoid = " + bufvoid);
+          LOG.info("kvstart = " + kvstart + "(" + (kvstart * 4) +
+                   "); kvend = " + kvend + "(" + (kvend * 4) +
+                   "); length = " + (distanceTo(kvend, kvstart,
+                         kvmeta.capacity()) + 1) + "/" + maxRec);
+        }
+        sortAndSpill();
+      }
+    } catch (InterruptedException e) {
+      throw new IOException("Interrupted while waiting for the writer", e);
+    } finally {
+      spillLock.unlock();
+    }
+    assert !spillLock.isHeldByCurrentThread();
+    // shut down spill thread and wait for it to exit. Since the preceding
+    // ensures that it is finished with its work (and sortAndSpill did not
+    // throw), we elect to use an interrupt instead of setting a flag.
+    // Spilling simultaneously from this thread while the spill thread
+    // finishes its work might be both a useful way to extend this and also
+    // sufficient motivation for the latter approach.
+    try {
+      spillThread.interrupt();
+      spillThread.join();
+    } catch (InterruptedException e) {
+      throw new IOException("Spill failed", e);
+    }
+    // release sort buffer before the merge
+    //FIXME
+    //kvbuffer = null;
+    mergeParts();
+    Path outputPath = mapOutputFile.getOutputFile();
+    fileOutputByteCounter.increment(rfs.getFileStatus(outputPath).getLen());
+  }
+
+  @Override
+  public void close() throws IOException { }
+
+  protected class SpillThread extends Thread {
+
+    @Override
+    public void run() {
+      spillLock.lock();
+      spillThreadRunning = true;
+      try {
+        while (true) {
+          spillDone.signal();
+          while (!spillInProgress) {
+            spillReady.await();
+          }
+          try {
+            spillLock.unlock();
+            sortAndSpill();
+          } catch (Throwable t) {
+            LOG.warn("Got an exception in sortAndSpill", t);
+            sortSpillException = t;
+          } finally {
+            spillLock.lock();
+            if (bufend < bufstart) {
+              bufvoid = kvbuffer.length;
+            }
+            kvstart = kvend;
+            bufstart = bufend;
+            spillInProgress = false;
+          }
+        }
+      } catch (InterruptedException e) {
+        Thread.currentThread().interrupt();
+      } finally {
+        spillLock.unlock();
+        spillThreadRunning = false;
+      }
+    }
+  }
+
+  private void checkSpillException() throws IOException {
+    final Throwable lspillException = sortSpillException;
+    if (lspillException != null) {
+      if (lspillException instanceof Error) {
+        final String logMsg = "Task " + outputContext.getUniqueIdentifier()
+            + " failed : " + StringUtils.stringifyException(lspillException);
+        outputContext.fatalError(lspillException, logMsg);
+      }
+      throw new IOException("Spill failed", lspillException);
+    }
+  }
+
+  private void startSpill() {
+    assert !spillInProgress;
+    kvend = (kvindex + NMETA) % kvmeta.capacity();
+    bufend = bufmark;
+    spillInProgress = true;
+    if (LOG.isInfoEnabled()) {
+      LOG.info("Spilling map output");
+      LOG.info("bufstart = " + bufstart + "; bufend = " + bufmark +
+               "; bufvoid = " + bufvoid);
+      LOG.info("kvstart = " + kvstart + "(" + (kvstart * 4) +
+               "); kvend = " + kvend + "(" + (kvend * 4) +
+               "); length = " + (distanceTo(kvend, kvstart,
+                     kvmeta.capacity()) + 1) + "/" + maxRec);
+    }
+    spillReady.signal();
+  }
+
+  int getMetaStart() {
+    return kvend / NMETA;
+  }
+
+  int getMetaEnd() {
+    return 1 + // kvend is a valid record
+        (kvstart >= kvend
+        ? kvstart
+        : kvmeta.capacity() + kvstart) / NMETA;
+  }
+
+  protected void sortAndSpill()
+      throws IOException, InterruptedException {
+    final int mstart = getMetaStart();
+    final int mend = getMetaEnd();
+    sorter.sort(this, mstart, mend, nullProgressable);
+    spill(mstart, mend);
+  }
+
+  protected void spill(int mstart, int mend)
+      throws IOException, InterruptedException {
+
+    //approximate the length of the output file to be the length of the
+    //buffer + header lengths for the partitions
+    final long size = (bufend >= bufstart
+        ? bufend - bufstart
+        : (bufvoid - bufend) + bufstart) +
+                partitions * APPROX_HEADER_LENGTH;
+    FSDataOutputStream out = null;
+    try {
+      // create spill file
+      final TezSpillRecord spillRec = new TezSpillRecord(partitions);
+      final Path filename =
+          mapOutputFile.getSpillFileForWrite(numSpills, size);
+      out = rfs.create(filename);
+
+      int spindex = mstart;
+      final InMemValBytes value = createInMemValBytes();
+      for (int i = 0; i < partitions; ++i) {
+        IFile.Writer writer = null;
+        try {
+          long segmentStart = out.getPos();
+          writer = new Writer(conf, out, keyClass, valClass, codec,
+                                    spilledRecordsCounter);
+          if (combiner == null) {
+            // spill directly
+            DataInputBuffer key = new DataInputBuffer();
+            while (spindex < mend &&
+                kvmeta.get(offsetFor(spindex) + PARTITION) == i) {
+              final int kvoff = offsetFor(spindex);
+              key.reset(
+                  kvbuffer,
+                  kvmeta.get(kvoff + KEYSTART),
+                  (kvmeta.get(kvoff + VALSTART) - kvmeta.get(kvoff + KEYSTART))
+                  );
+              getVBytesForOffset(kvoff, value);
+              writer.append(key, value);
+              ++spindex;
+            }
+          } else {
+            int spstart = spindex;
+            while (spindex < mend &&
+                kvmeta.get(offsetFor(spindex)
+                          + PARTITION) == i) {
+              ++spindex;
+            }
+            // Note: we would like to avoid the combiner if we've fewer
+            // than some threshold of records for a partition
+            if (spstart != spindex) {
+              TezRawKeyValueIterator kvIter =
+                new MRResultIterator(spstart, spindex);
+              if (LOG.isDebugEnabled()) {
+                LOG.debug("Running combine processor");
+              }
+              runCombineProcessor(kvIter, writer);
+            }
+          }
+
+          // close the writer
+          writer.close();
+
+          // record offsets
+          final TezIndexRecord rec =
+              new TezIndexRecord(
+                  segmentStart,
+                  writer.getRawLength(),
+                  writer.getCompressedLength());
+          spillRec.putIndex(rec, i);
+
+          writer = null;
+        } finally {
+          if (null != writer) writer.close();
+        }
+      }
+
+      if (totalIndexCacheMemory >= indexCacheMemoryLimit) {
+        // create spill index file
+        Path indexFilename =
+            mapOutputFile.getSpillIndexFileForWrite(numSpills, partitions
+                * MAP_OUTPUT_INDEX_RECORD_LENGTH);
+        spillRec.writeToFile(indexFilename, conf);
+      } else {
+        indexCacheList.add(spillRec);
+        totalIndexCacheMemory +=
+          spillRec.size() * MAP_OUTPUT_INDEX_RECORD_LENGTH;
+      }
+      LOG.info("Finished spill " + numSpills);
+      ++numSpills;
+    } finally {
+      if (out != null) out.close();
+    }
+  }
+
+  /**
+   * Handles the degenerate case where serialization fails to fit in
+   * the in-memory buffer, so we must spill the record from collect
+   * directly to a spill file. Consider this "losing".
+   */
+  private void spillSingleRecord(final Object key, final Object value,
+                                 int partition) throws IOException {
+    long size = kvbuffer.length + partitions * APPROX_HEADER_LENGTH;
+    FSDataOutputStream out = null;
+    try {
+      // create spill file
+      final TezSpillRecord spillRec = new TezSpillRecord(partitions);
+      final Path filename =
+          mapOutputFile.getSpillFileForWrite(numSpills, size);
+      out = rfs.create(filename);
+
+      // we don't run the combiner for a single record
+      for (int i = 0; i < partitions; ++i) {
+        IFile.Writer writer = null;
+        try {
+          long segmentStart = out.getPos();
+          // Create a new codec, don't care!
+          writer = new IFile.Writer(conf, out, keyClass, valClass, codec,
+                                          spilledRecordsCounter);
+
+          if (i == partition) {
+            final long recordStart = out.getPos();
+            writer.append(key, value);
+            // Note that our map byte count will not be accurate with
+            // compression
+            mapOutputByteCounter.increment(out.getPos() - recordStart);
+          }
+          writer.close();
+
+          // record offsets
+          TezIndexRecord rec =
+              new TezIndexRecord(
+                  segmentStart,
+                  writer.getRawLength(),
+                  writer.getCompressedLength());
+          spillRec.putIndex(rec, i);
+
+          writer = null;
+        } catch (IOException e) {
+          if (null != writer) writer.close();
+          throw e;
+        }
+      }
+      if (totalIndexCacheMemory >= indexCacheMemoryLimit) {
+        // create spill index file
+        Path indexFilename =
+            mapOutputFile.getSpillIndexFileForWrite(numSpills, partitions
+                * MAP_OUTPUT_INDEX_RECORD_LENGTH);
+        spillRec.writeToFile(indexFilename, conf);
+      } else {
+        indexCacheList.add(spillRec);
+        totalIndexCacheMemory +=
+          spillRec.size() * MAP_OUTPUT_INDEX_RECORD_LENGTH;
+      }
+      ++numSpills;
+    } finally {
+      if (out != null) out.close();
+    }
+  }
+
+  protected int getInMemVBytesLength(int kvoff) {
+    // get the keystart for the next serialized value to be the end
+    // of this value. If this is the last value in the buffer, use bufend
+    final int nextindex = kvoff == kvend
+      ? bufend
+      : kvmeta.get(
+          (kvoff - NMETA + kvmeta.capacity() + KEYSTART) % kvmeta.capacity());
+    // calculate the length of the value
+    int vallen = (nextindex >= kvmeta.get(kvoff + VALSTART))
+      ? nextindex - kvmeta.get(kvoff + VALSTART)
+      : (bufvoid - kvmeta.get(kvoff + VALSTART)) + nextindex;
+      return vallen;
+  }
+
+  /**
+   * Given an offset, populate vbytes with the associated set of
+   * deserialized value bytes. Should only be called during a spill.
+   */
+  int getVBytesForOffset(int kvoff, InMemValBytes vbytes) {
+    int vallen = getInMemVBytesLength(kvoff);
+    vbytes.reset(kvbuffer, kvmeta.get(kvoff + VALSTART), vallen);
+    return vallen;
+  }
+
+  /**
+   * Inner class wrapping valuebytes, used for appendRaw.
+   */
+  static class InMemValBytes extends DataInputBuffer {
+    private byte[] buffer;
+    private int start;
+    private int length;
+    private final int bufvoid;
+
+    public InMemValBytes(int bufvoid) {
+      this.bufvoid = bufvoid;
+    }
+
+    public void reset(byte[] buffer, int start, int length) {
+      this.buffer = buffer;
+      this.start = start;
+      this.length = length;
+
+      if (start + length > bufvoid) {
+        this.buffer = new byte[this.length];
+        final int taillen = bufvoid - start;
+        System.arraycopy(buffer, start, this.buffer, 0, taillen);
+        System.arraycopy(buffer, 0, this.buffer, taillen, length-taillen);
+        this.start = 0;
+      }
+
+      super.reset(this.buffer, this.start, this.length);
+    }
+  }
+
+  InMemValBytes createInMemValBytes() {
+    return new InMemValBytes(bufvoid);
+  }
+
+  protected class MRResultIterator implements TezRawKeyValueIterator {
+    private final DataInputBuffer keybuf = new DataInputBuffer();
+    private final InMemValBytes vbytes = createInMemValBytes();
+    private final int end;
+    private int current;
+    public MRResultIterator(int start, int end) {
+      this.end = end;
+      current = start - 1;
+    }
+    public boolean next() throws IOException {
+      return ++current < end;
+    }
+    public DataInputBuffer getKey() throws IOException {
+      final int kvoff = offsetFor(current);
+      keybuf.reset(kvbuffer, kvmeta.get(kvoff + KEYSTART),
+          kvmeta.get(kvoff + VALSTART) - kvmeta.get(kvoff + KEYSTART));
+      return keybuf;
+    }
+    public DataInputBuffer getValue() throws IOException {
+      getVBytesForOffset(offsetFor(current), vbytes);
+      return vbytes;
+    }
+    public Progress getProgress() {
+      return null;
+    }
+    public void close() { }
+  }
+
+  private void mergeParts() throws IOException {
+    // get the approximate size of the final output/index files
+    long finalOutFileSize = 0;
+    long finalIndexFileSize = 0;
+    final Path[] filename = new Path[numSpills];
+    final String taskIdentifier = outputContext.getUniqueIdentifier();
+
+    for(int i = 0; i < numSpills; i++) {
+      filename[i] = mapOutputFile.getSpillFile(i);
+      finalOutFileSize += rfs.getFileStatus(filename[i]).getLen();
+    }
+    if (numSpills == 1) { //the spill is the final output
+      sameVolRename(filename[0],
+          mapOutputFile.getOutputFileForWriteInVolume(filename[0]));
+      if (indexCacheList.size() == 0) {
+        sameVolRename(mapOutputFile.getSpillIndexFile(0),
+          mapOutputFile.getOutputIndexFileForWriteInVolume(filename[0]));
+      } else {
+        indexCacheList.get(0).writeToFile(
+          mapOutputFile.getOutputIndexFileForWriteInVolume(filename[0]), conf);
+      }
+      return;
+    }
+
+    // read in paged indices
+    for (int i = indexCacheList.size(); i < numSpills; ++i) {
+      Path indexFileName = mapOutputFile.getSpillIndexFile(i);
+      indexCacheList.add(new TezSpillRecord(indexFileName, conf));
+    }
+
+    //make correction in the length to include the sequence file header
+    //lengths for each partition
+    finalOutFileSize += partitions * APPROX_HEADER_LENGTH;
+    finalIndexFileSize = partitions * MAP_OUTPUT_INDEX_RECORD_LENGTH;
+    Path finalOutputFile =
+        mapOutputFile.getOutputFileForWrite(finalOutFileSize);
+    Path finalIndexFile =
+        mapOutputFile.getOutputIndexFileForWrite(finalIndexFileSize);
+
+    //The output stream for the final single output file
+    FSDataOutputStream finalOut = rfs.create(finalOutputFile, true, 4096);
+
+    if (numSpills == 0) {
+      //create dummy files
+
+      TezSpillRecord sr = new TezSpillRecord(partitions);
+      try {
+        for (int i = 0; i < partitions; i++) {
+          long segmentStart = finalOut.getPos();
+          Writer writer =
+            new Writer(conf, finalOut, keyClass, valClass, codec, null);
+          writer.close();
+
+          TezIndexRecord rec =
+              new TezIndexRecord(
+                  segmentStart,
+                  writer.getRawLength(),
+                  writer.getCompressedLength());
+          sr.putIndex(rec, i);
+        }
+        sr.writeToFile(finalIndexFile, conf);
+      } finally {
+        finalOut.close();
+      }
+      return;
+    }
+    else {
+      TezMerger.considerFinalMergeForProgress();
+
+      final TezSpillRecord spillRec = new TezSpillRecord(partitions);
+      for (int parts = 0; parts < partitions; parts++) {
+        //create the segments to be merged
+        List<Segment> segmentList =
+          new ArrayList<Segment>(numSpills);
+        for(int i = 0; i < numSpills; i++) {
+          TezIndexRecord indexRecord = indexCacheList.get(i).getIndex(parts);
+
+          Segment s =
+            new Segment(conf, rfs, filename[i], indexRecord.getStartOffset(),
+                             indexRecord.getPartLength(), codec, true);
+          segmentList.add(i, s);
+
+          if (LOG.isDebugEnabled()) {
+            LOG.debug("TaskIdentifier=" + taskIdentifier + " Partition=" + parts +
+                "Spill =" + i + "(" + indexRecord.getStartOffset() + "," +
+                indexRecord.getRawLength() + ", " +
+                indexRecord.getPartLength() + ")");
+          }
+        }
+
+        int mergeFactor =
+            this.conf.getInt(TezJobConfig.TEZ_RUNTIME_IO_SORT_FACTOR,
+                TezJobConfig.DEFAULT_TEZ_RUNTIME_IO_SORT_FACTOR);
+        // sort the segments only if there are intermediate merges
+        boolean sortSegments = segmentList.size() > mergeFactor;
+        //merge
+        TezRawKeyValueIterator kvIter = TezMerger.merge(conf, rfs,
+                       keyClass, valClass, codec,
+                       segmentList, mergeFactor,
+                       new Path(taskIdentifier),
+                       (RawComparator)ConfigUtils.getIntermediateOutputKeyComparator(conf),
+                       nullProgressable, sortSegments,
+                       null, spilledRecordsCounter,
+                       null); // Not using any Progress in TezMerger. Should just work.
+
+        //write merged output to disk
+        long segmentStart = finalOut.getPos();
+        Writer writer =
+            new Writer(conf, finalOut, keyClass, valClass, codec,
+                spilledRecordsCounter);
+        if (combiner == null || numSpills < minSpillsForCombine) {
+          TezMerger.writeFile(kvIter, writer,
+              nullProgressable, conf);
+        } else {
+          runCombineProcessor(kvIter, writer);
+        }
+        writer.close();
+
+        // record offsets
+        final TezIndexRecord rec =
+            new TezIndexRecord(
+                segmentStart,
+                writer.getRawLength(),
+                writer.getCompressedLength());
+        spillRec.putIndex(rec, parts);
+      }
+      spillRec.writeToFile(finalIndexFile, conf);
+      finalOut.close();
+      for(int i = 0; i < numSpills; i++) {
+        rfs.delete(filename[i],true);
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/dflt/InMemoryShuffleSorter.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/dflt/InMemoryShuffleSorter.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/dflt/InMemoryShuffleSorter.java
new file mode 100644
index 0000000..92ae916
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/dflt/InMemoryShuffleSorter.java
@@ -0,0 +1,126 @@
+/**
+* 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.library.common.sort.impl.dflt;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.IntBuffer;
+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.io.WritableUtils;
+import org.apache.hadoop.util.DataChecksum;
+import org.apache.tez.runtime.api.TezOutputContext;
+import org.apache.tez.runtime.library.common.shuffle.impl.ShuffleHeader;
+import org.apache.tez.runtime.library.common.shuffle.server.ShuffleHandler;
+import org.apache.tez.runtime.library.common.sort.impl.IFile;
+
+public class InMemoryShuffleSorter extends DefaultSorter {
+
+  private static final Log LOG = LogFactory.getLog(InMemoryShuffleSorter.class);
+  
+  static final int IFILE_EOF_LENGTH = 
+      2 * WritableUtils.getVIntSize(IFile.EOF_MARKER);
+  static final int IFILE_CHECKSUM_LENGTH = DataChecksum.Type.CRC32.size;
+  
+  private List<Integer> spillIndices = new ArrayList<Integer>();
+  private List<ShuffleHeader> shuffleHeaders = new ArrayList<ShuffleHeader>();
+
+  ShuffleHandler shuffleHandler = new ShuffleHandler(this);
+  
+  byte[] kvbuffer;
+  IntBuffer kvmeta;
+
+  @Override
+  public void initialize(TezOutputContext outputContext, Configuration conf, int numOutputs) throws IOException {
+    super.initialize(outputContext, conf, numOutputs);
+    shuffleHandler.initialize(outputContext, conf);
+  }
+
+  @Override
+  protected void spill(int mstart, int mend) 
+      throws IOException, InterruptedException {
+    // Start the shuffleHandler
+    shuffleHandler.start();
+
+    // Don't spill!
+    
+    // Make a copy
+    this.kvbuffer = super.kvbuffer;
+    this.kvmeta = super.kvmeta;
+
+    // Just save spill-indices for serving later
+    int spindex = mstart;
+    for (int i = 0; i < partitions; ++i) {
+      spillIndices.add(spindex);
+      
+      int length = 0;
+      while (spindex < mend &&
+          kvmeta.get(offsetFor(spindex) + PARTITION) == i) {
+
+        final int kvoff = offsetFor(spindex);
+        int keyLen = 
+            kvmeta.get(kvoff + VALSTART) - kvmeta.get(kvoff + KEYSTART);
+        int valLen = getInMemVBytesLength(kvoff);
+        length += 
+            (keyLen + WritableUtils.getVIntSize(keyLen)) + 
+            (valLen + WritableUtils.getVIntSize(valLen));
+
+        ++spindex;
+      }
+      length += IFILE_EOF_LENGTH;
+      
+      shuffleHeaders.add( 
+          new ShuffleHeader(
+              outputContext.getUniqueIdentifier(), // TODO Verify that this is correct. 
+              length + IFILE_CHECKSUM_LENGTH, length, i)
+          );
+      LOG.info("shuffleHeader[" + i + "]:" +
+      		" rawLen=" + length + " partLen=" + (length + IFILE_CHECKSUM_LENGTH) + 
+          " spillIndex=" + spillIndices.get(i));
+    }
+    
+    LOG.info("Saved " + spillIndices.size() + " spill-indices and " + 
+        shuffleHeaders.size() + " shuffle headers");
+  }
+
+  @Override
+  public InputStream getSortedStream(int partition) {
+    return new SortBufferInputStream(this, partition);
+  }
+
+  @Override
+  public void close() throws IOException {
+    // FIXME
+    //shuffleHandler.stop();
+  }
+
+  @Override
+  public ShuffleHeader getShuffleHeader(int reduce) {
+    return shuffleHeaders.get(reduce);
+  }
+
+  public int getSpillIndex(int partition) {
+    return spillIndices.get(partition);
+  }
+
+}


[08/50] [abbrv] TEZ-444. Rename *.new* packages back to what they should be, remove dead code from the old packages - mapreduce module (part of TEZ-398). (sseth)

Posted by ss...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/3d609458/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/newprocessor/MRTask.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/newprocessor/MRTask.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/newprocessor/MRTask.java
deleted file mode 100644
index d71dba0..0000000
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/newprocessor/MRTask.java
+++ /dev/null
@@ -1,731 +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.newprocessor;
-
-import java.io.IOException;
-import java.net.URI;
-import java.text.NumberFormat;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.concurrent.atomic.AtomicBoolean;
-
-import javax.crypto.SecretKey;
-
-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.fs.FileSystem;
-import org.apache.hadoop.fs.LocalDirAllocator;
-import org.apache.hadoop.fs.FileSystem.Statistics;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.DataInputBuffer;
-import org.apache.hadoop.io.RawComparator;
-import org.apache.hadoop.mapred.FileAlreadyExistsException;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.JobContext;
-import org.apache.hadoop.mapred.MapOutputFile;
-import org.apache.hadoop.mapred.RawKeyValueIterator;
-import org.apache.hadoop.mapred.TaskAttemptContext;
-import org.apache.hadoop.mapred.TaskAttemptID;
-import org.apache.hadoop.mapred.TaskID;
-import org.apache.hadoop.mapreduce.OutputCommitter;
-import org.apache.hadoop.mapreduce.TaskType;
-import org.apache.hadoop.mapreduce.filecache.DistributedCache;
-import org.apache.hadoop.mapreduce.lib.reduce.WrappedReducer;
-import org.apache.hadoop.mapreduce.security.token.JobTokenSecretManager;
-import org.apache.hadoop.mapreduce.task.ReduceContextImpl;
-import org.apache.hadoop.security.Credentials;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.security.token.Token;
-import org.apache.hadoop.util.Progress;
-import org.apache.hadoop.util.StringUtils;
-import org.apache.hadoop.util.DiskChecker.DiskErrorException;
-import org.apache.hadoop.yarn.api.ApplicationConstants.Environment;
-import org.apache.hadoop.yarn.util.ResourceCalculatorProcessTree;
-import org.apache.tez.common.Constants;
-import org.apache.tez.common.TezJobConfig;
-import org.apache.tez.common.TezTaskStatus.State;
-import org.apache.tez.common.TezUtils;
-import org.apache.tez.common.counters.TaskCounter;
-import org.apache.tez.common.counters.TezCounter;
-import org.apache.tez.common.counters.TezCounters;
-import org.apache.tez.dag.records.TezDAGID;
-import org.apache.tez.engine.common.security.JobTokenIdentifier;
-import org.apache.tez.engine.common.security.TokenCache;
-import org.apache.tez.engine.common.sort.impl.TezRawKeyValueIterator;
-import org.apache.tez.engine.newapi.LogicalOutput;
-import org.apache.tez.engine.newapi.TezProcessorContext;
-import org.apache.tez.engine.records.OutputContext;
-import org.apache.tez.mapreduce.hadoop.DeprecatedKeys;
-import org.apache.tez.mapreduce.hadoop.IDConverter;
-import org.apache.tez.mapreduce.hadoop.MRConfig;
-import org.apache.tez.mapreduce.hadoop.MRJobConfig;
-import org.apache.tez.mapreduce.hadoop.newmapred.TaskAttemptContextImpl;
-import org.apache.tez.mapreduce.hadoop.mapreduce.JobContextImpl;
-import org.apache.tez.mapreduce.newoutput.SimpleOutput;
-//import org.apache.tez.mapreduce.partition.MRPartitioner;
-import org.apache.tez.mapreduce.task.impl.YarnOutputFiles;
-
-@SuppressWarnings("deprecation")
-public abstract class MRTask {
-
-  static final Log LOG = LogFactory.getLog(MRTask.class);
-
-  protected JobConf jobConf;
-  protected JobContext jobContext;
-  protected TaskAttemptContext taskAttemptContext;
-  protected OutputCommitter committer;
-
-  // Current counters
-  transient TezCounters counters;
-  protected GcTimeUpdater gcUpdater;
-  private ResourceCalculatorProcessTree pTree;
-  private long initCpuCumulativeTime = 0;
-  protected TezProcessorContext processorContext;
-  protected TaskAttemptID taskAttemptId;
-  protected Progress progress = new Progress();
-  protected SecretKey jobTokenSecret;
-
-  boolean isMap;
-
-  /* flag to track whether task is done */
-  AtomicBoolean taskDone = new AtomicBoolean(false);
-
-  /** Construct output file names so that, when an output directory listing is
-   * sorted lexicographically, positions correspond to output partitions.*/
-  private static final NumberFormat NUMBER_FORMAT = NumberFormat.getInstance();
-  static {
-    NUMBER_FORMAT.setMinimumIntegerDigits(5);
-    NUMBER_FORMAT.setGroupingUsed(false);
-  }
-
-  protected MRTaskReporter mrReporter;
-  protected boolean useNewApi;
-
-  /**
-   * A Map where Key-> URIScheme and value->FileSystemStatisticUpdater
-   */
-  private Map<String, FileSystemStatisticUpdater> statisticUpdaters =
-     new HashMap<String, FileSystemStatisticUpdater>();
-
-  public MRTask(boolean isMap) {
-    this.isMap = isMap;
-  }
-
-  // TODO how to update progress
-  public void initialize(TezProcessorContext context) throws IOException,
-  InterruptedException {
-
-    DeprecatedKeys.init();
-
-    processorContext = context;
-    counters = context.getCounters();
-    this.taskAttemptId = new TaskAttemptID(
-        new TaskID(
-            Long.toString(context.getApplicationId().getClusterTimestamp()),
-            context.getApplicationId().getId(),
-            (isMap ? TaskType.MAP : TaskType.REDUCE),
-            context.getTaskIndex()),
-          context.getTaskAttemptNumber());
-    // TODO TEZAM4 Figure out initialization / run sequence of Input, Process,
-    // Output. Phase is MR specific.
-    gcUpdater = new GcTimeUpdater(counters);
-
-    byte[] userPayload = context.getUserPayload();
-    Configuration conf = TezUtils.createConfFromUserPayload(userPayload);
-    if (conf instanceof JobConf) {
-      this.jobConf = (JobConf)conf;
-    } else {
-      this.jobConf = new JobConf(conf);
-    }
-    jobConf.set(Constants.TEZ_ENGINE_TASK_ATTEMPT_ID,
-        taskAttemptId.toString());
-    jobConf.setInt(MRJobConfig.APPLICATION_ATTEMPT_ID,
-        context.getDAGAttemptNumber());
-
-    initResourceCalculatorPlugin();
-
-    LOG.info("MRTask.inited: taskAttemptId = " + taskAttemptId.toString());
-
-    // TODO Post MRR
-    // A single file per vertex will likely be a better solution. Does not
-    // require translation - client can take care of this. Will work independent
-    // of whether the configuration is for intermediate tasks or not. Has the
-    // overhead of localizing multiple files per job - i.e. the client would
-    // need to write these files to hdfs, add them as local resources per
-    // vertex. A solution like this may be more practical once it's possible to
-    // submit configuration parameters to the AM and effectively tasks via RPC.
-
-    jobConf.set(MRJobConfig.VERTEX_NAME, processorContext.getTaskVertexName());
-
-    if (LOG.isDebugEnabled() && userPayload != null) {
-      Iterator<Entry<String, String>> iter = jobConf.iterator();
-      String taskIdStr = taskAttemptId.getTaskID().toString();
-      while (iter.hasNext()) {
-        Entry<String, String> confEntry = iter.next();
-        LOG.debug("TaskConf Entry"
-            + ", taskId=" + taskIdStr
-            + ", key=" + confEntry.getKey()
-            + ", value=" + confEntry.getValue());
-      }
-    }
-
-    configureMRTask();
-  }
-
-  private void configureMRTask()
-      throws IOException, InterruptedException {
-
-    Credentials credentials = UserGroupInformation.getCurrentUser()
-        .getCredentials();
-    jobConf.setCredentials(credentials);
-    // TODO Can this be avoided all together. Have the MRTezOutputCommitter use
-    // the Tez parameter.
-    // TODO This could be fetched from the env if YARN is setting it for all
-    // Containers.
-    // Set it in conf, so as to be able to be used the the OutputCommitter.
-
-    jobConf.setClass(MRConfig.TASK_LOCAL_OUTPUT_CLASS, YarnOutputFiles.class,
-        MapOutputFile.class); // MR
-
-    // Not needed. This is probably being set via the source/consumer meta
-    Token<JobTokenIdentifier> jobToken = TokenCache.getJobToken(credentials);
-    if (jobToken != null) {
-      // Will MR ever run without a job token.
-      SecretKey sk = JobTokenSecretManager.createSecretKey(jobToken
-          .getPassword());
-      this.jobTokenSecret = sk;
-    } else {
-      LOG.warn("No job token set");
-    }
-
-    configureLocalDirs();
-
-    if (jobConf.get(TezJobConfig.DAG_CREDENTIALS_BINARY) != null) {
-      jobConf.set(MRJobConfig.MAPREDUCE_JOB_CREDENTIALS_BINARY,
-          jobConf.get(TezJobConfig.DAG_CREDENTIALS_BINARY));
-    }
-
-    // Set up the DistributedCache related configs
-    setupDistributedCacheConfig(jobConf);
-  }
-
-  private void configureLocalDirs() throws IOException {
-    // TODO NEWTEZ Is most of this functionality required ?
-    jobConf.setStrings(TezJobConfig.LOCAL_DIRS, processorContext.getWorkDirs());
-    jobConf.set(TezJobConfig.TASK_LOCAL_RESOURCE_DIR, System.getenv(Environment.PWD.name()));
-
-    jobConf.setStrings(MRConfig.LOCAL_DIR, processorContext.getWorkDirs());
-
-    LocalDirAllocator lDirAlloc = new LocalDirAllocator(TezJobConfig.LOCAL_DIRS);
-    Path workDir = null;
-    // First, try to find the JOB_LOCAL_DIR on this host.
-    try {
-      workDir = lDirAlloc.getLocalPathToRead("work", jobConf);
-    } catch (DiskErrorException e) {
-      // DiskErrorException means dir not found. If not found, it will
-      // be created below.
-    }
-    if (workDir == null) {
-      // JOB_LOCAL_DIR doesn't exist on this host -- Create it.
-      workDir = lDirAlloc.getLocalPathForWrite("work", jobConf);
-      FileSystem lfs = FileSystem.getLocal(jobConf).getRaw();
-      boolean madeDir = false;
-      try {
-        madeDir = lfs.mkdirs(workDir);
-      } catch (FileAlreadyExistsException e) {
-        // Since all tasks will be running in their own JVM, the race condition
-        // exists where multiple tasks could be trying to create this directory
-        // at the same time. If this task loses the race, it's okay because
-        // the directory already exists.
-        madeDir = true;
-        workDir = lDirAlloc.getLocalPathToRead("work", jobConf);
-      }
-      if (!madeDir) {
-          throw new IOException("Mkdirs failed to create "
-              + workDir.toString());
-      }
-    }
-    // TODO NEWTEZ Is this required ?
-    jobConf.set(TezJobConfig.JOB_LOCAL_DIR, workDir.toString());
-    jobConf.set(MRJobConfig.JOB_LOCAL_DIR, workDir.toString());
-  }
-
-  /**
-   * Set up the DistributedCache related configs to make
-   * {@link DistributedCache#getLocalCacheFiles(Configuration)} and
-   * {@link DistributedCache#getLocalCacheArchives(Configuration)} working.
-   *
-   * @param job
-   * @throws IOException
-   */
-  private static void setupDistributedCacheConfig(final JobConf job)
-      throws IOException {
-
-    String localWorkDir = (job.get(TezJobConfig.TASK_LOCAL_RESOURCE_DIR));
-    // ^ ^ all symlinks are created in the current work-dir
-
-    // Update the configuration object with localized archives.
-    URI[] cacheArchives = DistributedCache.getCacheArchives(job);
-    if (cacheArchives != null) {
-      List<String> localArchives = new ArrayList<String>();
-      for (int i = 0; i < cacheArchives.length; ++i) {
-        URI u = cacheArchives[i];
-        Path p = new Path(u);
-        Path name = new Path((null == u.getFragment()) ? p.getName()
-            : u.getFragment());
-        String linkName = name.toUri().getPath();
-        localArchives.add(new Path(localWorkDir, linkName).toUri().getPath());
-      }
-      if (!localArchives.isEmpty()) {
-        job.set(MRJobConfig.CACHE_LOCALARCHIVES, StringUtils
-            .arrayToString(localArchives.toArray(new String[localArchives
-                .size()])));
-      }
-    }
-
-    // Update the configuration object with localized files.
-    URI[] cacheFiles = DistributedCache.getCacheFiles(job);
-    if (cacheFiles != null) {
-      List<String> localFiles = new ArrayList<String>();
-      for (int i = 0; i < cacheFiles.length; ++i) {
-        URI u = cacheFiles[i];
-        Path p = new Path(u);
-        Path name = new Path((null == u.getFragment()) ? p.getName()
-            : u.getFragment());
-        String linkName = name.toUri().getPath();
-        localFiles.add(new Path(localWorkDir, linkName).toUri().getPath());
-      }
-      if (!localFiles.isEmpty()) {
-        job.set(MRJobConfig.CACHE_LOCALFILES, StringUtils
-            .arrayToString(localFiles.toArray(new String[localFiles.size()])));
-      }
-    }
-  }
-
-
-  private void initResourceCalculatorPlugin() {
-    Class<? extends ResourceCalculatorProcessTree> clazz =
-        this.jobConf.getClass(MRConfig.RESOURCE_CALCULATOR_PROCESS_TREE,
-            null, ResourceCalculatorProcessTree.class);
-    pTree = ResourceCalculatorProcessTree
-        .getResourceCalculatorProcessTree(System.getenv().get("JVM_PID"), clazz, this.jobConf);
-    LOG.info(" Using ResourceCalculatorProcessTree : " + pTree);
-    if (pTree != null) {
-      pTree.updateProcessTree();
-      initCpuCumulativeTime = pTree.getCumulativeCpuTime();
-    }
-  }
-
-  public TezProcessorContext getUmbilical() {
-    return this.processorContext;
-  }
-
-  public void initTask() throws IOException,
-                                InterruptedException {
-    this.mrReporter = new MRTaskReporter(processorContext);
-    this.useNewApi = jobConf.getUseNewMapper();
-    TezDAGID dagId = IDConverter.fromMRTaskAttemptId(taskAttemptId).getTaskID()
-        .getVertexID().getDAGId();
-
-    this.jobContext = new JobContextImpl(jobConf, dagId, mrReporter);
-    this.taskAttemptContext =
-        new TaskAttemptContextImpl(jobConf, taskAttemptId, mrReporter);
-
-    if (getState() == State.UNASSIGNED) {
-      setState(State.RUNNING);
-    }
-
-//    combineProcessor = null;
-//    boolean useCombiner = false;
-//    if (useNewApi) {
-//      try {
-//        useCombiner = (taskAttemptContext.getCombinerClass() != null);
-//      } catch (ClassNotFoundException e) {
-//        throw new IOException("Could not find combiner class", e);
-//      }
-//    } else {
-//      useCombiner = (job.getCombinerClass() != null);
-//    }
-//    if (useCombiner) {
-//      combineProcessor = new MRCombiner(this);
-//      combineProcessor.initialize(job, getTaskReporter());
-//    } else {
-//    }
-
-    localizeConfiguration(jobConf);
-  }
-
-//  public void initPartitioner(JobConf job) throws IOException,
-//      InterruptedException {
-//    partitioner = new MRPartitioner(this);
-//    ((MRPartitioner) partitioner).initialize(job, getTaskReporter());
-//  }
-
-  public MRTaskReporter getMRReporter() {
-    return mrReporter;
-  }
-
-  public void setState(State state) {
-    // TODO Auto-generated method stub
-
-  }
-
-  public State getState() {
-    // TODO Auto-generated method stub
-    return null;
-  }
-
-  public OutputCommitter getCommitter() {
-    return committer;
-  }
-
-  public void setCommitter(OutputCommitter committer) {
-    this.committer = committer;
-  }
-
-  public TezCounters getCounters() { return counters; }
-
-  public void setConf(JobConf jobConf) {
-    this.jobConf = jobConf;
-  }
-
-  public JobConf getConf() {
-    return this.jobConf;
-  }
-
-  /**
-   * Gets a handle to the Statistics instance based on the scheme associated
-   * with path.
-   *
-   * @param path the path.
-   * @param conf the configuration to extract the scheme from if not part of
-   *   the path.
-   * @return a Statistics instance, or null if none is found for the scheme.
-   */
-  @Private
-  public static List<Statistics> getFsStatistics(Path path, Configuration conf) throws IOException {
-    List<Statistics> matchedStats = new ArrayList<FileSystem.Statistics>();
-    path = path.getFileSystem(conf).makeQualified(path);
-    String scheme = path.toUri().getScheme();
-    for (Statistics stats : FileSystem.getAllStatistics()) {
-      if (stats.getScheme().equals(scheme)) {
-        matchedStats.add(stats);
-      }
-    }
-    return matchedStats;
-  }
-
-  @Private
-  public synchronized String getOutputName() {
-    return "part-" + NUMBER_FORMAT.format(taskAttemptId.getTaskID().getId());
-  }
-
-  public void waitBeforeCompletion(MRTaskReporter reporter) throws IOException,
-      InterruptedException {
-  }
-
-  public void outputReady(MRTaskReporter reporter, OutputContext outputContext)
-      throws IOException,
-      InterruptedException {
-    LOG.info("Task: " + taskAttemptId + " reporting outputReady");
-    updateCounters();
-    statusUpdate();
-  }
-
-  public void done(LogicalOutput output) throws IOException, InterruptedException {
-    updateCounters();
-
-    LOG.info("Task:" + taskAttemptId + " is done."
-        + " And is in the process of committing");
-    // TODO change this to use the new context
-    // TODO TEZ Interaciton between Commit and OutputReady. Merge ?
-    if (output instanceof SimpleOutput) {
-      SimpleOutput sOut = (SimpleOutput)output;
-      if (sOut.isCommitRequired()) {
-        //wait for commit approval and commit
-        // TODO EVENTUALLY - Commit is not required for map tasks.
-        // skip a couple of RPCs before exiting.
-        commit(sOut);
-      }
-    }
-    taskDone.set(true);
-    // Make sure we send at least one set of counter increments. It's
-    // ok to call updateCounters() in this thread after comm thread stopped.
-    updateCounters();
-    sendLastUpdate();
-    //signal the tasktracker that we are done
-    //sendDone(umbilical);
-  }
-
-  /**
-   * Send a status update to the task tracker
-   * @param umbilical
-   * @throws IOException
-   */
-  public void statusUpdate() throws IOException, InterruptedException {
-    // TODO call progress update here if not being called within Map/Reduce
-  }
-
-  /**
-   * Sends last status update before sending umbilical.done();
-   */
-  private void sendLastUpdate()
-      throws IOException, InterruptedException {
-    statusUpdate();
-  }
-
-  private void commit(SimpleOutput output) throws IOException {
-    int retries = 3;
-    while (true) {
-      // This will loop till the AM asks for the task to be killed. As
-      // against, the AM sending a signal to the task to kill itself
-      // gracefully.
-      try {
-        if (processorContext.canCommit()) {
-          break;
-        }
-        Thread.sleep(1000);
-      } catch(InterruptedException ie) {
-        //ignore
-      } catch (IOException ie) {
-        LOG.warn("Failure sending canCommit: "
-            + StringUtils.stringifyException(ie));
-        if (--retries == 0) {
-          throw ie;
-        }
-      }
-    }
-
-    // task can Commit now
-    try {
-      LOG.info("Task " + taskAttemptId + " is allowed to commit now");
-      output.commit();
-      return;
-    } catch (IOException iee) {
-      LOG.warn("Failure committing: " +
-          StringUtils.stringifyException(iee));
-      //if it couldn't commit a successfully then delete the output
-      discardOutput(output);
-      throw iee;
-    }
-  }
-
-  private
-  void discardOutput(SimpleOutput output) {
-    try {
-      output.abort();
-    } catch (IOException ioe)  {
-      LOG.warn("Failure cleaning up: " +
-               StringUtils.stringifyException(ioe));
-    }
-  }
-
-
-  public void updateCounters() {
-    // TODO Auto-generated method stub
-    // TODO TEZAM Implement.
-    Map<String, List<FileSystem.Statistics>> map = new
-        HashMap<String, List<FileSystem.Statistics>>();
-    for(Statistics stat: FileSystem.getAllStatistics()) {
-      String uriScheme = stat.getScheme();
-      if (map.containsKey(uriScheme)) {
-        List<FileSystem.Statistics> list = map.get(uriScheme);
-        list.add(stat);
-      } else {
-        List<FileSystem.Statistics> list = new ArrayList<FileSystem.Statistics>();
-        list.add(stat);
-        map.put(uriScheme, list);
-      }
-    }
-    for (Map.Entry<String, List<FileSystem.Statistics>> entry: map.entrySet()) {
-      FileSystemStatisticUpdater updater = statisticUpdaters.get(entry.getKey());
-      if(updater==null) {//new FileSystem has been found in the cache
-        updater =
-            new FileSystemStatisticUpdater(counters, entry.getValue(),
-                entry.getKey());
-        statisticUpdaters.put(entry.getKey(), updater);
-      }
-      updater.updateCounters();
-    }
-
-    gcUpdater.incrementGcCounter();
-    updateResourceCounters();
-  }
-
-  /**
-   * Updates the {@link TaskCounter#COMMITTED_HEAP_BYTES} counter to reflect the
-   * current total committed heap space usage of this JVM.
-   */
-  private void updateHeapUsageCounter() {
-    long currentHeapUsage = Runtime.getRuntime().totalMemory();
-    counters.findCounter(TaskCounter.COMMITTED_HEAP_BYTES)
-            .setValue(currentHeapUsage);
-  }
-
-  /**
-   * Update resource information counters
-   */
-  void updateResourceCounters() {
-    // Update generic resource counters
-    updateHeapUsageCounter();
-
-    // Updating resources specified in ResourceCalculatorPlugin
-    if (pTree == null) {
-      return;
-    }
-    pTree.updateProcessTree();
-    long cpuTime = pTree.getCumulativeCpuTime();
-    long pMem = pTree.getCumulativeRssmem();
-    long vMem = pTree.getCumulativeVmem();
-    // Remove the CPU time consumed previously by JVM reuse
-    cpuTime -= initCpuCumulativeTime;
-    counters.findCounter(TaskCounter.CPU_MILLISECONDS).setValue(cpuTime);
-    counters.findCounter(TaskCounter.PHYSICAL_MEMORY_BYTES).setValue(pMem);
-    counters.findCounter(TaskCounter.VIRTUAL_MEMORY_BYTES).setValue(vMem);
-  }
-
-
-  public static String normalizeStatus(String status, Configuration conf) {
-    // Check to see if the status string is too long
-    // and truncate it if needed.
-    int progressStatusLength = conf.getInt(
-        MRConfig.PROGRESS_STATUS_LEN_LIMIT_KEY,
-        MRConfig.PROGRESS_STATUS_LEN_LIMIT_DEFAULT);
-    if (status.length() > progressStatusLength) {
-      LOG.warn("Task status: \"" + status + "\" truncated to max limit ("
-          + progressStatusLength + " characters)");
-      status = status.substring(0, progressStatusLength);
-    }
-    return status;
-  }
-
-  protected static <INKEY,INVALUE,OUTKEY,OUTVALUE>
-  org.apache.hadoop.mapreduce.Reducer<INKEY,INVALUE,OUTKEY,OUTVALUE>.Context
-  createReduceContext(org.apache.hadoop.mapreduce.Reducer
-                        <INKEY,INVALUE,OUTKEY,OUTVALUE> reducer,
-                      Configuration job,
-                      TaskAttemptID taskId,
-                      final TezRawKeyValueIterator rIter,
-                      org.apache.hadoop.mapreduce.Counter inputKeyCounter,
-                      org.apache.hadoop.mapreduce.Counter inputValueCounter,
-                      org.apache.hadoop.mapreduce.RecordWriter<OUTKEY,OUTVALUE> output,
-                      org.apache.hadoop.mapreduce.OutputCommitter committer,
-                      org.apache.hadoop.mapreduce.StatusReporter reporter,
-                      RawComparator<INKEY> comparator,
-                      Class<INKEY> keyClass, Class<INVALUE> valueClass
-  ) throws IOException, InterruptedException {
-    RawKeyValueIterator r =
-        new RawKeyValueIterator() {
-
-          @Override
-          public boolean next() throws IOException {
-            return rIter.next();
-          }
-
-          @Override
-          public DataInputBuffer getValue() throws IOException {
-            return rIter.getValue();
-          }
-
-          @Override
-          public Progress getProgress() {
-            return rIter.getProgress();
-          }
-
-          @Override
-          public DataInputBuffer getKey() throws IOException {
-            return rIter.getKey();
-          }
-
-          @Override
-          public void close() throws IOException {
-            rIter.close();
-          }
-        };
-    org.apache.hadoop.mapreduce.ReduceContext<INKEY, INVALUE, OUTKEY, OUTVALUE>
-    reduceContext =
-      new ReduceContextImpl<INKEY, INVALUE, OUTKEY, OUTVALUE>(
-          job,
-          taskId,
-          r,
-          inputKeyCounter,
-          inputValueCounter,
-          output,
-          committer,
-          reporter,
-          comparator,
-          keyClass,
-          valueClass);
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Using key class: " + keyClass
-          + ", valueClass: " + valueClass);
-    }
-
-    org.apache.hadoop.mapreduce.Reducer<INKEY,INVALUE,OUTKEY,OUTVALUE>.Context
-        reducerContext =
-          new WrappedReducer<INKEY, INVALUE, OUTKEY, OUTVALUE>().getReducerContext(
-              reduceContext);
-
-    return reducerContext;
-  }
-
-  public void taskCleanup()
-      throws IOException, InterruptedException {
-    // set phase for this task
-    statusUpdate();
-    LOG.info("Runnning cleanup for the task");
-    // do the cleanup
-    committer.abortTask(taskAttemptContext);
-  }
-
-  public void localizeConfiguration(JobConf jobConf)
-      throws IOException, InterruptedException {
-    jobConf.set(JobContext.TASK_ID, taskAttemptId.getTaskID().toString());
-    jobConf.set(JobContext.TASK_ATTEMPT_ID, taskAttemptId.toString());
-    jobConf.setInt(JobContext.TASK_PARTITION,
-        taskAttemptId.getTaskID().getId());
-    jobConf.set(JobContext.ID, taskAttemptId.getJobID().toString());
-  }
-
-  public abstract TezCounter getOutputRecordsCounter();
-
-  public abstract TezCounter getInputRecordsCounter();
-
-  public org.apache.hadoop.mapreduce.TaskAttemptContext getTaskAttemptContext() {
-    return taskAttemptContext;
-  }
-
-  public JobContext getJobContext() {
-    return jobContext;
-  }
-
-  public TaskAttemptID getTaskAttemptId() {
-    return taskAttemptId;
-  }
-
-  public TezProcessorContext getTezEngineTaskContext() {
-    return processorContext;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/3d609458/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/newprocessor/MRTaskReporter.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/newprocessor/MRTaskReporter.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/newprocessor/MRTaskReporter.java
deleted file mode 100644
index c7c9567..0000000
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/newprocessor/MRTaskReporter.java
+++ /dev/null
@@ -1,122 +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.newprocessor;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.mapred.Counters;
-import org.apache.hadoop.mapred.InputSplit;
-import org.apache.hadoop.mapred.Reporter;
-import org.apache.tez.common.counters.TezCounter;
-import org.apache.tez.engine.newapi.TezInputContext;
-import org.apache.tez.engine.newapi.TezOutputContext;
-import org.apache.tez.engine.newapi.TezProcessorContext;
-import org.apache.tez.engine.newapi.TezTaskContext;
-import org.apache.tez.mapreduce.hadoop.mapred.MRCounters;
-import org.apache.tez.mapreduce.hadoop.newmapred.MRReporter;
-
-@InterfaceAudience.Private
-@InterfaceStability.Unstable
-public class MRTaskReporter
-    extends org.apache.hadoop.mapreduce.StatusReporter
-    implements Reporter {
-
-  private final TezTaskContext context;
-  private final boolean isProcessorContext;
-  private final Reporter reporter;
-
-  private InputSplit split = null;
-
-  public MRTaskReporter(TezProcessorContext context) {
-    this.context = context;
-    this.reporter = new MRReporter(context);
-    this.isProcessorContext = true;
-  }
-
-  public MRTaskReporter(TezOutputContext context) {
-    this.context = context;
-    this.reporter = new MRReporter(context);
-    this.isProcessorContext = false;
-  }
-  
-  public MRTaskReporter(TezInputContext context) {
-    this.context= context;
-    this.reporter = new MRReporter(context);
-    this.isProcessorContext = false;
-  }
-
-  public void setProgress(float progress) {
-    if (isProcessorContext) {
-      ((TezProcessorContext)context).setProgress(progress);
-    } else {
-      // TODO FIXME NEWTEZ - will simpleoutput's reporter use this api?
-    }
-  }
-
-  public void setStatus(String status) {
-    reporter.setStatus(status);
-  }
-
-  public float getProgress() {
-    return reporter.getProgress();
-  };
-
-  public void progress() {
-    reporter.progress();
-  }
-
-  public Counters.Counter getCounter(String group, String name) {
-    TezCounter counter = context.getCounters().findCounter(group, name);
-    MRCounters.MRCounter mrCounter = null;
-    if (counter != null) {
-      mrCounter = new MRCounters.MRCounter(counter);
-    }
-    return mrCounter;
-  }
-
-  public Counters.Counter getCounter(Enum<?> name) {
-    TezCounter counter = context.getCounters().findCounter(name);
-    MRCounters.MRCounter mrCounter = null;
-    if (counter != null) {
-      mrCounter = new MRCounters.MRCounter(counter);
-    }
-    return mrCounter;
-  }
-
-  public void incrCounter(Enum<?> key, long amount) {
-    reporter.incrCounter(key, amount);
-  }
-
-  public void incrCounter(String group, String counter, long amount) {
-    reporter.incrCounter(group, counter, amount);
-  }
-
-  public void setInputSplit(InputSplit split) {
-    this.split = split;
-  }
-
-  public InputSplit getInputSplit() throws UnsupportedOperationException {
-    if (split == null) {
-      throw new UnsupportedOperationException("Input only available on map");
-    } else {
-      return split;
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/3d609458/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/newprocessor/map/MapProcessor.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/newprocessor/map/MapProcessor.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/newprocessor/map/MapProcessor.java
deleted file mode 100644
index 21df743..0000000
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/newprocessor/map/MapProcessor.java
+++ /dev/null
@@ -1,341 +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.newprocessor.map;
-
-import java.io.IOException;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.JobContext;
-import org.apache.hadoop.mapred.MapRunnable;
-import org.apache.hadoop.mapred.OutputCollector;
-import org.apache.hadoop.mapred.RecordReader;
-import org.apache.hadoop.mapred.Reporter;
-import org.apache.hadoop.mapreduce.TaskAttemptContext;
-import org.apache.hadoop.mapreduce.lib.map.WrappedMapper;
-import org.apache.hadoop.util.ReflectionUtils;
-import org.apache.tez.common.counters.TaskCounter;
-import org.apache.tez.common.counters.TezCounter;
-import org.apache.tez.dag.api.TezException;
-import org.apache.tez.engine.lib.output.OnFileSortedOutput;
-import org.apache.tez.engine.newapi.Event;
-import org.apache.tez.engine.newapi.KVReader;
-import org.apache.tez.engine.newapi.KVWriter;
-import org.apache.tez.engine.newapi.LogicalIOProcessor;
-import org.apache.tez.engine.newapi.LogicalInput;
-import org.apache.tez.engine.newapi.LogicalOutput;
-import org.apache.tez.engine.newapi.TezProcessorContext;
-import org.apache.tez.mapreduce.hadoop.newmapreduce.MapContextImpl;
-import org.apache.tez.mapreduce.newinput.SimpleInput;
-import org.apache.tez.mapreduce.newinput.SimpleInputLegacy;
-import org.apache.tez.mapreduce.newoutput.SimpleOutput;
-import org.apache.tez.mapreduce.newprocessor.MRTask;
-import org.apache.tez.mapreduce.newprocessor.MRTaskReporter;
-
-@SuppressWarnings({ "unchecked", "rawtypes" })
-public class MapProcessor extends MRTask implements LogicalIOProcessor {
-
-  private static final Log LOG = LogFactory.getLog(MapProcessor.class);
-
-  public MapProcessor(){
-    super(true);
-  }
-
-  @Override
-  public void initialize(TezProcessorContext processorContext)
-      throws IOException {
-    try {
-      super.initialize(processorContext);
-    } catch (InterruptedException e) {
-      throw new IOException(e);
-    }
-  }
-
-
-  @Override
-  public void handleEvents(List<Event> processorEvents) {
-    // TODO Auto-generated method stub
-
-  }
-
-  public void close() throws IOException {
-    // TODO Auto-generated method stub
-
-  }
-
-  @Override
-  public void run(Map<String, LogicalInput> inputs,
-      Map<String, LogicalOutput> outputs) throws Exception {
-
-    LOG.info("Running map: " + processorContext.getUniqueIdentifier());
-
-    initTask();
-
-    if (inputs.size() != 1
-        || outputs.size() != 1) {
-      throw new IOException("Cannot handle multiple inputs or outputs"
-          + ", inputCount=" + inputs.size()
-          + ", outputCount=" + outputs.size());
-    }
-    LogicalInput in = inputs.values().iterator().next();
-    LogicalOutput out = outputs.values().iterator().next();
-
-    // Sanity check
-    if (!(in instanceof SimpleInputLegacy)) {
-      throw new IOException(new TezException(
-          "Only Simple Input supported. Input: " + in.getClass()));
-    }
-    SimpleInputLegacy input = (SimpleInputLegacy)in;
-
-    KVWriter kvWriter = null;
-    if (!(out instanceof OnFileSortedOutput)) {
-      kvWriter = ((SimpleOutput)out).getWriter();
-    } else {
-      kvWriter = ((OnFileSortedOutput)out).getWriter();
-    }
-
-    if (useNewApi) {
-      runNewMapper(jobConf, mrReporter, input, kvWriter);
-    } else {
-      runOldMapper(jobConf, mrReporter, input, kvWriter);
-    }
-
-    done(out);
-  }
-
-  void runOldMapper(
-      final JobConf job,
-      final MRTaskReporter reporter,
-      final SimpleInputLegacy input,
-      final KVWriter output
-      ) throws IOException, InterruptedException {
-
-    // Initialize input in-line since it sets parameters which may be used by the processor.
-    // Done only for SimpleInput.
-    // TODO use new method in SimpleInput to get required info
-    //input.initialize(job, master);
-
-    RecordReader in = new OldRecordReader(input);
-
-    OutputCollector collector = new OldOutputCollector(output);
-
-    MapRunnable runner =
-        (MapRunnable)ReflectionUtils.newInstance(job.getMapRunnerClass(), job);
-
-    runner.run(in, collector, (Reporter)reporter);
-    // start the sort phase only if there are reducers
-    this.statusUpdate();
-  }
-
-  private void runNewMapper(final JobConf job,
-      MRTaskReporter reporter,
-      final SimpleInputLegacy in,
-      KVWriter out
-      ) throws IOException, InterruptedException {
-
-    // Initialize input in-line since it sets parameters which may be used by the processor.
-    // Done only for SimpleInput.
-    // TODO use new method in SimpleInput to get required info
-    //in.initialize(job, master);
-
-    // make a task context so we can get the classes
-    org.apache.hadoop.mapreduce.TaskAttemptContext taskContext =
-        getTaskAttemptContext();
-
-    // make a mapper
-    org.apache.hadoop.mapreduce.Mapper mapper;
-    try {
-      mapper = (org.apache.hadoop.mapreduce.Mapper)
-          ReflectionUtils.newInstance(taskContext.getMapperClass(), job);
-    } catch (ClassNotFoundException cnfe) {
-      throw new IOException(cnfe);
-    }
-
-    org.apache.hadoop.mapreduce.RecordReader input =
-        new NewRecordReader(in);
-
-    org.apache.hadoop.mapreduce.RecordWriter output =
-        new NewOutputCollector(out);
-
-    org.apache.hadoop.mapreduce.InputSplit split = in.getNewInputSplit();
-
-    org.apache.hadoop.mapreduce.MapContext
-    mapContext =
-    new MapContextImpl(
-        job, taskAttemptId,
-        input, output,
-        getCommitter(),
-        processorContext, split);
-
-    org.apache.hadoop.mapreduce.Mapper.Context mapperContext =
-        new WrappedMapper().getMapContext(mapContext);
-
-    input.initialize(split, mapperContext);
-    mapper.run(mapperContext);
-    this.statusUpdate();
-    input.close();
-    output.close(mapperContext);
-  }
-
-  private static class NewRecordReader extends
-      org.apache.hadoop.mapreduce.RecordReader {
-    private final SimpleInput in;
-    private KVReader reader;
-
-    private NewRecordReader(SimpleInput in) throws IOException {
-      this.in = in;
-      this.reader = in.getReader();
-    }
-
-    @Override
-    public void initialize(org.apache.hadoop.mapreduce.InputSplit split,
-        TaskAttemptContext context) throws IOException,
-        InterruptedException {
-      //in.initializeNewRecordReader(split, context);
-    }
-
-    @Override
-    public boolean nextKeyValue() throws IOException,
-        InterruptedException {
-      return reader.next();
-    }
-
-    @Override
-    public Object getCurrentKey() throws IOException,
-        InterruptedException {
-      return reader.getCurrentKV().getKey();
-    }
-
-    @Override
-    public Object getCurrentValue() throws IOException,
-        InterruptedException {
-      return reader.getCurrentKV().getValues().iterator().next();
-    }
-
-    @Override
-    public float getProgress() throws IOException, InterruptedException {
-      return in.getProgress();
-    }
-
-    @Override
-    public void close() throws IOException {
-    }
-  }
-
-  private static class OldRecordReader implements RecordReader {
-    private final SimpleInputLegacy simpleInput;
-
-    private OldRecordReader(SimpleInputLegacy simpleInput) {
-      this.simpleInput = simpleInput;
-    }
-
-    @Override
-    public boolean next(Object key, Object value) throws IOException {
-      // TODO broken
-//      simpleInput.setKey(key);
-//      simpleInput.setValue(value);
-//      try {
-//        return simpleInput.hasNext();
-//      } catch (InterruptedException ie) {
-//        throw new IOException(ie);
-//      }
-      return simpleInput.getOldRecordReader().next(key, value);
-    }
-
-    @Override
-    public Object createKey() {
-      return simpleInput.getOldRecordReader().createKey();
-    }
-
-    @Override
-    public Object createValue() {
-      return simpleInput.getOldRecordReader().createValue();
-    }
-
-    @Override
-    public long getPos() throws IOException {
-      return simpleInput.getOldRecordReader().getPos();
-    }
-
-    @Override
-    public void close() throws IOException {
-    }
-
-    @Override
-    public float getProgress() throws IOException {
-      try {
-        return simpleInput.getProgress();
-      } catch (InterruptedException ie) {
-        throw new IOException(ie);
-      }
-    }
-  }
-
-  private static class OldOutputCollector
-  implements OutputCollector {
-    private final KVWriter output;
-
-    OldOutputCollector(KVWriter output) {
-      this.output = output;
-    }
-
-    public void collect(Object key, Object value) throws IOException {
-        output.write(key, value);
-    }
-  }
-
-  private class NewOutputCollector
-    extends org.apache.hadoop.mapreduce.RecordWriter {
-    private final KVWriter out;
-
-    NewOutputCollector(KVWriter out) throws IOException {
-      this.out = out;
-    }
-
-    @Override
-    public void write(Object key, Object value) throws IOException, InterruptedException {
-      out.write(key, value);
-    }
-
-    @Override
-    public void close(TaskAttemptContext context
-                      ) throws IOException, InterruptedException {
-    }
-  }
-
-  @Override
-  public void localizeConfiguration(JobConf jobConf)
-      throws IOException, InterruptedException {
-    super.localizeConfiguration(jobConf);
-    jobConf.setBoolean(JobContext.TASK_ISMAP, true);
-  }
-
-  @Override
-  public TezCounter getOutputRecordsCounter() {
-    return processorContext.getCounters().findCounter(TaskCounter.MAP_OUTPUT_RECORDS);
-  }
-
-  @Override
-  public TezCounter getInputRecordsCounter() {
-    return processorContext.getCounters().findCounter(TaskCounter.MAP_INPUT_RECORDS);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/3d609458/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/newprocessor/reduce/ReduceProcessor.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/newprocessor/reduce/ReduceProcessor.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/newprocessor/reduce/ReduceProcessor.java
deleted file mode 100644
index cedcdd6..0000000
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/newprocessor/reduce/ReduceProcessor.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.newprocessor.reduce;
-
-import java.io.IOException;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.io.DataInputBuffer;
-import org.apache.hadoop.io.RawComparator;
-import org.apache.hadoop.mapred.Counters.Counter;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.JobContext;
-import org.apache.hadoop.mapred.OutputCollector;
-import org.apache.hadoop.mapred.Reducer;
-import org.apache.hadoop.mapreduce.TaskAttemptContext;
-import org.apache.hadoop.util.Progress;
-import org.apache.hadoop.util.Progressable;
-import org.apache.hadoop.util.ReflectionUtils;
-import org.apache.tez.common.counters.TaskCounter;
-import org.apache.tez.common.counters.TezCounter;
-import org.apache.tez.engine.common.ConfigUtils;
-import org.apache.tez.engine.common.sort.impl.TezRawKeyValueIterator;
-import org.apache.tez.engine.lib.output.OnFileSortedOutput;
-import org.apache.tez.engine.newapi.Event;
-import org.apache.tez.engine.newapi.KVReader;
-import org.apache.tez.engine.newapi.KVWriter;
-import org.apache.tez.engine.newapi.LogicalIOProcessor;
-import org.apache.tez.engine.newapi.LogicalInput;
-import org.apache.tez.engine.newapi.LogicalOutput;
-import org.apache.tez.engine.newapi.TezProcessorContext;
-import org.apache.tez.mapreduce.newinput.ShuffledMergedInputLegacy;
-import org.apache.tez.mapreduce.newoutput.SimpleOutput;
-import org.apache.tez.mapreduce.newprocessor.MRTask;
-import org.apache.tez.mapreduce.newprocessor.MRTaskReporter;
-
-
-@SuppressWarnings({ "unchecked", "rawtypes" })
-public class ReduceProcessor
-extends MRTask
-implements LogicalIOProcessor {
-
-  private static final Log LOG = LogFactory.getLog(ReduceProcessor.class);
-
-  private Counter reduceInputKeyCounter;
-  private Counter reduceInputValueCounter;
-
-  public ReduceProcessor() {
-    super(false);
-  }
-
-  @Override
-  public void initialize(TezProcessorContext processorContext)
-      throws IOException {
-    try {
-      super.initialize(processorContext);
-    } catch (InterruptedException e) {
-      throw new IOException(e);
-    }
-  }
-
-
-  @Override
-  public void handleEvents(List<Event> processorEvents) {
-    // TODO Auto-generated method stub
-
-  }
-
-  public void close() throws IOException {
-    // TODO Auto-generated method stub
-
-  }
-
-  @Override
-  public void run(Map<String, LogicalInput> inputs,
-      Map<String, LogicalOutput> outputs) throws Exception {
-
-    LOG.info("Running reduce: " + processorContext.getUniqueIdentifier());
-
-    initTask();
-
-    if (outputs.size() <= 0 || outputs.size() > 1) {
-      throw new IOException("Invalid number of outputs"
-          + ", outputCount=" + outputs.size());
-    }
-
-    if (inputs.size() <= 0 || inputs.size() > 1) {
-      throw new IOException("Invalid number of inputs"
-          + ", inputCount=" + inputs.size());
-    }
-
-    LogicalInput in = inputs.values().iterator().next();
-    LogicalOutput out = outputs.values().iterator().next();
-
-    this.statusUpdate();
-
-    Class keyClass = ConfigUtils.getIntermediateInputKeyClass(jobConf);
-    Class valueClass = ConfigUtils.getIntermediateInputValueClass(jobConf);
-    LOG.info("Using keyClass: " + keyClass);
-    LOG.info("Using valueClass: " + valueClass);
-    RawComparator comparator =
-        ConfigUtils.getInputKeySecondaryGroupingComparator(jobConf);
-    LOG.info("Using comparator: " + comparator);
-
-    reduceInputKeyCounter =
-        mrReporter.getCounter(TaskCounter.REDUCE_INPUT_GROUPS);
-    reduceInputValueCounter =
-        mrReporter.getCounter(TaskCounter.REDUCE_INPUT_RECORDS);
-
-    // Sanity check
-    if (!(in instanceof ShuffledMergedInputLegacy)) {
-      throw new IOException("Illegal input to reduce: " + in.getClass());
-    }
-    ShuffledMergedInputLegacy shuffleInput = (ShuffledMergedInputLegacy)in;
-    KVReader kvReader = shuffleInput.getReader();
-
-    KVWriter kvWriter = null;
-    if((out instanceof SimpleOutput)) {
-      kvWriter = ((SimpleOutput) out).getWriter();
-    } else if ((out instanceof OnFileSortedOutput)) {
-      kvWriter = ((OnFileSortedOutput) out).getWriter();
-    } else {
-      throw new IOException("Illegal input to reduce: " + in.getClass());
-    }
-
-    if (useNewApi) {
-      try {
-        runNewReducer(
-            jobConf,
-            mrReporter,
-            shuffleInput, comparator,  keyClass, valueClass,
-            kvWriter);
-      } catch (ClassNotFoundException cnfe) {
-        throw new IOException(cnfe);
-      }
-    } else {
-      runOldReducer(
-          jobConf, mrReporter,
-          kvReader, comparator, keyClass, valueClass, kvWriter);
-    }
-
-    done(out);
-  }
-
-  void runOldReducer(JobConf job,
-      final MRTaskReporter reporter,
-      KVReader input,
-      RawComparator comparator,
-      Class keyClass,
-      Class valueClass,
-      final KVWriter output) throws IOException, InterruptedException {
-
-    Reducer reducer =
-        ReflectionUtils.newInstance(job.getReducerClass(), job);
-
-    // make output collector
-
-    OutputCollector collector =
-        new OutputCollector() {
-      public void collect(Object key, Object value)
-          throws IOException {
-        output.write(key, value);
-      }
-    };
-
-    // apply reduce function
-    try {
-      ReduceValuesIterator values =
-          new ReduceValuesIterator(
-              input, reporter, reduceInputValueCounter);
-
-      values.informReduceProgress();
-      while (values.more()) {
-        reduceInputKeyCounter.increment(1);
-        reducer.reduce(values.getKey(), values, collector, reporter);
-        values.informReduceProgress();
-      }
-
-      //Clean up: repeated in catch block below
-      reducer.close();
-      //End of clean up.
-    } catch (IOException ioe) {
-      try {
-        reducer.close();
-      } catch (IOException ignored) {
-      }
-
-      throw ioe;
-    }
-  }
-
-  private static class ReduceValuesIterator<KEY,VALUE>
-  implements Iterator<VALUE> {
-    private Counter reduceInputValueCounter;
-    private KVReader in;
-    private Progressable reporter;
-    private Object currentKey;
-    private Iterator<Object> currentValues;
-
-    public ReduceValuesIterator (KVReader in,
-        Progressable reporter,
-        Counter reduceInputValueCounter)
-            throws IOException {
-      this.reduceInputValueCounter = reduceInputValueCounter;
-      this.in = in;
-      this.reporter = reporter;
-    }
-
-    public boolean more() throws IOException {
-      boolean more = in.next();
-      if(more) {
-        currentKey = in.getCurrentKV().getKey();
-        currentValues = in.getCurrentKV().getValues().iterator();
-      } else {
-        currentKey = null;
-        currentValues = null;
-      }
-      return more;
-    }
-
-    public KEY getKey() throws IOException {
-      return (KEY) currentKey;
-    }
-
-    public void informReduceProgress() {
-      reporter.progress();
-    }
-
-    @Override
-    public boolean hasNext() {
-      return currentValues.hasNext();
-    }
-
-    @Override
-    public VALUE next() {
-      reduceInputValueCounter.increment(1);
-      return (VALUE) currentValues.next();
-    }
-
-    @Override
-    public void remove() {
-      throw new UnsupportedOperationException();
-    }
-
-  }
-
-  void runNewReducer(JobConf job,
-      final MRTaskReporter reporter,
-      ShuffledMergedInputLegacy input,
-      RawComparator comparator,
-      Class keyClass,
-      Class valueClass,
-      final KVWriter out
-      ) throws IOException,InterruptedException,
-      ClassNotFoundException {
-
-    // make a task context so we can get the classes
-    org.apache.hadoop.mapreduce.TaskAttemptContext taskContext = getTaskAttemptContext();
-
-    // make a reducer
-    org.apache.hadoop.mapreduce.Reducer reducer =
-        (org.apache.hadoop.mapreduce.Reducer)
-        ReflectionUtils.newInstance(taskContext.getReducerClass(), job);
-
-    // wrap value iterator to report progress.
-    final TezRawKeyValueIterator rawIter = input.getIterator();
-    TezRawKeyValueIterator rIter = new TezRawKeyValueIterator() {
-      public void close() throws IOException {
-        rawIter.close();
-      }
-      public DataInputBuffer getKey() throws IOException {
-        return rawIter.getKey();
-      }
-      public Progress getProgress() {
-        return rawIter.getProgress();
-      }
-      public DataInputBuffer getValue() throws IOException {
-        return rawIter.getValue();
-      }
-      public boolean next() throws IOException {
-        boolean ret = rawIter.next();
-        reporter.setProgress(rawIter.getProgress().getProgress());
-        return ret;
-      }
-    };
-
-    org.apache.hadoop.mapreduce.RecordWriter trackedRW =
-        new org.apache.hadoop.mapreduce.RecordWriter() {
-
-      @Override
-      public void write(Object key, Object value) throws IOException,
-      InterruptedException {
-        out.write(key, value);
-      }
-
-      @Override
-      public void close(TaskAttemptContext context) throws IOException,
-      InterruptedException {
-      }
-    };
-
-    org.apache.hadoop.mapreduce.Reducer.Context reducerContext =
-        createReduceContext(
-            reducer, job, taskAttemptId,
-            rIter, reduceInputKeyCounter,
-            reduceInputValueCounter,
-            trackedRW,
-            committer,
-            reporter, comparator, keyClass,
-            valueClass);
-
-
-
-    reducer.run(reducerContext);
-    trackedRW.close(reducerContext);
-  }
-
-  @Override
-  public void localizeConfiguration(JobConf jobConf)
-      throws IOException, InterruptedException {
-    super.localizeConfiguration(jobConf);
-    jobConf.setBoolean(JobContext.TASK_ISMAP, false);
-  }
-
-  @Override
-  public TezCounter getOutputRecordsCounter() {
-    return processorContext.getCounters().findCounter(TaskCounter.REDUCE_OUTPUT_RECORDS);
-  }
-
-  @Override
-  public TezCounter getInputRecordsCounter() {
-    return processorContext.getCounters().findCounter(TaskCounter.REDUCE_INPUT_GROUPS);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/3d609458/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/output/SimpleOutput.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/output/SimpleOutput.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/output/SimpleOutput.java
index 060e28c..91fb8cc 100644
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/output/SimpleOutput.java
+++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/output/SimpleOutput.java
@@ -1,181 +1,226 @@
-/**
- * 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.output;
 
 import java.io.IOException;
+import java.text.NumberFormat;
 import java.util.List;
+import java.util.concurrent.atomic.AtomicBoolean;
 
+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.Path;
 import org.apache.hadoop.fs.FileSystem.Statistics;
+import org.apache.hadoop.mapred.FileOutputCommitter;
+import org.apache.hadoop.mapred.FileOutputFormat;
 import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.JobContext;
+import org.apache.hadoop.mapred.TaskAttemptID;
+import org.apache.hadoop.mapred.TaskID;
+import org.apache.hadoop.mapreduce.OutputCommitter;
+import org.apache.hadoop.mapreduce.OutputFormat;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.TaskType;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputFormatCounter;
 import org.apache.hadoop.util.ReflectionUtils;
-import org.apache.tez.common.TezEngineTaskContext;
+import org.apache.tez.common.TezUtils;
+import org.apache.tez.common.counters.TaskCounter;
 import org.apache.tez.common.counters.TezCounter;
-import org.apache.tez.engine.api.Master;
-import org.apache.tez.engine.api.Output;
-import org.apache.tez.engine.records.OutputContext;
-import org.apache.tez.mapreduce.processor.MRTask;
+import org.apache.tez.engine.newapi.Event;
+import org.apache.tez.engine.newapi.KVWriter;
+import org.apache.tez.engine.newapi.LogicalOutput;
+import org.apache.tez.engine.newapi.TezOutputContext;
+import org.apache.tez.mapreduce.common.Utils;
+import org.apache.tez.mapreduce.hadoop.MRConfig;
+import org.apache.tez.mapreduce.hadoop.MRJobConfig;
+import org.apache.tez.mapreduce.hadoop.mapred.MRReporter;
+import org.apache.tez.mapreduce.hadoop.mapreduce.TaskAttemptContextImpl;
 import org.apache.tez.mapreduce.processor.MRTaskReporter;
 
-/**
- * {@link SimpleOutput} is an {@link Output} which persists key/value pairs
- * written to it. 
- * 
- * It is compatible with all standard Apache Hadoop MapReduce 
- * {@link OutputFormat} implementations. 
- */
-@SuppressWarnings({ "unchecked", "rawtypes" })
-public class SimpleOutput implements Output {
-
-  private MRTask task;
-  
+public class SimpleOutput implements LogicalOutput {
+
+  private static final Log LOG = LogFactory.getLog(SimpleOutput.class);
+
+  private static final NumberFormat NUMBER_FORMAT = NumberFormat.getInstance();
+  static {
+    NUMBER_FORMAT.setMinimumIntegerDigits(5);
+    NUMBER_FORMAT.setGroupingUsed(false);
+  }
+
+  private TezOutputContext outputContext;
+  private JobConf jobConf;
   boolean useNewApi;
-  JobConf jobConf;
-  
-  org.apache.hadoop.mapreduce.TaskAttemptContext taskAttemptContext;
+  private AtomicBoolean closed = new AtomicBoolean(false);
+
+  @SuppressWarnings("rawtypes")
   org.apache.hadoop.mapreduce.OutputFormat newOutputFormat;
+  @SuppressWarnings("rawtypes")
   org.apache.hadoop.mapreduce.RecordWriter newRecordWriter;
-  
+
+  @SuppressWarnings("rawtypes")
   org.apache.hadoop.mapred.OutputFormat oldOutputFormat;
+  @SuppressWarnings("rawtypes")
   org.apache.hadoop.mapred.RecordWriter oldRecordWriter;
-  
+
   private TezCounter outputRecordCounter;
-  private TezCounter fileOutputByteCounter; 
+  private TezCounter fileOutputByteCounter;
   private List<Statistics> fsStats;
-  private MRTaskReporter reporter;
-  
-  public SimpleOutput(TezEngineTaskContext task)
-  {}
-  
-  public void setTask(MRTask task) {
-    this.task = task;
-  }
-  
-  public void initialize(Configuration conf, Master master) throws IOException,
-      InterruptedException {
 
-    if (task == null) {
-      return;
-    }
-    
-    if (conf instanceof JobConf) {
-      jobConf = (JobConf)conf;
-    } else {
-      jobConf = new JobConf(conf);
-    }
-    
-    useNewApi = jobConf.getUseNewMapper();
-    taskAttemptContext = task.getTaskAttemptContext();
-    
-    outputRecordCounter = task.getOutputRecordsCounter();
-    fileOutputByteCounter = task.getFileOutputBytesCounter();
-
-    reporter = task.getMRReporter();
-    
+  private TaskAttemptContext newApiTaskAttemptContext;
+  private org.apache.hadoop.mapred.TaskAttemptContext oldApiTaskAttemptContext;
+
+  private boolean isMapperOutput;
+
+  private OutputCommitter committer;
+
+  @Override
+  public List<Event> initialize(TezOutputContext outputContext)
+      throws IOException, InterruptedException {
+    LOG.info("Initializing Simple Output");
+    this.outputContext = outputContext;
+    Configuration conf = TezUtils.createConfFromUserPayload(
+        outputContext.getUserPayload());
+    this.jobConf = new JobConf(conf);
+    this.useNewApi = this.jobConf.getUseNewMapper();
+    this.isMapperOutput = jobConf.getBoolean(MRConfig.IS_MAP_PROCESSOR,
+        false);
+    jobConf.setInt(MRJobConfig.APPLICATION_ATTEMPT_ID,
+        outputContext.getDAGAttemptNumber());
+
+    outputRecordCounter = outputContext.getCounters().findCounter(
+        TaskCounter.MAP_OUTPUT_RECORDS);
+    fileOutputByteCounter = outputContext.getCounters().findCounter(
+        FileOutputFormatCounter.BYTES_WRITTEN);
+
     if (useNewApi) {
+      newApiTaskAttemptContext = createTaskAttemptContext();
       try {
         newOutputFormat =
             ReflectionUtils.newInstance(
-                taskAttemptContext.getOutputFormatClass(), jobConf);
+                newApiTaskAttemptContext.getOutputFormatClass(), jobConf);
       } catch (ClassNotFoundException cnfe) {
         throw new IOException(cnfe);
       }
-      
+
       List<Statistics> matchedStats = null;
-      if (newOutputFormat instanceof 
+      if (newOutputFormat instanceof
           org.apache.hadoop.mapreduce.lib.output.FileOutputFormat) {
-        matchedStats = 
-            MRTask.getFsStatistics(
+        matchedStats =
+            Utils.getFsStatistics(
                 org.apache.hadoop.mapreduce.lib.output.FileOutputFormat
-                    .getOutputPath(taskAttemptContext), 
+                    .getOutputPath(newApiTaskAttemptContext),
                 jobConf);
       }
       fsStats = matchedStats;
 
       long bytesOutPrev = getOutputBytes();
-      newRecordWriter = 
-          newOutputFormat.getRecordWriter(this.taskAttemptContext);
+      try {
+        newRecordWriter =
+            newOutputFormat.getRecordWriter(newApiTaskAttemptContext);
+      } catch (InterruptedException e) {
+        throw new IOException("Interrupted while creating record writer", e);
+      }
       long bytesOutCurr = getOutputBytes();
       fileOutputByteCounter.increment(bytesOutCurr - bytesOutPrev);
     } else {
+      TaskAttemptID taskAttemptId = new TaskAttemptID(new TaskID(Long.toString(
+          outputContext.getApplicationId().getClusterTimestamp()),
+          outputContext.getApplicationId().getId(),
+          (isMapperOutput ? TaskType.MAP : TaskType.REDUCE),
+          outputContext.getTaskIndex()),
+          outputContext.getTaskAttemptNumber());
+      jobConf.set(JobContext.TASK_ATTEMPT_ID, taskAttemptId.toString());
+      jobConf.set(JobContext.TASK_ID, taskAttemptId.getTaskID().toString());
+      jobConf.setBoolean(JobContext.TASK_ISMAP, isMapperOutput);
+      jobConf.setInt(JobContext.TASK_PARTITION,
+          taskAttemptId.getTaskID().getId());
+      jobConf.set(JobContext.ID, taskAttemptId.getJobID().toString());
+
+      oldApiTaskAttemptContext =
+          new org.apache.tez.mapreduce.hadoop.mapred.TaskAttemptContextImpl(
+              jobConf, taskAttemptId,
+              new MRTaskReporter(outputContext));
       oldOutputFormat = jobConf.getOutputFormat();
-      
+
       List<Statistics> matchedStats = null;
-      if (oldOutputFormat instanceof org.apache.hadoop.mapred.FileOutputFormat) {
-        matchedStats = 
-            MRTask.getFsStatistics(
+      if (oldOutputFormat
+          instanceof org.apache.hadoop.mapred.FileOutputFormat) {
+        matchedStats =
+            Utils.getFsStatistics(
                 org.apache.hadoop.mapred.FileOutputFormat.getOutputPath(
-                    jobConf), 
+                    jobConf),
                 jobConf);
       }
       fsStats = matchedStats;
 
       FileSystem fs = FileSystem.get(jobConf);
-      String finalName = task.getOutputName();
+      String finalName = getOutputName();
 
       long bytesOutPrev = getOutputBytes();
-      oldRecordWriter = 
+      oldRecordWriter =
           oldOutputFormat.getRecordWriter(
-              fs, jobConf, finalName, reporter);
+              fs, jobConf, finalName, new MRReporter(outputContext));
       long bytesOutCurr = getOutputBytes();
       fileOutputByteCounter.increment(bytesOutCurr - bytesOutPrev);
     }
+    initCommitter(jobConf, useNewApi);
+
+    LOG.info("Initialized Simple Output"
+        + ", using_new_api: " + useNewApi);
+    return null;
   }
-  
-  public void write(Object key, Object value) 
+
+  public void initCommitter(JobConf job, boolean useNewApi)
       throws IOException, InterruptedException {
 
-    reporter.progress();
-    long bytesOutPrev = getOutputBytes();
-  
     if (useNewApi) {
-      newRecordWriter.write(key, value);
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("using new api for output committer");
+      }
+
+      OutputFormat<?, ?> outputFormat = null;
+      try {
+        outputFormat = ReflectionUtils.newInstance(
+            newApiTaskAttemptContext.getOutputFormatClass(), job);
+      } catch (ClassNotFoundException cnfe) {
+        throw new IOException("Unknown OutputFormat", cnfe);
+      }
+      this.committer = outputFormat.getOutputCommitter(
+          newApiTaskAttemptContext);
     } else {
-      oldRecordWriter.write(key, value);
+      this.committer = job.getOutputCommitter();
     }
-    
-    long bytesOutCurr = getOutputBytes();
-    fileOutputByteCounter.increment(bytesOutCurr - bytesOutPrev);
-    outputRecordCounter.increment(1);
 
+    Path outputPath = FileOutputFormat.getOutputPath(job);
+    if (outputPath != null) {
+      if ((this.committer instanceof FileOutputCommitter)) {
+        FileOutputFormat.setWorkOutputPath(job,
+            ((FileOutputCommitter) this.committer).getTaskAttemptPath(
+                oldApiTaskAttemptContext));
+      } else {
+        FileOutputFormat.setWorkOutputPath(job, outputPath);
+      }
+    }
+    if (useNewApi) {
+      this.committer.setupTask(newApiTaskAttemptContext);
+    } else {
+      this.committer.setupTask(oldApiTaskAttemptContext);
+    }
   }
 
-  public void close() throws IOException, InterruptedException {
-    reporter.progress();
-    long bytesOutPrev = getOutputBytes();
+  public boolean isCommitRequired() throws IOException {
     if (useNewApi) {
-      newRecordWriter.close(taskAttemptContext);
+      return committer.needsTaskCommit(newApiTaskAttemptContext);
     } else {
-      oldRecordWriter.close(null);
+      return committer.needsTaskCommit(oldApiTaskAttemptContext);
     }
-    long bytesOutCurr = getOutputBytes();
-    fileOutputByteCounter.increment(bytesOutCurr - bytesOutPrev);
   }
 
-  public org.apache.hadoop.mapreduce.OutputFormat getNewOutputFormat() {
-    return newOutputFormat;
-  }
-  
-  public org.apache.hadoop.mapred.OutputFormat getOldOutputFormat() {
-    return oldOutputFormat;
+  private TaskAttemptContext createTaskAttemptContext() {
+    return new TaskAttemptContextImpl(this.jobConf, outputContext,
+        isMapperOutput);
   }
-  
+
   private long getOutputBytes() {
     if (fsStats == null) return 0;
     long bytesWritten = 0;
@@ -185,9 +230,97 @@ public class SimpleOutput implements Output {
     return bytesWritten;
   }
 
+  private String getOutputName() {
+    return "part-" + NUMBER_FORMAT.format(outputContext.getTaskIndex());
+  }
+
+  @Override
+  public KVWriter getWriter() throws IOException {
+    return new KVWriter() {
+      private final boolean useNewWriter = useNewApi;
+
+      @SuppressWarnings("unchecked")
+      @Override
+      public void write(Object key, Object value) throws IOException {
+        long bytesOutPrev = getOutputBytes();
+        if (useNewWriter) {
+          try {
+            newRecordWriter.write(key, value);
+          } catch (InterruptedException e) {
+            Thread.currentThread().interrupt();
+            throw new IOException("Interrupted while writing next key-value",e);
+          }
+        } else {
+          oldRecordWriter.write(key, value);
+        }
+
+        long bytesOutCurr = getOutputBytes();
+        fileOutputByteCounter.increment(bytesOutCurr - bytesOutPrev);
+        outputRecordCounter.increment(1);
+      }
+    };
+  }
+
+  @Override
+  public void handleEvents(List<Event> outputEvents) {
+    // Not expecting any events at the moment.
+  }
+
   @Override
-  public OutputContext getOutputContext() {
+  public synchronized List<Event> close() throws IOException {
+    if (closed.getAndSet(true)) {
+      return null;
+    }
+
+    LOG.info("Closing Simple Output");
+    long bytesOutPrev = getOutputBytes();
+    if (useNewApi) {
+      try {
+        newRecordWriter.close(newApiTaskAttemptContext);
+      } catch (InterruptedException e) {
+        throw new IOException("Interrupted while closing record writer", e);
+      }
+    } else {
+      oldRecordWriter.close(null);
+    }
+    long bytesOutCurr = getOutputBytes();
+    fileOutputByteCounter.increment(bytesOutCurr - bytesOutPrev);
+    LOG.info("Closed Simple Output");
     return null;
   }
 
+  @Override
+  public void setNumPhysicalOutputs(int numOutputs) {
+    // Nothing to do for now
+  }
+
+  /**
+   * SimpleOutput expects that a Processor call commit prior to the
+   * Processor's completion
+   * @throws IOException
+   */
+  public void commit() throws IOException {
+    close();
+    if (useNewApi) {
+      committer.commitTask(newApiTaskAttemptContext);
+    } else {
+      committer.commitTask(oldApiTaskAttemptContext);
+    }
+  }
+
+
+  /**
+   * SimpleOutput expects that a Processor call abort in case of any error
+   * ( including an error during commit ) prior to the Processor's completion
+   * @throws IOException
+   */
+  public void abort() throws IOException {
+    close();
+    if (useNewApi) {
+      committer.abortTask(newApiTaskAttemptContext);
+    } else {
+      committer.abortTask(oldApiTaskAttemptContext);
+    }
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/3d609458/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/partition/MRPartitioner.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/partition/MRPartitioner.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/partition/MRPartitioner.java
index 61dfcd1..d061ad5 100644
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/partition/MRPartitioner.java
+++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/partition/MRPartitioner.java
@@ -1,70 +1,54 @@
 /**
-* 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.
-*/
+ * 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.partition;
 
-import java.io.IOException;
-
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.Partitioner;
 import org.apache.hadoop.util.ReflectionUtils;
-import org.apache.tez.engine.api.Master;
-import org.apache.tez.mapreduce.processor.MRTask;
+import org.apache.tez.common.TezJobConfig;
+import org.apache.tez.engine.common.ConfigUtils;
+import org.apache.tez.mapreduce.hadoop.MRJobConfig;
 
-@SuppressWarnings({"rawtypes", "unchecked"})
+@SuppressWarnings({ "rawtypes", "unchecked" })
 public class MRPartitioner implements org.apache.tez.engine.api.Partitioner {
 
   static final Log LOG = LogFactory.getLog(MRPartitioner.class);
-  private final MRTask task;
-  
-  JobConf jobConf;
-  boolean useNewApi;
-  
-  org.apache.hadoop.mapred.Partitioner oldPartitioner;
-  org.apache.hadoop.mapreduce.Partitioner newPartitioner;
 
-  public MRPartitioner(MRTask task) {
-    this.task = task;
-  }
-  
-  public void initialize(Configuration conf, Master master) 
-      throws IOException, InterruptedException {
-    if (conf instanceof JobConf) {
-      jobConf = (JobConf)conf;
-    } else {
-      jobConf = new JobConf(conf);
-    }
-    
-    useNewApi = jobConf.getUseNewMapper();
-    final int partitions = this.task.getTezEngineTaskContext()
-        .getOutputSpecList().get(0).getNumOutputs();
+  private final boolean useNewApi;
+  private int partitions = 1;
+
+  private org.apache.hadoop.mapreduce.Partitioner newPartitioner;
+  private org.apache.hadoop.mapred.Partitioner oldPartitioner;
+
+  public MRPartitioner(Configuration conf) {
+    this.useNewApi = ConfigUtils.useNewApi(conf);
+    this.partitions = conf.getInt(TezJobConfig.TEZ_ENGINE_NUM_EXPECTED_PARTITIONS, 1);
+
     if (useNewApi) {
       if (partitions > 1) {
-        try {
-          newPartitioner = (org.apache.hadoop.mapreduce.Partitioner)
-            ReflectionUtils.newInstance(
-                task.getJobContext().getPartitionerClass(), jobConf);
-        } catch (ClassNotFoundException cnfe) {
-          throw new IOException(cnfe);
-        }
+        newPartitioner = (org.apache.hadoop.mapreduce.Partitioner) ReflectionUtils
+            .newInstance(
+                (Class<? extends org.apache.hadoop.mapreduce.Partitioner<?, ?>>) conf
+                    .getClass(MRJobConfig.PARTITIONER_CLASS_ATTR,
+                        org.apache.hadoop.mapreduce.lib.partition.HashPartitioner.class), conf);
       } else {
         newPartitioner = new org.apache.hadoop.mapreduce.Partitioner() {
           @Override
@@ -75,24 +59,24 @@ public class MRPartitioner implements org.apache.tez.engine.api.Partitioner {
       }
     } else {
       if (partitions > 1) {
-        oldPartitioner = (Partitioner)
-          ReflectionUtils.newInstance(jobConf.getPartitionerClass(), jobConf);
+        oldPartitioner = (org.apache.hadoop.mapred.Partitioner) ReflectionUtils.newInstance(
+            (Class<? extends org.apache.hadoop.mapred.Partitioner>) conf.getClass(
+                "mapred.partitioner.class", org.apache.hadoop.mapred.lib.HashPartitioner.class), conf);
       } else {
-        oldPartitioner = new Partitioner() {
+        oldPartitioner = new org.apache.hadoop.mapred.Partitioner() {
           @Override
-          public void configure(JobConf job) {}
-          
+          public void configure(JobConf job) {
+          }
+
           @Override
           public int getPartition(Object key, Object value, int numPartitions) {
             return numPartitions - 1;
           }
         };
       }
-
     }
-
   }
-  
+
   @Override
   public int getPartition(Object key, Object value, int numPartitions) {
     if (useNewApi) {
@@ -101,5 +85,4 @@ public class MRPartitioner implements org.apache.tez.engine.api.Partitioner {
       return oldPartitioner.getPartition(key, value, numPartitions);
     }
   }
-
-}
+}
\ No newline at end of file


[13/50] [abbrv] Rename *.new* packages back to what they should be, remove dead code from the old packages - mapreduce module - tez-engine module (part of TEZ-398). (sseth)

Posted by ss...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/lib/oldoutput/OldInMemorySortedOutput.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/lib/oldoutput/OldInMemorySortedOutput.java b/tez-engine/src/main/java/org/apache/tez/engine/lib/oldoutput/OldInMemorySortedOutput.java
deleted file mode 100644
index 9ac92ba..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/lib/oldoutput/OldInMemorySortedOutput.java
+++ /dev/null
@@ -1,58 +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.engine.lib.oldoutput;
-
-import java.io.IOException;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.tez.common.RunningTaskContext;
-import org.apache.tez.common.TezEngineTaskContext;
-import org.apache.tez.engine.api.Master;
-import org.apache.tez.engine.api.Output;
-import org.apache.tez.engine.common.sort.SortingOutput;
-import org.apache.tez.engine.records.OutputContext;
-
-/**
- * {@link OldInMemorySortedOutput} is an {@link Output} which sorts key/value pairs 
- * written to it and persists it to a file.
- */
-public class OldInMemorySortedOutput implements SortingOutput {
-  
-  public OldInMemorySortedOutput(TezEngineTaskContext task) throws IOException {
-  }
-  
-  public void initialize(Configuration conf, Master master) 
-      throws IOException, InterruptedException {
-  }
-
-  public void setTask(RunningTaskContext task) {
-  }
-  
-  public void write(Object key, Object value) throws IOException,
-      InterruptedException {
-  }
-
-  public void close() throws IOException, InterruptedException {
-  }
-
-  @Override
-  public OutputContext getOutputContext() {
-    return null;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/lib/oldoutput/OldLocalOnFileSorterOutput.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/lib/oldoutput/OldLocalOnFileSorterOutput.java b/tez-engine/src/main/java/org/apache/tez/engine/lib/oldoutput/OldLocalOnFileSorterOutput.java
deleted file mode 100644
index b7f913c..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/lib/oldoutput/OldLocalOnFileSorterOutput.java
+++ /dev/null
@@ -1,38 +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.engine.lib.oldoutput;
-
-import java.io.IOException;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.tez.common.TezEngineTaskContext;
-
-public class OldLocalOnFileSorterOutput extends OldOnFileSortedOutput {
-
-  private static final Log LOG = LogFactory.getLog(OldLocalOnFileSorterOutput.class);
-
-  public OldLocalOnFileSorterOutput(TezEngineTaskContext task) throws IOException {
-    super(task);
-  }
-
-  @Override
-  public void close() throws IOException, InterruptedException {
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/lib/oldoutput/OldOnFileSortedOutput.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/lib/oldoutput/OldOnFileSortedOutput.java b/tez-engine/src/main/java/org/apache/tez/engine/lib/oldoutput/OldOnFileSortedOutput.java
deleted file mode 100644
index f259df9..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/lib/oldoutput/OldOnFileSortedOutput.java
+++ /dev/null
@@ -1,62 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.tez.engine.lib.oldoutput;
-
-import java.io.IOException;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.tez.common.RunningTaskContext;
-import org.apache.tez.common.TezEngineTaskContext;
-import org.apache.tez.engine.api.Master;
-import org.apache.tez.engine.api.Output;
-import org.apache.tez.engine.common.sort.SortingOutput;
-import org.apache.tez.engine.records.OutputContext;
-
-/**
- * {@link OldOnFileSortedOutput} is an {@link Output} which sorts key/value pairs 
- * written to it and persists it to a file.
- */
-public class OldOnFileSortedOutput implements SortingOutput {
-
-  public OldOnFileSortedOutput(TezEngineTaskContext task) throws IOException {
-  }
-  
-  @Override
-  public void initialize(Configuration conf, Master master) 
-      throws IOException, InterruptedException {
-  }
-
-  @Override
-  public void setTask(RunningTaskContext task) {
-  }
-  
-  @Override
-  public void write(Object key, Object value) throws IOException,
-      InterruptedException {
-  }
-
-  @Override
-  public void close() throws IOException, InterruptedException {
-  }
-
-  @Override
-  public OutputContext getOutputContext() {
-    return null;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/lib/output/InMemorySortedOutput.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/lib/output/InMemorySortedOutput.java b/tez-engine/src/main/java/org/apache/tez/engine/lib/output/InMemorySortedOutput.java
index 5d2a2ba..218aa21 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/lib/output/InMemorySortedOutput.java
+++ b/tez-engine/src/main/java/org/apache/tez/engine/lib/output/InMemorySortedOutput.java
@@ -22,9 +22,9 @@ import java.util.Collections;
 import java.util.List;
 
 import org.apache.tez.common.TezUtils;
+import org.apache.tez.engine.api.KVWriter;
 import org.apache.tez.engine.common.sort.impl.dflt.InMemoryShuffleSorter;
 import org.apache.tez.engine.newapi.Event;
-import org.apache.tez.engine.newapi.KVWriter;
 import org.apache.tez.engine.newapi.LogicalOutput;
 import org.apache.tez.engine.newapi.Output;
 import org.apache.tez.engine.newapi.TezOutputContext;

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/lib/output/LocalOnFileSorterOutput.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/lib/output/LocalOnFileSorterOutput.java b/tez-engine/src/main/java/org/apache/tez/engine/lib/output/LocalOnFileSorterOutput.java
index d23ac1e..963276d 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/lib/output/LocalOnFileSorterOutput.java
+++ b/tez-engine/src/main/java/org/apache/tez/engine/lib/output/LocalOnFileSorterOutput.java
@@ -25,7 +25,7 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.tez.engine.common.task.local.newoutput.TezTaskOutput;
+import org.apache.tez.engine.common.task.local.output.TezTaskOutput;
 import org.apache.tez.engine.newapi.Event;
 
 public class LocalOnFileSorterOutput extends OnFileSortedOutput {

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/lib/output/OnFileSortedOutput.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/lib/output/OnFileSortedOutput.java b/tez-engine/src/main/java/org/apache/tez/engine/lib/output/OnFileSortedOutput.java
index ffb36c5..7e0ca37 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/lib/output/OnFileSortedOutput.java
+++ b/tez-engine/src/main/java/org/apache/tez/engine/lib/output/OnFileSortedOutput.java
@@ -26,11 +26,11 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.yarn.api.ApplicationConstants;
 import org.apache.tez.common.TezJobConfig;
 import org.apache.tez.common.TezUtils;
+import org.apache.tez.engine.api.KVWriter;
 import org.apache.tez.engine.common.shuffle.newimpl.ShuffleUserPayloads.DataMovementEventPayloadProto;
 import org.apache.tez.engine.common.sort.impl.ExternalSorter;
 import org.apache.tez.engine.common.sort.impl.dflt.DefaultSorter;
 import org.apache.tez.engine.newapi.Event;
-import org.apache.tez.engine.newapi.KVWriter;
 import org.apache.tez.engine.newapi.LogicalOutput;
 import org.apache.tez.engine.newapi.TezOutputContext;
 import org.apache.tez.engine.newapi.events.DataMovementEvent;

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/lib/output/OnFileUnorderedKVOutput.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/lib/output/OnFileUnorderedKVOutput.java b/tez-engine/src/main/java/org/apache/tez/engine/lib/output/OnFileUnorderedKVOutput.java
index ec193c5..37edde8 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/lib/output/OnFileUnorderedKVOutput.java
+++ b/tez-engine/src/main/java/org/apache/tez/engine/lib/output/OnFileUnorderedKVOutput.java
@@ -24,10 +24,10 @@ import java.util.List;
 
 import org.apache.hadoop.yarn.api.ApplicationConstants;
 import org.apache.tez.dag.api.TezUncheckedException;
+import org.apache.tez.engine.api.KVWriter;
 import org.apache.tez.engine.broadcast.output.FileBasedKVWriter;
 import org.apache.tez.engine.common.shuffle.newimpl.ShuffleUserPayloads.DataMovementEventPayloadProto;
 import org.apache.tez.engine.newapi.Event;
-import org.apache.tez.engine.newapi.KVWriter;
 import org.apache.tez.engine.newapi.LogicalOutput;
 import org.apache.tez.engine.newapi.TezOutputContext;
 import org.apache.tez.engine.newapi.events.DataMovementEvent;

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/newapi/KVReader.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/newapi/KVReader.java b/tez-engine/src/main/java/org/apache/tez/engine/newapi/KVReader.java
deleted file mode 100644
index 79615ce..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/newapi/KVReader.java
+++ /dev/null
@@ -1,79 +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.engine.newapi;
-
-import java.io.IOException;
-
-/**
- * A key/value(s) pair based {@link Reader}.
- * 
- * Example usage
- * <code>
- * while (kvReader.next()) {
- *   KVRecord kvRecord = getCurrentKV();
- *   Object key =  kvRecord.getKey();
- *   Iterable values = kvRecord.getValues();
- * </code>
- *
- */
-public interface KVReader extends Reader {
-
-  /**
-   * Moves to the next key/values(s) pair
-   * 
-   * @return true if another key/value(s) pair exists, false if there are no more.
-   * @throws IOException
-   *           if an error occurs
-   */
-  public boolean next() throws IOException;
-
-  /**
-   * Return the current key/value(s) pair. Use moveToNext() to advance.
-   * @return
-   * @throws IOException
-   */
-  public KVRecord getCurrentKV() throws IOException;
-  
-  // TODO NEWTEZ Move this to getCurrentKey and getCurrentValue independently. Otherwise usage pattern seems to be getCurrentKV.getKey, getCurrentKV.getValue
-  
-  // TODO NEWTEZ KVRecord which does not need to return a list!
-  // TODO NEWTEZ Parameterize this
-  /**
-   * Represents a key and an associated set of values
-   *
-   */
-  public static class KVRecord {
-
-    private Object key;
-    private Iterable<Object> values;
-
-    public KVRecord(Object key, Iterable<Object> values) {
-      this.key = key;
-      this.values = values;
-    }
-
-    public Object getKey() {
-      return this.key;
-    }
-
-    public Iterable<Object> getValues() {
-      return this.values;
-    }
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/newapi/KVWriter.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/newapi/KVWriter.java b/tez-engine/src/main/java/org/apache/tez/engine/newapi/KVWriter.java
deleted file mode 100644
index ad48912..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/newapi/KVWriter.java
+++ /dev/null
@@ -1,38 +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.engine.newapi;
-
-import java.io.IOException;
-
-/**
- * A key/value(s) pair based {@link Writer}
- */
-public interface KVWriter extends Writer {
-  /**
-   * Writes a key/value pair.
-   * 
-   * @param key
-   *          the key to write
-   * @param value
-   *          the value to write
-   * @throws IOException
-   *           if an error occurs
-   */
-  public void write(Object key, Object value) throws IOException;
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/newapi/events/TaskAttemptCompletedEvent.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/newapi/events/TaskAttemptCompletedEvent.java b/tez-engine/src/main/java/org/apache/tez/engine/newapi/events/TaskAttemptCompletedEvent.java
deleted file mode 100644
index d3a582d..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/newapi/events/TaskAttemptCompletedEvent.java
+++ /dev/null
@@ -1,28 +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.engine.newapi.events;
-
-import org.apache.tez.engine.newapi.Event;
-
-public class TaskAttemptCompletedEvent extends Event {
-
-  public TaskAttemptCompletedEvent() {
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/newapi/events/TaskAttemptFailedEvent.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/newapi/events/TaskAttemptFailedEvent.java b/tez-engine/src/main/java/org/apache/tez/engine/newapi/events/TaskAttemptFailedEvent.java
deleted file mode 100644
index 772d7fe..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/newapi/events/TaskAttemptFailedEvent.java
+++ /dev/null
@@ -1,35 +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.engine.newapi.events;
-
-import org.apache.tez.engine.newapi.Event;
-
-public class TaskAttemptFailedEvent extends Event {
-
-  private final String diagnostics;
-
-  public TaskAttemptFailedEvent(String diagnostics) {
-    this.diagnostics = diagnostics;
-  }
-
-  public String getDiagnostics() {
-    return diagnostics;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/newapi/events/TaskStatusUpdateEvent.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/newapi/events/TaskStatusUpdateEvent.java b/tez-engine/src/main/java/org/apache/tez/engine/newapi/events/TaskStatusUpdateEvent.java
deleted file mode 100644
index 0f09867..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/newapi/events/TaskStatusUpdateEvent.java
+++ /dev/null
@@ -1,70 +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.engine.newapi.events;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-
-import org.apache.hadoop.io.Writable;
-import org.apache.tez.common.counters.TezCounters;
-import org.apache.tez.engine.newapi.Event;
-
-public class TaskStatusUpdateEvent extends Event implements Writable {
-
-  private TezCounters tezCounters;
-  private float progress;
-
-  public TaskStatusUpdateEvent() {
-  }
-
-  public TaskStatusUpdateEvent(TezCounters tezCounters, float progress) {
-    this.tezCounters = tezCounters;
-    this.progress = progress;
-  }
-
-  public TezCounters getCounters() {
-    return tezCounters;
-  }
-
-  public float getProgress() {
-    return progress;
-  }
-
-  @Override
-  public void write(DataOutput out) throws IOException {
-    out.writeFloat(progress);
-    if (tezCounters != null) {
-      out.writeBoolean(true);
-      tezCounters.write(out);
-    } else {
-      out.writeBoolean(false);
-    }
-  }
-
-  @Override
-  public void readFields(DataInput in) throws IOException {
-    progress = in.readFloat();
-    if (in.readBoolean()) {
-      tezCounters = new TezCounters();
-      tezCounters.readFields(in);
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/newapi/impl/EventMetaData.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/newapi/impl/EventMetaData.java b/tez-engine/src/main/java/org/apache/tez/engine/newapi/impl/EventMetaData.java
deleted file mode 100644
index 9faafc5..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/newapi/impl/EventMetaData.java
+++ /dev/null
@@ -1,152 +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.engine.newapi.impl;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-
-import org.apache.hadoop.io.Writable;
-import org.apache.tez.dag.records.TezTaskAttemptID;
-
-/**
- * Class that encapsulates all the information to identify the unique
- * object that either generated an Event or is the recipient of an Event.
- */
-public class EventMetaData implements Writable {
-
-  public static enum EventProducerConsumerType {
-    INPUT,
-    PROCESSOR,
-    OUTPUT,
-    SYSTEM
-  }
-
-  /**
-   * Producer Type ( one of Input/Output/Processor ) that generated the Event
-   * or Consumer Type that will consume the Event.
-   */
-  private EventProducerConsumerType producerConsumerType;
-
-  /**
-   * Name of the vertex where the event was generated.
-   */
-  private String taskVertexName;
-
-  /**
-   * Name of the vertex to which the Input or Output is connected to.
-   */
-  private String edgeVertexName;
-
-  /**
-   * i'th physical input/output that this event maps to.
-   */
-  private int index;
-
-  /**
-   * Task Attempt ID
-   */
-  private TezTaskAttemptID taskAttemptID;
-
-  public EventMetaData() {
-  }
-
-  public EventMetaData(EventProducerConsumerType generator,
-      String taskVertexName, String edgeVertexName,
-      TezTaskAttemptID taskAttemptID) {
-    this.producerConsumerType = generator;
-    this.taskVertexName = taskVertexName;
-    this.edgeVertexName = edgeVertexName;
-    this.taskAttemptID = taskAttemptID;
-  }
-
-  public EventProducerConsumerType getEventGenerator() {
-    return producerConsumerType;
-  }
-
-  public TezTaskAttemptID getTaskAttemptID() {
-    return taskAttemptID;
-  }
-
-  public String getTaskVertexName() {
-    return taskVertexName;
-  }
-
-  public String getEdgeVertexName() {
-    return edgeVertexName;
-  }
-
-  @Override
-  public void write(DataOutput out) throws IOException {
-    out.writeInt(producerConsumerType.ordinal());
-    if (taskVertexName != null) {
-      out.writeBoolean(true);
-      out.writeUTF(taskVertexName);
-    } else {
-      out.writeBoolean(false);
-    }
-    if (edgeVertexName != null) {
-      out.writeBoolean(true);
-      out.writeUTF(edgeVertexName);
-    } else {
-      out.writeBoolean(false);
-    }
-    if(taskAttemptID != null) {
-      out.writeBoolean(true);
-      taskAttemptID.write(out);
-    } else {
-      out.writeBoolean(false);
-    }
-    
-    out.writeInt(index);
-  }
-
-  @Override
-  public void readFields(DataInput in) throws IOException {
-    producerConsumerType = EventProducerConsumerType.values()[in.readInt()];
-    if (in.readBoolean()) {
-      taskVertexName = in.readUTF();
-    }
-    if (in.readBoolean()) {
-      edgeVertexName = in.readUTF();
-    }
-    if (in.readBoolean()) {
-      taskAttemptID = new TezTaskAttemptID();
-      taskAttemptID.readFields(in);
-    }
-    index = in.readInt();
-  }
-
-  public int getIndex() {
-    return index;
-  }
-
-  public void setIndex(int index) {
-    this.index = index;
-  }
-
-  @Override
-  public String toString() {
-    return "{ producerConsumerType=" + producerConsumerType
-        + ", taskVertexName=" + taskVertexName
-        + ", edgeVertexName=" + edgeVertexName
-        + ", taskAttemptId=" + taskAttemptID
-        + ", index=" + index + " }";
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/newapi/impl/EventType.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/newapi/impl/EventType.java b/tez-engine/src/main/java/org/apache/tez/engine/newapi/impl/EventType.java
deleted file mode 100644
index 87d6665..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/newapi/impl/EventType.java
+++ /dev/null
@@ -1,29 +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.engine.newapi.impl;
-
-public enum EventType {
-  TASK_ATTEMPT_COMPLETED_EVENT,
-  TASK_ATTEMPT_FAILED_EVENT,
-  DATA_MOVEMENT_EVENT,
-  INPUT_READ_ERROR_EVENT,
-  INPUT_FAILED_EVENT,
-  INTPUT_INFORMATION_EVENT,
-  TASK_STATUS_UPDATE_EVENT
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/newapi/impl/InputSpec.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/newapi/impl/InputSpec.java b/tez-engine/src/main/java/org/apache/tez/engine/newapi/impl/InputSpec.java
deleted file mode 100644
index a2b8cc8..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/newapi/impl/InputSpec.java
+++ /dev/null
@@ -1,88 +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.engine.newapi.impl;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-
-import org.apache.hadoop.io.Writable;
-import org.apache.tez.dag.api.DagTypeConverters;
-import org.apache.tez.dag.api.InputDescriptor;
-import org.apache.tez.dag.api.records.DAGProtos.TezEntityDescriptorProto;
-
-public class InputSpec implements Writable {
-
-  private String sourceVertexName;
-  private InputDescriptor inputDescriptor;
-  private int physicalEdgeCount;
-
-  public InputSpec() {
-  }
-
-  public InputSpec(String sourceVertexName, InputDescriptor inputDescriptor,
-      int physicalEdgeCount) {
-    this.sourceVertexName = sourceVertexName;
-    this.inputDescriptor = inputDescriptor;
-    this.physicalEdgeCount = physicalEdgeCount;
-  }
-
-  public String getSourceVertexName() {
-    return sourceVertexName;
-  }
-
-  public InputDescriptor getInputDescriptor() {
-    return inputDescriptor;
-  }
-
-  public int getPhysicalEdgeCount() {
-    return physicalEdgeCount;
-  }
-
-  @Override
-  public void write(DataOutput out) throws IOException {
-    // TODONEWTEZ convert to PB
-    out.writeUTF(sourceVertexName);
-    out.writeInt(physicalEdgeCount);
-    byte[] inputDescBytes =
-        DagTypeConverters.convertToDAGPlan(inputDescriptor).toByteArray();
-    out.writeInt(inputDescBytes.length);
-    out.write(inputDescBytes);
-  }
-
-  @Override
-  public void readFields(DataInput in) throws IOException {
-    sourceVertexName = in.readUTF();
-    physicalEdgeCount = in.readInt();
-    int inputDescLen = in.readInt();
-    byte[] inputDescBytes = new byte[inputDescLen];
-    in.readFully(inputDescBytes);
-    inputDescriptor =
-        DagTypeConverters.convertInputDescriptorFromDAGPlan(
-            TezEntityDescriptorProto.parseFrom(inputDescBytes));
-  }
-
-  public String toString() {
-    return "{ sourceVertexName=" + sourceVertexName
-        + ", physicalEdgeCount" + physicalEdgeCount
-        + ", inputClassName=" + inputDescriptor.getClassName()
-        + " }";
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/newapi/impl/OutputSpec.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/newapi/impl/OutputSpec.java b/tez-engine/src/main/java/org/apache/tez/engine/newapi/impl/OutputSpec.java
deleted file mode 100644
index 1b34ef0..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/newapi/impl/OutputSpec.java
+++ /dev/null
@@ -1,87 +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.engine.newapi.impl;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-
-import org.apache.hadoop.io.Writable;
-import org.apache.tez.dag.api.DagTypeConverters;
-import org.apache.tez.dag.api.OutputDescriptor;
-import org.apache.tez.dag.api.records.DAGProtos.TezEntityDescriptorProto;
-
-public class OutputSpec implements Writable {
-
-  private String destinationVertexName;
-  private OutputDescriptor outputDescriptor;
-  private int physicalEdgeCount;
-
-  public OutputSpec() {
-  }
-
-  public OutputSpec(String destinationVertexName,
-      OutputDescriptor inputDescriptor, int physicalEdgeCount) {
-    this.destinationVertexName = destinationVertexName;
-    this.outputDescriptor = inputDescriptor;
-    this.physicalEdgeCount = physicalEdgeCount;
-  }
-
-  public String getDestinationVertexName() {
-    return destinationVertexName;
-  }
-
-  public OutputDescriptor getOutputDescriptor() {
-    return outputDescriptor;
-  }
-
-  public int getPhysicalEdgeCount() {
-    return physicalEdgeCount;
-  }
-
-  @Override
-  public void write(DataOutput out) throws IOException {
-    // TODONEWTEZ convert to PB
-    out.writeUTF(destinationVertexName);
-    out.writeInt(physicalEdgeCount);
-    byte[] inputDescBytes =
-        DagTypeConverters.convertToDAGPlan(outputDescriptor).toByteArray();
-    out.writeInt(inputDescBytes.length);
-    out.write(inputDescBytes);
-  }
-
-  @Override
-  public void readFields(DataInput in) throws IOException {
-    destinationVertexName = in.readUTF();
-    physicalEdgeCount = in.readInt();
-    int inputDescLen = in.readInt();
-    byte[] inputDescBytes = new byte[inputDescLen];
-    in.readFully(inputDescBytes);
-    outputDescriptor =
-        DagTypeConverters.convertOutputDescriptorFromDAGPlan(
-            TezEntityDescriptorProto.parseFrom(inputDescBytes));
-  }
-
-  public String toString() {
-    return "{ destinationVertexName=" + destinationVertexName
-        + ", physicalEdgeCount" + physicalEdgeCount
-        + ", outputClassName=" + outputDescriptor.getClassName()
-        + " }";
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/newapi/impl/TaskSpec.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/newapi/impl/TaskSpec.java b/tez-engine/src/main/java/org/apache/tez/engine/newapi/impl/TaskSpec.java
deleted file mode 100644
index 8290e30..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/newapi/impl/TaskSpec.java
+++ /dev/null
@@ -1,146 +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.engine.newapi.impl;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.hadoop.io.Writable;
-import org.apache.tez.dag.api.DagTypeConverters;
-import org.apache.tez.dag.api.ProcessorDescriptor;
-import org.apache.tez.dag.api.records.DAGProtos.TezEntityDescriptorProto;
-import org.apache.tez.dag.records.TezTaskAttemptID;
-
-public class TaskSpec implements Writable {
-
-  private TezTaskAttemptID taskAttemptId;
-  private String vertexName;
-  private String user;
-  private ProcessorDescriptor processorDescriptor;
-  private List<InputSpec> inputSpecList;
-  private List<OutputSpec> outputSpecList;
-
-  public TaskSpec() {
-  }
-
-  // TODO NEWTEZ Remove user
-  public TaskSpec(TezTaskAttemptID taskAttemptID, String user,
-      String vertexName, ProcessorDescriptor processorDescriptor,
-      List<InputSpec> inputSpecList, List<OutputSpec> outputSpecList) {
-    this.taskAttemptId = taskAttemptID;
-    this.vertexName = vertexName;
-    this.user = user;
-    this.processorDescriptor = processorDescriptor;
-    this.inputSpecList = inputSpecList;
-    this.outputSpecList = outputSpecList;
-  }
-
-  public String getVertexName() {
-    return vertexName;
-  }
-
-  public TezTaskAttemptID getTaskAttemptID() {
-    return taskAttemptId;
-  }
-
-  public String getUser() {
-    return user;
-  }
-
-  public ProcessorDescriptor getProcessorDescriptor() {
-    return processorDescriptor;
-  }
-
-  public List<InputSpec> getInputs() {
-    return inputSpecList;
-  }
-
-  public List<OutputSpec> getOutputs() {
-    return outputSpecList;
-  }
-
-  @Override
-  public void write(DataOutput out) throws IOException {
-    taskAttemptId.write(out);
-    out.writeUTF(vertexName);
-    byte[] procDesc =
-        DagTypeConverters.convertToDAGPlan(processorDescriptor).toByteArray();
-    out.writeInt(procDesc.length);
-    out.write(procDesc);
-    out.writeInt(inputSpecList.size());
-    for (InputSpec inputSpec : inputSpecList) {
-      inputSpec.write(out);
-    }
-    out.writeInt(outputSpecList.size());
-    for (OutputSpec outputSpec : outputSpecList) {
-      outputSpec.write(out);
-    }
-  }
-
-  @Override
-  public void readFields(DataInput in) throws IOException {
-    taskAttemptId = new TezTaskAttemptID();
-    taskAttemptId.readFields(in);
-    vertexName = in.readUTF();
-    int procDescLength = in.readInt();
-    // TODO at least 3 buffer copies here. Need to convert this to full PB
-    // TEZ-305
-    byte[] procDescBytes = new byte[procDescLength];
-    in.readFully(procDescBytes);
-    processorDescriptor =
-        DagTypeConverters.convertProcessorDescriptorFromDAGPlan(
-            TezEntityDescriptorProto.parseFrom(procDescBytes));
-    int numInputSpecs = in.readInt();
-    inputSpecList = new ArrayList<InputSpec>(numInputSpecs);
-    for (int i = 0; i < numInputSpecs; i++) {
-      InputSpec inputSpec = new InputSpec();
-      inputSpec.readFields(in);
-      inputSpecList.add(inputSpec);
-    }
-    int numOutputSpecs = in.readInt();
-    outputSpecList = new ArrayList<OutputSpec>(numOutputSpecs);
-    for (int i = 0; i < numOutputSpecs; i++) {
-      OutputSpec outputSpec = new OutputSpec();
-      outputSpec.readFields(in);
-      outputSpecList.add(outputSpec);
-    }
-  }
-
-  @Override
-  public String toString() {
-    StringBuffer sb = new StringBuffer();
-    sb.append("TaskAttemptID:" + taskAttemptId);
-    sb.append("processorName=" + processorDescriptor.getClassName()
-        + ", inputSpecListSize=" + inputSpecList.size()
-        + ", outputSpecListSize=" + outputSpecList.size());
-    sb.append(", inputSpecList=[");
-    for (InputSpec i : inputSpecList) {
-      sb.append("{" + i.toString() + "}, ");
-    }
-    sb.append("], outputSpecList=[");
-    for (OutputSpec i : outputSpecList) {
-      sb.append("{" + i.toString() + "}, ");
-    }
-    sb.append("]");
-    return sb.toString();
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/newapi/impl/TezEvent.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/newapi/impl/TezEvent.java b/tez-engine/src/main/java/org/apache/tez/engine/newapi/impl/TezEvent.java
deleted file mode 100644
index 0f65750..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/newapi/impl/TezEvent.java
+++ /dev/null
@@ -1,248 +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.engine.newapi.impl;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-
-import org.apache.hadoop.io.Writable;
-import org.apache.tez.dag.api.TezUncheckedException;
-import org.apache.tez.engine.api.events.EventProtos.DataMovementEventProto;
-import org.apache.tez.engine.api.events.EventProtos.InputFailedEventProto;
-import org.apache.tez.engine.api.events.EventProtos.InputInformationEventProto;
-import org.apache.tez.engine.api.events.EventProtos.InputReadErrorEventProto;
-import org.apache.tez.engine.api.events.SystemEventProtos.TaskAttemptCompletedEventProto;
-import org.apache.tez.engine.api.events.SystemEventProtos.TaskAttemptFailedEventProto;
-import org.apache.tez.engine.newapi.Event;
-import org.apache.tez.engine.newapi.events.DataMovementEvent;
-import org.apache.tez.engine.newapi.events.InputFailedEvent;
-import org.apache.tez.engine.newapi.events.InputInformationEvent;
-import org.apache.tez.engine.newapi.events.InputReadErrorEvent;
-import org.apache.tez.engine.newapi.events.TaskAttemptCompletedEvent;
-import org.apache.tez.engine.newapi.events.TaskAttemptFailedEvent;
-import org.apache.tez.engine.newapi.events.TaskStatusUpdateEvent;
-
-import com.google.protobuf.ByteString;
-
-public class TezEvent implements Writable {
-
-  private EventType eventType;
-
-  private Event event;
-
-  private EventMetaData sourceInfo;
-
-  private EventMetaData destinationInfo;
-
-  public TezEvent() {
-  }
-
-  public TezEvent(Event event, EventMetaData sourceInfo) {
-    this.event = event;
-    this.setSourceInfo(sourceInfo);
-    if (event instanceof DataMovementEvent) {
-      eventType = EventType.DATA_MOVEMENT_EVENT;
-    } else if (event instanceof InputReadErrorEvent) {
-      eventType = EventType.INPUT_READ_ERROR_EVENT;
-    } else if (event instanceof TaskAttemptFailedEvent) {
-      eventType = EventType.TASK_ATTEMPT_FAILED_EVENT;
-    } else if (event instanceof TaskAttemptCompletedEvent) {
-      eventType = EventType.TASK_ATTEMPT_COMPLETED_EVENT;
-    } else if (event instanceof InputInformationEvent) {
-      eventType = EventType.INTPUT_INFORMATION_EVENT;
-    } else if (event instanceof InputFailedEvent) {
-      eventType = EventType.INPUT_FAILED_EVENT;
-    } else if (event instanceof TaskStatusUpdateEvent) {
-      eventType = EventType.TASK_STATUS_UPDATE_EVENT;
-    } else {
-      throw new TezUncheckedException("Unknown event, event="
-          + event.getClass().getName());
-    }
-  }
-
-  public Event getEvent() {
-    return event;
-  }
-
-  public EventMetaData getSourceInfo() {
-    return sourceInfo;
-  }
-
-  public void setSourceInfo(EventMetaData sourceInfo) {
-    this.sourceInfo = sourceInfo;
-  }
-
-  public EventMetaData getDestinationInfo() {
-    return destinationInfo;
-  }
-
-  public void setDestinationInfo(EventMetaData destinationInfo) {
-    this.destinationInfo = destinationInfo;
-  }
-
-  public EventType getEventType() {
-    return eventType;
-  }
-
-  private void serializeEvent(DataOutput out) throws IOException {
-    if (event == null) {
-      out.writeBoolean(false);
-      return;
-    }
-    out.writeBoolean(true);
-    out.writeInt(eventType.ordinal());
-    if (eventType.equals(EventType.TASK_STATUS_UPDATE_EVENT)) {
-      // TODO NEWTEZ convert to PB
-      TaskStatusUpdateEvent sEvt = (TaskStatusUpdateEvent) event;
-      sEvt.write(out);
-    } else {
-      byte[] eventBytes = null;
-      switch (eventType) {
-      case DATA_MOVEMENT_EVENT:
-        DataMovementEvent dmEvt = (DataMovementEvent) event;
-        eventBytes = DataMovementEventProto.newBuilder()
-          .setSourceIndex(dmEvt.getSourceIndex())
-          .setTargetIndex(dmEvt.getTargetIndex())
-          .setUserPayload(ByteString.copyFrom(dmEvt.getUserPayload()))
-          .build().toByteArray();
-        break;
-      case INPUT_READ_ERROR_EVENT:
-        InputReadErrorEvent ideEvt = (InputReadErrorEvent) event;
-        eventBytes = InputReadErrorEventProto.newBuilder()
-            .setIndex(ideEvt.getIndex())
-            .setDiagnostics(ideEvt.getDiagnostics())
-            .build().toByteArray();
-        break;
-      case TASK_ATTEMPT_FAILED_EVENT:
-        TaskAttemptFailedEvent tfEvt = (TaskAttemptFailedEvent) event;
-        eventBytes = TaskAttemptFailedEventProto.newBuilder()
-            .setDiagnostics(tfEvt.getDiagnostics())
-            .build().toByteArray();
-        break;
-      case TASK_ATTEMPT_COMPLETED_EVENT:
-        eventBytes = TaskAttemptCompletedEventProto.newBuilder()
-            .build().toByteArray();
-        break;
-      case INPUT_FAILED_EVENT:
-        InputFailedEvent ifEvt = (InputFailedEvent) event;
-        eventBytes = InputFailedEventProto.newBuilder()
-            .setSourceIndex(ifEvt.getSourceIndex())
-            .setTargetIndex(ifEvt.getTargetIndex())
-            .setVersion(ifEvt.getVersion()).build().toByteArray();
-      case INTPUT_INFORMATION_EVENT:
-        InputInformationEvent iEvt = (InputInformationEvent) event;
-        eventBytes = InputInformationEventProto.newBuilder()
-            .setUserPayload(ByteString.copyFrom(iEvt.getUserPayload()))
-            .build().toByteArray();
-      default:
-        throw new TezUncheckedException("Unknown TezEvent"
-           + ", type=" + eventType);
-      }
-      out.writeInt(eventBytes.length);
-      out.write(eventBytes);
-    }
-  }
-
-  private void deserializeEvent(DataInput in) throws IOException {
-    if (!in.readBoolean()) {
-      event = null;
-      return;
-    }
-    eventType = EventType.values()[in.readInt()];
-    if (eventType.equals(EventType.TASK_STATUS_UPDATE_EVENT)) {
-      // TODO NEWTEZ convert to PB
-      event = new TaskStatusUpdateEvent();
-      ((TaskStatusUpdateEvent)event).readFields(in);
-    } else {
-      int eventBytesLen = in.readInt();
-      byte[] eventBytes = new byte[eventBytesLen];
-      in.readFully(eventBytes);
-      switch (eventType) {
-      case DATA_MOVEMENT_EVENT:
-        DataMovementEventProto dmProto =
-            DataMovementEventProto.parseFrom(eventBytes);
-        event = new DataMovementEvent(dmProto.getSourceIndex(),
-            dmProto.getTargetIndex(),
-            dmProto.getUserPayload().toByteArray());
-        break;
-      case INPUT_READ_ERROR_EVENT:
-        InputReadErrorEventProto ideProto =
-            InputReadErrorEventProto.parseFrom(eventBytes);
-        event = new InputReadErrorEvent(ideProto.getDiagnostics(),
-            ideProto.getIndex(), ideProto.getVersion());
-        break;
-      case TASK_ATTEMPT_FAILED_EVENT:
-        TaskAttemptFailedEventProto tfProto =
-            TaskAttemptFailedEventProto.parseFrom(eventBytes);
-        event = new TaskAttemptFailedEvent(tfProto.getDiagnostics());
-        break;
-      case TASK_ATTEMPT_COMPLETED_EVENT:
-        event = new TaskAttemptCompletedEvent();
-        break;
-      case INPUT_FAILED_EVENT:
-        InputFailedEventProto ifProto =
-            InputFailedEventProto.parseFrom(eventBytes);
-        event = new InputFailedEvent(ifProto.getSourceIndex(),
-            ifProto.getTargetIndex(), ifProto.getVersion());
-        break;
-      case INTPUT_INFORMATION_EVENT:
-        InputInformationEventProto infoProto =
-            InputInformationEventProto.parseFrom(eventBytes);
-        event = new InputInformationEvent(
-            infoProto.getUserPayload().toByteArray());
-        break;
-      default:
-        throw new TezUncheckedException("Unknown TezEvent"
-           + ", type=" + eventType);
-      }
-    }
-  }
-
-  @Override
-  public void write(DataOutput out) throws IOException {
-    serializeEvent(out);
-    if (sourceInfo != null) {
-      out.writeBoolean(true);
-      sourceInfo.write(out);
-    } else {
-      out.writeBoolean(false);
-    }
-    if (destinationInfo != null) {
-      out.writeBoolean(true);
-      destinationInfo.write(out);
-    } else {
-      out.writeBoolean(false);
-    }
-  }
-
-  @Override
-  public void readFields(DataInput in) throws IOException {
-    deserializeEvent(in);
-    if (in.readBoolean()) {
-      sourceInfo = new EventMetaData();
-      sourceInfo.readFields(in);
-    }
-    if (in.readBoolean()) {
-      destinationInfo = new EventMetaData();
-      destinationInfo.readFields(in);
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/newapi/impl/TezHeartbeatRequest.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/newapi/impl/TezHeartbeatRequest.java b/tez-engine/src/main/java/org/apache/tez/engine/newapi/impl/TezHeartbeatRequest.java
deleted file mode 100644
index 79a0968..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/newapi/impl/TezHeartbeatRequest.java
+++ /dev/null
@@ -1,137 +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.engine.newapi.impl;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.Writable;
-import org.apache.tez.dag.records.TezTaskAttemptID;
-
-
-public class TezHeartbeatRequest implements Writable {
-
-  private String containerIdentifier;
-  private List<TezEvent> events;
-  private TezTaskAttemptID currentTaskAttemptID;
-  private int startIndex;
-  private int maxEvents;
-  private long requestId;
-
-  public TezHeartbeatRequest() {
-  }
-
-  public TezHeartbeatRequest(long requestId, List<TezEvent> events,
-      String containerIdentifier, TezTaskAttemptID taskAttemptID,
-      int startIndex, int maxEvents) {
-    this.containerIdentifier = containerIdentifier;
-    this.requestId = requestId;
-    this.events = Collections.unmodifiableList(events);
-    this.startIndex = startIndex;
-    this.maxEvents = maxEvents;
-    this.currentTaskAttemptID = taskAttemptID;
-  }
-
-  public String getContainerIdentifier() {
-    return containerIdentifier;
-  }
-
-  public List<TezEvent> getEvents() {
-    return events;
-  }
-
-  public int getStartIndex() {
-    return startIndex;
-  }
-
-  public int getMaxEvents() {
-    return maxEvents;
-  }
-
-  public long getRequestId() {
-    return requestId;
-  }
-
-  public TezTaskAttemptID getCurrentTaskAttemptID() {
-    return currentTaskAttemptID;
-  }
-
-  @Override
-  public void write(DataOutput out) throws IOException {
-    if (events != null) {
-      out.writeBoolean(true);
-      out.writeInt(events.size());
-      for (TezEvent e : events) {
-        e.write(out);
-      }
-    } else {
-      out.writeBoolean(false);
-    }
-    if (currentTaskAttemptID != null) {
-      out.writeBoolean(true);
-      currentTaskAttemptID.write(out);
-    } else {
-      out.writeBoolean(false);
-    }
-    out.writeInt(startIndex);
-    out.writeInt(maxEvents);
-    out.writeLong(requestId);
-    Text.writeString(out, containerIdentifier);
-  }
-
-  @Override
-  public void readFields(DataInput in) throws IOException {
-    if (in.readBoolean()) {
-      int eventsCount = in.readInt();
-      events = new ArrayList<TezEvent>(eventsCount);
-      for (int i = 0; i < eventsCount; ++i) {
-        TezEvent e = new TezEvent();
-        e.readFields(in);
-        events.add(e);
-      }
-    }
-    if (in.readBoolean()) {
-      currentTaskAttemptID = new TezTaskAttemptID();
-      currentTaskAttemptID.readFields(in);
-    } else {
-      currentTaskAttemptID = null;
-    }
-    startIndex = in.readInt();
-    maxEvents = in.readInt();
-    requestId = in.readLong();
-    containerIdentifier = Text.readString(in);
-  }
-
-  @Override
-  public String toString() {
-    return "{ "
-        + " containerId=" + containerIdentifier
-        + ", requestId=" + requestId
-        + ", startIndex=" + startIndex
-        + ", maxEventsToGet=" + maxEvents
-        + ", taskAttemptId" + currentTaskAttemptID
-        + ", eventCount=" + (events != null ? events.size() : 0)
-        + " }";
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/newapi/impl/TezHeartbeatResponse.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/newapi/impl/TezHeartbeatResponse.java b/tez-engine/src/main/java/org/apache/tez/engine/newapi/impl/TezHeartbeatResponse.java
deleted file mode 100644
index addd17f..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/newapi/impl/TezHeartbeatResponse.java
+++ /dev/null
@@ -1,105 +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.engine.newapi.impl;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-
-import org.apache.hadoop.io.Writable;
-
-public class TezHeartbeatResponse implements Writable {
-
-  private long lastRequestId;
-  private boolean shouldDie = false;
-  private List<TezEvent> events;
-
-  public TezHeartbeatResponse() {
-  }
-
-  public TezHeartbeatResponse(List<TezEvent> events) {
-    this.events = Collections.unmodifiableList(events);
-  }
-
-  public List<TezEvent> getEvents() {
-    return events;
-  }
-
-  public boolean shouldDie() {
-    return shouldDie;
-  }
-
-  public long getLastRequestId() {
-    return lastRequestId;
-  }
-
-  public void setEvents(List<TezEvent> events) {
-    this.events = Collections.unmodifiableList(events);
-  }
-
-  public void setLastRequestId(long lastRequestId ) {
-    this.lastRequestId = lastRequestId;
-  }
-
-  public void setShouldDie() {
-    this.shouldDie = true;
-  }
-
-  @Override
-  public void write(DataOutput out) throws IOException {
-    out.writeLong(lastRequestId);
-    out.writeBoolean(shouldDie);
-    if(events != null) {
-      out.writeBoolean(true);
-      out.writeInt(events.size());
-      for (TezEvent e : events) {
-        e.write(out);
-      }
-    } else {
-      out.writeBoolean(false);
-    }
-  }
-
-  @Override
-  public void readFields(DataInput in) throws IOException {
-    lastRequestId = in.readLong();
-    shouldDie = in.readBoolean();
-    if(in.readBoolean()) {
-      int eventCount = in.readInt();
-      events = new ArrayList<TezEvent>(eventCount);
-      for (int i = 0; i < eventCount; ++i) {
-        TezEvent e = new TezEvent();
-        e.readFields(in);
-        events.add(e);
-      }
-    }
-  }
-
-  @Override
-  public String toString() {
-    return "{ "
-        + " lastRequestId=" + lastRequestId
-        + ", shouldDie=" + shouldDie
-        + ", eventCount=" + (events != null ? events.size() : 0)
-        + " }";
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/newapi/impl/TezInputContextImpl.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/newapi/impl/TezInputContextImpl.java b/tez-engine/src/main/java/org/apache/tez/engine/newapi/impl/TezInputContextImpl.java
deleted file mode 100644
index daafc5a..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/newapi/impl/TezInputContextImpl.java
+++ /dev/null
@@ -1,84 +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.engine.newapi.impl;
-
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.hadoop.classification.InterfaceAudience.Private;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.tez.common.counters.TezCounters;
-import org.apache.tez.dag.records.TezTaskAttemptID;
-import org.apache.tez.engine.newapi.Event;
-import org.apache.tez.engine.newapi.TezInputContext;
-import org.apache.tez.engine.newapi.impl.EventMetaData.EventProducerConsumerType;
-import org.apache.tez.engine.newruntime.RuntimeTask;
-
-public class TezInputContextImpl extends TezTaskContextImpl
-    implements TezInputContext {
-
-  private final byte[] userPayload;
-  private final String sourceVertexName;
-  private final EventMetaData sourceInfo;
-
-  @Private
-  public TezInputContextImpl(Configuration conf, int appAttemptNumber,
-      TezUmbilical tezUmbilical, String taskVertexName,
-      String sourceVertexName, TezTaskAttemptID taskAttemptID,
-      TezCounters counters, byte[] userPayload,
-      RuntimeTask runtimeTask, Map<String, ByteBuffer> serviceConsumerMetadata) {
-    super(conf, appAttemptNumber, taskVertexName, taskAttemptID,
-        counters, runtimeTask, tezUmbilical, serviceConsumerMetadata);
-    this.userPayload = userPayload;
-    this.sourceVertexName = sourceVertexName;
-    this.sourceInfo = new EventMetaData(
-        EventProducerConsumerType.INPUT, taskVertexName, sourceVertexName,
-        taskAttemptID);
-    this.uniqueIdentifier = String.format("%s_%s_%06d_%02d_%s", taskAttemptID
-        .getTaskID().getVertexID().getDAGId().toString(), taskVertexName,
-        getTaskIndex(), getTaskAttemptNumber(), sourceVertexName);
-  }
-
-  @Override
-  public void sendEvents(List<Event> events) {
-    List<TezEvent> tezEvents = new ArrayList<TezEvent>(events.size());
-    for (Event e : events) {
-      TezEvent tEvt = new TezEvent(e, sourceInfo);
-      tezEvents.add(tEvt);
-    }
-    tezUmbilical.addEvents(tezEvents);
-  }
-
-  @Override
-  public byte[] getUserPayload() {
-    return userPayload;
-  }
-
-  @Override
-  public String getSourceVertexName() {
-    return sourceVertexName;
-  }
-
-  @Override
-  public void fatalError(Throwable exception, String message) {
-    super.signalFatalError(exception, message, sourceInfo);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/newapi/impl/TezOutputContextImpl.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/newapi/impl/TezOutputContextImpl.java b/tez-engine/src/main/java/org/apache/tez/engine/newapi/impl/TezOutputContextImpl.java
deleted file mode 100644
index 9de41ae..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/newapi/impl/TezOutputContextImpl.java
+++ /dev/null
@@ -1,85 +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.engine.newapi.impl;
-
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.hadoop.classification.InterfaceAudience.Private;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.tez.common.counters.TezCounters;
-import org.apache.tez.dag.records.TezTaskAttemptID;
-import org.apache.tez.engine.newapi.Event;
-import org.apache.tez.engine.newapi.TezOutputContext;
-import org.apache.tez.engine.newapi.impl.EventMetaData.EventProducerConsumerType;
-import org.apache.tez.engine.newruntime.RuntimeTask;
-
-public class TezOutputContextImpl extends TezTaskContextImpl
-    implements TezOutputContext {
-
-  private final byte[] userPayload;
-  private final String destinationVertexName;
-  private final EventMetaData sourceInfo;
-
-  @Private
-  public TezOutputContextImpl(Configuration conf, int appAttemptNumber,
-      TezUmbilical tezUmbilical, String taskVertexName,
-      String destinationVertexName,
-      TezTaskAttemptID taskAttemptID, TezCounters counters,
-      byte[] userPayload, RuntimeTask runtimeTask,
-      Map<String, ByteBuffer> serviceConsumerMetadata) {
-    super(conf, appAttemptNumber, taskVertexName, taskAttemptID,
-        counters, runtimeTask, tezUmbilical, serviceConsumerMetadata);
-    this.userPayload = userPayload;
-    this.destinationVertexName = destinationVertexName;
-    this.sourceInfo = new EventMetaData(EventProducerConsumerType.OUTPUT,
-        taskVertexName, destinationVertexName, taskAttemptID);
-    this.uniqueIdentifier = String.format("%s_%s_%06d_%02d_%s", taskAttemptID
-        .getTaskID().getVertexID().getDAGId().toString(), taskVertexName,
-        getTaskIndex(), getTaskAttemptNumber(), destinationVertexName);
-  }
-
-  @Override
-  public void sendEvents(List<Event> events) {
-    List<TezEvent> tezEvents = new ArrayList<TezEvent>(events.size());
-    for (Event e : events) {
-      TezEvent tEvt = new TezEvent(e, sourceInfo);
-      tezEvents.add(tEvt);
-    }
-    tezUmbilical.addEvents(tezEvents);
-  }
-
-  @Override
-  public byte[] getUserPayload() {
-    return userPayload;
-  }
-
-  @Override
-  public String getDestinationVertexName() {
-    return destinationVertexName;
-  }
-
-  @Override
-  public void fatalError(Throwable exception, String message) {
-    super.signalFatalError(exception, message, sourceInfo);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/newapi/impl/TezProcessorContextImpl.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/newapi/impl/TezProcessorContextImpl.java b/tez-engine/src/main/java/org/apache/tez/engine/newapi/impl/TezProcessorContextImpl.java
deleted file mode 100644
index d710f7a..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/newapi/impl/TezProcessorContextImpl.java
+++ /dev/null
@@ -1,86 +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.engine.newapi.impl;
-
-import java.nio.ByteBuffer;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.tez.common.counters.TezCounters;
-import org.apache.tez.dag.records.TezTaskAttemptID;
-import org.apache.tez.engine.newapi.Event;
-import org.apache.tez.engine.newapi.TezProcessorContext;
-import org.apache.tez.engine.newapi.impl.EventMetaData.EventProducerConsumerType;
-import org.apache.tez.engine.newruntime.RuntimeTask;
-
-public class TezProcessorContextImpl extends TezTaskContextImpl
-  implements TezProcessorContext {
-
-  private final byte[] userPayload;
-  private final EventMetaData sourceInfo;
-
-  public TezProcessorContextImpl(Configuration conf, int appAttemptNumber,
-      TezUmbilical tezUmbilical, String vertexName,
-      TezTaskAttemptID taskAttemptID, TezCounters counters,
-      byte[] userPayload, RuntimeTask runtimeTask,
-      Map<String, ByteBuffer> serviceConsumerMetadata) {
-    super(conf, appAttemptNumber, vertexName, taskAttemptID,
-        counters, runtimeTask, tezUmbilical, serviceConsumerMetadata);
-    this.userPayload = userPayload;
-    this.sourceInfo = new EventMetaData(EventProducerConsumerType.PROCESSOR,
-        taskVertexName, "", taskAttemptID);
-    this.uniqueIdentifier = String.format("%s_%s_%06d_%02d", taskAttemptID
-        .getTaskID().getVertexID().getDAGId().toString(), taskVertexName,
-        getTaskIndex(), getTaskAttemptNumber());
-  }
-
-  @Override
-  public void sendEvents(List<Event> events) {
-    List<TezEvent> tezEvents = new ArrayList<TezEvent>(events.size());
-    for (Event e : events) {
-      TezEvent tEvt = new TezEvent(e, sourceInfo);
-      tezEvents.add(tEvt);
-    }
-    tezUmbilical.addEvents(tezEvents);
-  }
-
-  @Override
-  public byte[] getUserPayload() {
-    return userPayload;
-  }
-
-  @Override
-  public void setProgress(float progress) {
-    runtimeTask.setProgress(progress);
-  }
-
-  @Override
-  public void fatalError(Throwable exception, String message) {
-    super.signalFatalError(exception, message, sourceInfo);
-  }
-
-  @Override
-  public boolean canCommit() throws IOException {
-    return tezUmbilical.canCommit(this.taskAttemptID);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/newapi/impl/TezTaskContextImpl.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/newapi/impl/TezTaskContextImpl.java b/tez-engine/src/main/java/org/apache/tez/engine/newapi/impl/TezTaskContextImpl.java
deleted file mode 100644
index 1d17158..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/newapi/impl/TezTaskContextImpl.java
+++ /dev/null
@@ -1,145 +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.engine.newapi.impl;
-
-import java.nio.ByteBuffer;
-import java.util.Arrays;
-import java.util.Map;
-
-import org.apache.hadoop.classification.InterfaceAudience.Private;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.util.StringUtils;
-import org.apache.hadoop.yarn.api.records.ApplicationId;
-import org.apache.hadoop.yarn.util.AuxiliaryServiceHelper;
-import org.apache.tez.common.TezJobConfig;
-import org.apache.tez.common.counters.TezCounters;
-import org.apache.tez.dag.records.TezTaskAttemptID;
-import org.apache.tez.engine.newapi.TezTaskContext;
-import org.apache.tez.engine.newruntime.RuntimeTask;
-
-public abstract class TezTaskContextImpl implements TezTaskContext {
-
-  private final Configuration conf;
-  protected final String taskVertexName;
-  protected final TezTaskAttemptID taskAttemptID;
-  private final TezCounters counters;
-  private String[] workDirs;
-  protected String uniqueIdentifier;
-  protected final RuntimeTask runtimeTask;
-  protected final TezUmbilical tezUmbilical;
-  private final Map<String, ByteBuffer> serviceConsumerMetadata;
-  private final int appAttemptNumber;
-
-  @Private
-  public TezTaskContextImpl(Configuration conf, int appAttemptNumber,
-      String taskVertexName, TezTaskAttemptID taskAttemptID,
-      TezCounters counters, RuntimeTask runtimeTask,
-      TezUmbilical tezUmbilical, Map<String, ByteBuffer> serviceConsumerMetadata) {
-    this.conf = conf;
-    this.taskVertexName = taskVertexName;
-    this.taskAttemptID = taskAttemptID;
-    this.counters = counters;
-    // TODO Maybe change this to be task id specific at some point. For now
-    // Shuffle code relies on this being a path specified by YARN
-    this.workDirs = this.conf.getStrings(TezJobConfig.LOCAL_DIRS);
-    this.runtimeTask = runtimeTask;
-    this.tezUmbilical = tezUmbilical;
-    this.serviceConsumerMetadata = serviceConsumerMetadata;
-    // TODO NEWTEZ at some point dag attempt should not map to app attempt
-    this.appAttemptNumber = appAttemptNumber;
-  }
-
-  @Override
-  public ApplicationId getApplicationId() {
-    return taskAttemptID.getTaskID().getVertexID().getDAGId()
-        .getApplicationId();
-  }
-
-  @Override
-  public int getTaskIndex() {
-    return taskAttemptID.getTaskID().getId();
-  }
-
-  @Override
-  public int getDAGAttemptNumber() {
-    return appAttemptNumber;
-  }
-
-  @Override
-  public int getTaskAttemptNumber() {
-    return taskAttemptID.getId();
-  }
-
-  @Override
-  public String getDAGName() {
-    // TODO NEWTEZ Change to some form of the DAG name, for now using dagId as
-    // the unique identifier.
-    return taskAttemptID.getTaskID().getVertexID().getDAGId().toString();
-  }
-
-  @Override
-  public String getTaskVertexName() {
-    return taskVertexName;
-  }
-
-
-  @Override
-  public TezCounters getCounters() {
-    return counters;
-  }
-
-  @Override
-  public String[] getWorkDirs() {
-    return Arrays.copyOf(workDirs, workDirs.length);
-  }
-
-  @Override
-  public String getUniqueIdentifier() {
-    return uniqueIdentifier;
-  }
-
-  @Override
-  public ByteBuffer getServiceConsumerMetaData(String serviceName) {
-    return (ByteBuffer) serviceConsumerMetadata.get(serviceName)
-        .asReadOnlyBuffer().rewind();
-  }
-
-  @Override
-  public ByteBuffer getServiceProviderMetaData(String serviceName) {
-    return AuxiliaryServiceHelper.getServiceDataFromEnv(
-        serviceName, System.getenv());
-  }
-
-  protected void signalFatalError(Throwable t, String message,
-      EventMetaData sourceInfo) {
-    runtimeTask.setFatalError(t, message);
-    String diagnostics;
-    if (t != null && message != null) {
-      diagnostics = "exceptionThrown=" + StringUtils.stringifyException(t)
-          + ", errorMessage=" + message;
-    } else if (t == null && message == null) {
-      diagnostics = "Unknown error";
-    } else {
-      diagnostics = t != null ?
-          "exceptionThrown=" + StringUtils.stringifyException(t)
-          : " errorMessage=" + message;
-    }
-    tezUmbilical.signalFatalError(taskAttemptID, diagnostics, sourceInfo);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/newapi/impl/TezUmbilical.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/newapi/impl/TezUmbilical.java b/tez-engine/src/main/java/org/apache/tez/engine/newapi/impl/TezUmbilical.java
deleted file mode 100644
index 5889622..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/newapi/impl/TezUmbilical.java
+++ /dev/null
@@ -1,36 +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.engine.newapi.impl;
-
-import java.io.IOException;
-import java.util.Collection;
-
-import org.apache.tez.dag.records.TezTaskAttemptID;
-
-public interface TezUmbilical {
-
-  public void addEvents(Collection<TezEvent> events);
-
-  public void signalFatalError(TezTaskAttemptID taskAttemptID,
-      String diagnostics,
-      EventMetaData sourceInfo);
-
-  public boolean canCommit(TezTaskAttemptID taskAttemptID) throws IOException;
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/newruntime/LogicalIOProcessorRuntimeTask.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/newruntime/LogicalIOProcessorRuntimeTask.java b/tez-engine/src/main/java/org/apache/tez/engine/newruntime/LogicalIOProcessorRuntimeTask.java
index eb055b6..77299de 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/newruntime/LogicalIOProcessorRuntimeTask.java
+++ b/tez-engine/src/main/java/org/apache/tez/engine/newruntime/LogicalIOProcessorRuntimeTask.java
@@ -38,6 +38,16 @@ import org.apache.hadoop.util.StringUtils;
 import org.apache.tez.dag.api.ProcessorDescriptor;
 import org.apache.tez.dag.api.TezUncheckedException;
 import org.apache.tez.dag.records.TezTaskAttemptID;
+import org.apache.tez.engine.api.impl.EventMetaData;
+import org.apache.tez.engine.api.impl.InputSpec;
+import org.apache.tez.engine.api.impl.OutputSpec;
+import org.apache.tez.engine.api.impl.TaskSpec;
+import org.apache.tez.engine.api.impl.TezEvent;
+import org.apache.tez.engine.api.impl.TezInputContextImpl;
+import org.apache.tez.engine.api.impl.TezOutputContextImpl;
+import org.apache.tez.engine.api.impl.TezProcessorContextImpl;
+import org.apache.tez.engine.api.impl.TezUmbilical;
+import org.apache.tez.engine.api.impl.EventMetaData.EventProducerConsumerType;
 import org.apache.tez.engine.common.security.JobTokenIdentifier;
 import org.apache.tez.engine.newapi.Event;
 import org.apache.tez.engine.newapi.Input;
@@ -49,16 +59,6 @@ import org.apache.tez.engine.newapi.Processor;
 import org.apache.tez.engine.newapi.TezInputContext;
 import org.apache.tez.engine.newapi.TezOutputContext;
 import org.apache.tez.engine.newapi.TezProcessorContext;
-import org.apache.tez.engine.newapi.impl.EventMetaData;
-import org.apache.tez.engine.newapi.impl.EventMetaData.EventProducerConsumerType;
-import org.apache.tez.engine.newapi.impl.InputSpec;
-import org.apache.tez.engine.newapi.impl.OutputSpec;
-import org.apache.tez.engine.newapi.impl.TaskSpec;
-import org.apache.tez.engine.newapi.impl.TezEvent;
-import org.apache.tez.engine.newapi.impl.TezInputContextImpl;
-import org.apache.tez.engine.newapi.impl.TezOutputContextImpl;
-import org.apache.tez.engine.newapi.impl.TezProcessorContextImpl;
-import org.apache.tez.engine.newapi.impl.TezUmbilical;
 import org.apache.tez.engine.shuffle.common.ShuffleUtils;
 
 import com.google.common.base.Preconditions;

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/newruntime/RuntimeTask.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/newruntime/RuntimeTask.java b/tez-engine/src/main/java/org/apache/tez/engine/newruntime/RuntimeTask.java
index ee6cde8..22cbc7c 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/newruntime/RuntimeTask.java
+++ b/tez-engine/src/main/java/org/apache/tez/engine/newruntime/RuntimeTask.java
@@ -25,9 +25,9 @@ import java.util.concurrent.atomic.AtomicInteger;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.tez.common.counters.TezCounters;
 import org.apache.tez.dag.records.TezTaskAttemptID;
-import org.apache.tez.engine.newapi.impl.TaskSpec;
-import org.apache.tez.engine.newapi.impl.TezEvent;
-import org.apache.tez.engine.newapi.impl.TezUmbilical;
+import org.apache.tez.engine.api.impl.TaskSpec;
+import org.apache.tez.engine.api.impl.TezEvent;
+import org.apache.tez.engine.api.impl.TezUmbilical;
 
 public abstract class RuntimeTask {
 

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/runtime/RuntimeUtils.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/runtime/RuntimeUtils.java b/tez-engine/src/main/java/org/apache/tez/engine/runtime/RuntimeUtils.java
deleted file mode 100644
index c673d16..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/runtime/RuntimeUtils.java
+++ /dev/null
@@ -1,164 +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.engine.runtime;
-
-import java.lang.reflect.Constructor;
-import java.util.Map;
-import java.util.concurrent.ConcurrentHashMap;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.tez.common.InputSpec;
-import org.apache.tez.common.OutputSpec;
-import org.apache.tez.common.TezEngineTaskContext;
-import org.apache.tez.engine.api.Input;
-import org.apache.tez.engine.api.Output;
-import org.apache.tez.engine.api.Processor;
-import org.apache.tez.engine.task.RuntimeTask;
-
-public class RuntimeUtils {
-
-  private static final Log LOG = LogFactory.getLog(RuntimeUtils.class);
-
-  private static final Class<?>[] CONTEXT_ARRAY =
-      new Class[] { TezEngineTaskContext.class };
-  private static final Class<?>[] CONTEXT_INT_ARRAY =
-      new Class[] { TezEngineTaskContext.class, Integer.TYPE };
-  private static final Map<Class<?>, Constructor<?>> CONSTRUCTOR_CACHE =
-    new ConcurrentHashMap<Class<?>, Constructor<?>>();
-
-  @SuppressWarnings("unchecked")
-  public static <T> T getNewInstance(Class<T> theClass,
-      TezEngineTaskContext context) {
-    T result;
-    try {
-      Constructor<T> meth = (Constructor<T>) CONSTRUCTOR_CACHE.get(theClass);
-      if (meth == null) {
-        meth = theClass.getDeclaredConstructor(CONTEXT_ARRAY);
-        meth.setAccessible(true);
-        CONSTRUCTOR_CACHE.put(theClass, meth);
-      }
-      result = meth.newInstance(context);
-    } catch (Exception e) {
-      throw new RuntimeException(e);
-    }
-    return result;
-  }
-  
-  @SuppressWarnings("unchecked")
-  public static <T> T getNewInputInstance(Class<T> theClass,
-      TezEngineTaskContext context, int index) {
-    T result;
-    try {
-      Constructor<T> meth = (Constructor<T>) CONSTRUCTOR_CACHE.get(theClass);
-      if (meth == null) {
-        meth = theClass.getDeclaredConstructor(CONTEXT_INT_ARRAY);
-        meth.setAccessible(true);
-        CONSTRUCTOR_CACHE.put(theClass, meth);
-      }
-      result = meth.newInstance(context, index);
-    } catch (Exception e) {
-      throw new RuntimeException(e);
-    }
-    return result;
-  }
-
-  public static RuntimeTask createRuntimeTask(
-      TezEngineTaskContext taskContext) {
-    LOG.info("Creating a runtime task from TaskContext"
-        + ", Processor: " + taskContext.getProcessorName()
-        + ", InputCount=" + taskContext.getInputSpecList().size()
-        + ", OutputCount=" + taskContext.getOutputSpecList().size());
-
-    RuntimeTask t = null;
-    try {
-      Class<?> processorClazz =
-          Class.forName(taskContext.getProcessorName());
-
-      Processor processor = (Processor) getNewInstance(
-          processorClazz, taskContext);
-
-      Input[] inputs;
-      Output[] outputs;
-      if (taskContext.getInputSpecList().isEmpty()) {
-        if (LOG.isDebugEnabled()) {
-          LOG.debug("Initializing task with 0 inputs");
-        }
-        inputs = new Input[0];
-      } else {
-        int iSpecCount = taskContext.getInputSpecList().size();
-        inputs = new Input[iSpecCount];
-        for (int i = 0; i < iSpecCount; ++i) {
-          InputSpec inSpec = taskContext.getInputSpecList().get(i);
-          if (LOG.isDebugEnabled()) {
-            LOG.debug("Using Input"
-                + ", index=" + i
-                + ", inputClass=" + inSpec.getInputClassName());
-          }
-          Class<?> inputClazz = Class.forName(inSpec.getInputClassName());
-          Input input = (Input) getNewInputInstance(inputClazz, taskContext, i);
-          inputs[i] = input;
-        }
-      }
-      if (taskContext.getOutputSpecList().isEmpty()) {
-        if (LOG.isDebugEnabled()) {
-          LOG.debug("Initializing task with 0 outputs");
-        }
-        outputs = new Output[0];
-      } else {
-        int oSpecCount = taskContext.getOutputSpecList().size();
-        outputs = new Output[oSpecCount];
-        for (int i = 0; i < oSpecCount; ++i) {
-          OutputSpec outSpec = taskContext.getOutputSpecList().get(i);
-          if (LOG.isDebugEnabled()) {
-            LOG.debug("Using Output"
-                + ", index=" + i
-                + ", output=" + outSpec.getOutputClassName());
-          }
-          Class<?> outputClazz = Class.forName(outSpec.getOutputClassName());
-          Output output = (Output) getNewInstance(outputClazz, taskContext);
-          outputs[i] = output;
-        }
-      }
-      t = createRuntime(taskContext, processor, inputs, outputs);
-    } catch (ClassNotFoundException e) {
-      throw new RuntimeException("Unable to initialize RuntimeTask, context="
-          + taskContext, e);
-    }
-    return t;
-  }
-
-  private static RuntimeTask createRuntime(TezEngineTaskContext taskContext,
-      Processor processor, Input[] inputs, Output[] outputs) {
-    try {
-      // TODO Change this to use getNewInstance
-      Class<?> runtimeClazz = Class.forName(taskContext.getRuntimeName());
-      Constructor<?> ctor = runtimeClazz.getConstructor(
-          TezEngineTaskContext.class, Processor.class, Input[].class,
-          Output[].class);
-      ctor.setAccessible(true);
-      return (RuntimeTask) ctor.newInstance(taskContext, processor, inputs, outputs);
-    } catch (ClassNotFoundException e) {
-      throw new RuntimeException("Unable to load runtimeClass: "
-          + taskContext.getRuntimeName(), e);
-    } catch (Exception e) {
-      throw new RuntimeException(e);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/shuffle/common/DiskFetchedInput.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/shuffle/common/DiskFetchedInput.java b/tez-engine/src/main/java/org/apache/tez/engine/shuffle/common/DiskFetchedInput.java
index e9bfe36..531e460 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/shuffle/common/DiskFetchedInput.java
+++ b/tez-engine/src/main/java/org/apache/tez/engine/shuffle/common/DiskFetchedInput.java
@@ -29,7 +29,7 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.LocalDirAllocator;
 import org.apache.hadoop.fs.Path;
 import org.apache.tez.engine.common.InputAttemptIdentifier;
-import org.apache.tez.engine.common.task.local.newoutput.TezTaskOutputFiles;
+import org.apache.tez.engine.common.task.local.output.TezTaskOutputFiles;
 
 import com.google.common.base.Preconditions;
 


[23/50] [abbrv] TEZ-443. Merge tez-dag-api and tez-engine-api into a single module - tez-api (part of TEZ-398). (sseth)

Posted by ss...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/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
new file mode 100644
index 0000000..9cb602c
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/dag/api/DAG.java
@@ -0,0 +1,377 @@
+/**
+ * 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.api;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Stack;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.api.records.LocalResource;
+import org.apache.hadoop.yarn.api.records.LocalResourceType;
+import org.apache.hadoop.yarn.api.records.Resource;
+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.VertexLocationHint.TaskLocationHint;
+import org.apache.tez.dag.api.records.DAGProtos.ConfigurationProto;
+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.PlanKeyValuePair;
+import org.apache.tez.dag.api.records.DAGProtos.PlanLocalResource;
+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.VertexPlan;
+
+
+public class DAG { // FIXME rename to Topology
+  final List<Vertex> vertices;
+  final List<Edge> edges;
+  final String name;
+
+  public DAG(String name) {
+    this.vertices = new ArrayList<Vertex>();
+    this.edges = new ArrayList<Edge>();
+    this.name = name;
+  }
+
+  public synchronized DAG addVertex(Vertex vertex) {
+    if (vertices.contains(vertex)) {
+      throw new IllegalArgumentException(
+          "Vertex " + vertex + " already defined!");
+    }
+    vertices.add(vertex);
+    return this;
+  }
+
+  @Private
+  public synchronized List<Vertex> getVertices() {
+    return Collections.unmodifiableList(this.vertices);
+  }
+
+  public synchronized DAG addEdge(Edge edge) {
+    // Sanity checks
+    if (!vertices.contains(edge.getInputVertex())) {
+      throw new IllegalArgumentException(
+          "Input vertex " + edge.getInputVertex() + " doesn't exist!");
+    }
+    if (!vertices.contains(edge.getOutputVertex())) {
+      throw new IllegalArgumentException(
+          "Output vertex " + edge.getOutputVertex() + " doesn't exist!");
+    }
+    if (edges.contains(edge)) {
+      throw new IllegalArgumentException(
+          "Edge " + edge + " already defined!");
+    }
+
+    // Inform the vertices
+    edge.getInputVertex().addOutputVertex(edge.getOutputVertex(), edge.getId());
+    edge.getOutputVertex().addInputVertex(edge.getInputVertex(), edge.getId());
+
+    edges.add(edge);
+    return this;
+  }
+
+  public String getName() {
+    return this.name;
+  }
+
+  // AnnotatedVertex is used by verify()
+  private static class AnnotatedVertex {
+    Vertex v;
+
+    int index; //for Tarjan's algorithm
+    int lowlink; //for Tarjan's algorithm
+    boolean onstack; //for Tarjan's algorithm
+
+    int outDegree;
+
+    private AnnotatedVertex(Vertex v){
+       this.v = v;
+       index = -1;
+       lowlink = -1;
+       outDegree = 0;
+    }
+  }
+
+  // verify()
+  //
+  // Default rules
+  //   Illegal:
+  //     - duplicate vertex id
+  //     - cycles
+  //
+  //   Ok:
+  //     - orphaned vertex.  Occurs in map-only
+  //     - islands.  Occurs if job has unrelated workflows.
+  //
+  //   Not yet categorized:
+  //     - orphaned vertex in DAG of >1 vertex.  Could be unrelated map-only job.
+  //     - v1->v2 via two edges.  perhaps some self-join job would use this?
+  //
+  // "restricted" mode:
+  //   In short term, the supported DAGs are limited. Call with restricted=true for these verifications.
+  //   Illegal:
+  //     - any vertex with more than one input or output edge. (n-ary input, n-ary merge)
+  public void verify() throws IllegalStateException {
+    verify(true);
+  }
+
+  public void verify(boolean restricted) throws IllegalStateException  {
+    if (vertices.isEmpty()) {
+      throw new IllegalStateException("Invalid dag containing 0 vertices");
+    }
+
+    Map<Vertex, List<Edge>> edgeMap = new HashMap<Vertex, List<Edge>>();
+    for(Edge e : edges){
+      Vertex inputVertex = e.getInputVertex();
+      List<Edge> edgeList = edgeMap.get(inputVertex);
+      if(edgeList == null){
+        edgeList = new ArrayList<Edge>();
+        edgeMap.put(inputVertex, edgeList);
+      }
+      edgeList.add(e);
+    }
+
+    // check for valid vertices, duplicate vertex names,
+    // and prepare for cycle detection
+    Map<String, AnnotatedVertex> vertexMap = new HashMap<String, AnnotatedVertex>();
+    for(Vertex v : vertices){
+      if(vertexMap.containsKey(v.getVertexName())){
+         throw new IllegalStateException("DAG contains multiple vertices"
+             + " with name: " + v.getVertexName());
+      }
+      vertexMap.put(v.getVertexName(), new AnnotatedVertex(v));
+    }
+
+    detectCycles(edgeMap, vertexMap);
+
+    if(restricted){
+      for(Edge e : edges){
+        vertexMap.get(e.getInputVertex().getVertexName()).outDegree++;
+        if (e.getEdgeProperty().getDataMovementType() !=
+            DataMovementType.SCATTER_GATHER) {
+          throw new IllegalStateException(
+              "Unsupported connection pattern on edge. " + e);
+        }
+        if (e.getEdgeProperty().getDataSourceType() !=
+            DataSourceType.PERSISTED) {
+          throw new IllegalStateException(
+              "Unsupported source type on edge. " + e);
+        }
+        if (e.getEdgeProperty().getSchedulingType() !=
+            SchedulingType.SEQUENTIAL) {
+          throw new IllegalStateException(
+              "Unsupported scheduling type on edge. " + e);
+        }
+      }
+      for(AnnotatedVertex av: vertexMap.values()){
+        if (av.outDegree > 1) {
+          throw new IllegalStateException("Vertex has outDegree>1: "
+              + av.v.getVertexName());
+        }
+      }
+    }
+  }
+
+  // Adaptation of Tarjan's algorithm for connected components.
+  // http://en.wikipedia.org/wiki/Tarjan%27s_strongly_connected_components_algorithm
+  private void detectCycles(Map<Vertex, List<Edge>> edgeMap, Map<String, AnnotatedVertex> vertexMap)
+      throws IllegalStateException{
+    Integer nextIndex = 0; // boxed integer so it is passed by reference.
+    Stack<AnnotatedVertex> stack = new Stack<DAG.AnnotatedVertex>();
+    for(AnnotatedVertex av: vertexMap.values()){
+      if(av.index == -1){
+        assert stack.empty();
+        strongConnect(av, vertexMap, edgeMap, stack, nextIndex);
+      }
+    }
+  }
+
+  // part of Tarjan's algorithm for connected components.
+  // http://en.wikipedia.org/wiki/Tarjan%27s_strongly_connected_components_algorithm
+  private void strongConnect(
+          AnnotatedVertex av,
+          Map<String, AnnotatedVertex> vertexMap,
+          Map<Vertex, List<Edge>> edgeMap,
+          Stack<AnnotatedVertex> stack, Integer nextIndex) throws IllegalStateException{
+    av.index = nextIndex;
+    av.lowlink = nextIndex;
+    nextIndex++;
+    stack.push(av);
+    av.onstack = true;
+
+    List<Edge> edges = edgeMap.get(av.v);
+    if(edges != null){
+      for(Edge e : edgeMap.get(av.v)){
+        AnnotatedVertex outVertex = vertexMap.get(e.getOutputVertex().getVertexName());
+        if(outVertex.index == -1){
+          strongConnect(outVertex, vertexMap, edgeMap, stack, nextIndex);
+          av.lowlink = Math.min(av.lowlink, outVertex.lowlink);
+        }
+        else if(outVertex.onstack){
+          // strongly connected component detected, but we will wait till later so that the full cycle can be displayed.
+          // update lowlink in case outputVertex should be considered the root of this component.
+          av.lowlink = Math.min(av.lowlink, outVertex.index);
+        }
+      }
+    }
+
+    if(av.lowlink == av.index ){
+       AnnotatedVertex pop = stack.pop();
+       pop.onstack = false;
+       if(pop != av){
+         // there was something on the stack other than this "av".
+         // this indicates there is a scc/cycle. It comprises all nodes from top of stack to "av"
+         StringBuilder message = new StringBuilder();
+         message.append(av.v.getVertexName() + " <- ");
+         for( ; pop != av; pop = stack.pop()){
+           message.append(pop.v.getVertexName() + " <- ");
+           pop.onstack = false;
+         }
+         message.append(av.v.getVertexName());
+         throw new IllegalStateException("DAG contains a cycle: " + message);
+       }
+    }
+  }
+
+
+  // create protobuf message describing DAG
+  @Private
+  public DAGPlan createDag(Configuration dagConf) {
+    verify(true);
+
+    DAGPlan.Builder dagBuilder = DAGPlan.newBuilder();
+
+    dagBuilder.setName(this.name);
+
+    for(Vertex vertex : vertices){
+      VertexPlan.Builder vertexBuilder = VertexPlan.newBuilder();
+      vertexBuilder.setName(vertex.getVertexName());
+      vertexBuilder.setType(PlanVertexType.NORMAL); // vertex type is implicitly NORMAL until  TEZ-46.
+      vertexBuilder.setProcessorDescriptor(DagTypeConverters
+          .convertToDAGPlan(vertex.getProcessorDescriptor()));
+
+      //task config
+      PlanTaskConfiguration.Builder taskConfigBuilder = PlanTaskConfiguration.newBuilder();
+      Resource resource = vertex.getTaskResource();
+      taskConfigBuilder.setNumTasks(vertex.getParallelism());
+      taskConfigBuilder.setMemoryMb(resource.getMemory());
+      taskConfigBuilder.setVirtualCores(resource.getVirtualCores());
+      taskConfigBuilder.setJavaOpts(vertex.getJavaOpts());
+
+      taskConfigBuilder.setTaskModule(vertex.getVertexName());
+      PlanLocalResource.Builder localResourcesBuilder = PlanLocalResource.newBuilder();
+      Map<String,LocalResource> lrs = vertex.getTaskLocalResources();
+      for(Entry<String, LocalResource> entry : lrs.entrySet()){
+        String key = entry.getKey();
+        LocalResource lr = entry.getValue();
+        localResourcesBuilder.setName(key);
+        localResourcesBuilder.setUri(
+            DagTypeConverters.convertToDAGPlan(lr.getResource()));
+        localResourcesBuilder.setSize(lr.getSize());
+        localResourcesBuilder.setTimeStamp(lr.getTimestamp());
+        localResourcesBuilder.setType(
+            DagTypeConverters.convertToDAGPlan(lr.getType()));
+        localResourcesBuilder.setVisibility(
+            DagTypeConverters.convertToDAGPlan(lr.getVisibility()));
+        if(lr.getType() == LocalResourceType.PATTERN){
+          if (lr.getPattern() == null || lr.getPattern().isEmpty()) {
+            throw new TezUncheckedException("LocalResource type set to pattern"
+                + " but pattern is null or empty");
+          }
+          localResourcesBuilder.setPattern(lr.getPattern());
+        }
+        taskConfigBuilder.addLocalResource(localResourcesBuilder);
+      }
+
+      if(vertex.getTaskEnvironment() != null){
+        for(String key : vertex.getTaskEnvironment().keySet()){
+          PlanKeyValuePair.Builder envSettingBuilder = PlanKeyValuePair.newBuilder();
+          envSettingBuilder.setKey(key);
+          envSettingBuilder.setValue(vertex.getTaskEnvironment().get(key));
+          taskConfigBuilder.addEnvironmentSetting(envSettingBuilder);
+        }
+      }
+
+      if(vertex.getTaskLocationsHint() != null ){
+        if(vertex.getTaskLocationsHint().getTaskLocationHints() != null){
+          for(TaskLocationHint hint : vertex.getTaskLocationsHint().getTaskLocationHints()){
+            PlanTaskLocationHint.Builder taskLocationHintBuilder = PlanTaskLocationHint.newBuilder();
+
+            if(hint.getDataLocalHosts() != null){
+              taskLocationHintBuilder.addAllHost(hint.getDataLocalHosts());
+            }
+            if(hint.getRacks() != null){
+              taskLocationHintBuilder.addAllRack(hint.getRacks());
+            }
+
+            vertexBuilder.addTaskLocationHint(taskLocationHintBuilder);
+          }
+        }
+      }
+
+      for(String inEdgeId : vertex.getInputEdgeIds()){
+        vertexBuilder.addInEdgeId(inEdgeId);
+      }
+
+      for(String outEdgeId : vertex.getOutputEdgeIds()){
+        vertexBuilder.addOutEdgeId(outEdgeId);
+      }
+
+      vertexBuilder.setTaskConfig(taskConfigBuilder);
+      dagBuilder.addVertex(vertexBuilder);
+    }
+
+    for(Edge edge : edges){
+      EdgePlan.Builder edgeBuilder = EdgePlan.newBuilder();
+      edgeBuilder.setId(edge.getId());
+      edgeBuilder.setInputVertexName(edge.getInputVertex().getVertexName());
+      edgeBuilder.setOutputVertexName(edge.getOutputVertex().getVertexName());
+      edgeBuilder.setDataMovementType(DagTypeConverters.convertToDAGPlan(edge.getEdgeProperty().getDataMovementType()));
+      edgeBuilder.setDataSourceType(DagTypeConverters.convertToDAGPlan(edge.getEdgeProperty().getDataSourceType()));
+      edgeBuilder.setSchedulingType(DagTypeConverters.convertToDAGPlan(edge.getEdgeProperty().getSchedulingType()));
+      edgeBuilder.setEdgeSource(DagTypeConverters.convertToDAGPlan(edge.getEdgeProperty().getEdgeSource()));
+      edgeBuilder.setEdgeDestination(DagTypeConverters.convertToDAGPlan(edge.getEdgeProperty().getEdgeDestination()));
+      dagBuilder.addEdge(edgeBuilder);
+    }
+
+    if(dagConf != null) {
+      Iterator<Entry<String, String>> iter = dagConf.iterator();
+      ConfigurationProto.Builder confProtoBuilder =
+          ConfigurationProto.newBuilder();
+      while (iter.hasNext()) {
+        Entry<String, String> entry = iter.next();
+        PlanKeyValuePair.Builder kvp = PlanKeyValuePair.newBuilder();
+        kvp.setKey(entry.getKey());
+        kvp.setValue(entry.getValue());
+        confProtoBuilder.addConfKeyValues(kvp);
+      }
+      dagBuilder.setDagKeyValues(confProtoBuilder);
+    }
+
+    return dagBuilder.build();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/src/main/java/org/apache/tez/dag/api/DagTypeConverters.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/DagTypeConverters.java b/tez-api/src/main/java/org/apache/tez/dag/api/DagTypeConverters.java
new file mode 100644
index 0000000..1fd78f1
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/dag/api/DagTypeConverters.java
@@ -0,0 +1,278 @@
+/**
+ * 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.api;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.yarn.api.records.LocalResource;
+import org.apache.hadoop.yarn.api.records.LocalResourceType;
+import org.apache.hadoop.yarn.api.records.LocalResourceVisibility;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.api.records.URL;
+import org.apache.hadoop.yarn.api.records.impl.pb.LocalResourcePBImpl;
+import org.apache.hadoop.yarn.util.ConverterUtils;
+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.VertexLocationHint.TaskLocationHint;
+import org.apache.tez.dag.api.records.DAGProtos.ConfigurationProto;
+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.PlanKeyValuePair;
+import org.apache.tez.dag.api.records.DAGProtos.PlanLocalResource;
+import org.apache.tez.dag.api.records.DAGProtos.PlanLocalResourceType;
+import org.apache.tez.dag.api.records.DAGProtos.PlanLocalResourceVisibility;
+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.TezEntityDescriptorProto;
+
+import com.google.protobuf.ByteString;
+
+
+public class DagTypeConverters {
+  
+  public static PlanLocalResourceVisibility convertToDAGPlan(LocalResourceVisibility visibility){
+    switch(visibility){
+      case PUBLIC : return PlanLocalResourceVisibility.PUBLIC;  
+      case PRIVATE : return PlanLocalResourceVisibility.PRIVATE;
+      case APPLICATION : return PlanLocalResourceVisibility.APPLICATION;
+      default : throw new RuntimeException("unknown 'visibility': " + visibility);
+    }
+  }
+  
+  public static LocalResourceVisibility convertFromDAGPlan(PlanLocalResourceVisibility visibility){
+    switch(visibility){
+      case PUBLIC : return LocalResourceVisibility.PUBLIC;  
+      case PRIVATE : return LocalResourceVisibility.PRIVATE;
+      case APPLICATION : return LocalResourceVisibility.APPLICATION;
+      default : throw new RuntimeException("unknown 'visibility': " + visibility);
+    }
+  }
+  
+  public static PlanEdgeDataSourceType convertToDAGPlan(DataSourceType sourceType){
+    switch(sourceType){
+      case PERSISTED : return PlanEdgeDataSourceType.PERSISTED;  
+      case PERSISTED_RELIABLE : return PlanEdgeDataSourceType.PERSISTED_RELIABLE;
+      case EPHEMERAL :  return PlanEdgeDataSourceType.EPHEMERAL;
+      default : throw new RuntimeException("unknown 'dataSourceType': " + sourceType);
+    }
+  }
+  
+  public static DataSourceType convertFromDAGPlan(PlanEdgeDataSourceType sourceType){
+    switch(sourceType){
+      case PERSISTED : return DataSourceType.PERSISTED;  
+      case PERSISTED_RELIABLE : return DataSourceType.PERSISTED_RELIABLE;
+      case EPHEMERAL :  return DataSourceType.EPHEMERAL;
+      default : throw new RuntimeException("unknown 'dataSourceType': " + sourceType);
+    }
+  }
+  
+  public static PlanEdgeDataMovementType convertToDAGPlan(DataMovementType type){
+    switch(type){
+      case ONE_TO_ONE : return PlanEdgeDataMovementType.ONE_TO_ONE;  
+      case BROADCAST : return PlanEdgeDataMovementType.BROADCAST;
+      case SCATTER_GATHER : return PlanEdgeDataMovementType.SCATTER_GATHER;
+      default : throw new RuntimeException("unknown 'dataMovementType': " + type);
+    }
+  }
+  
+  public static DataMovementType convertFromDAGPlan(PlanEdgeDataMovementType type){
+    switch(type){
+      case ONE_TO_ONE : return DataMovementType.ONE_TO_ONE;  
+      case BROADCAST : return DataMovementType.BROADCAST;
+      case SCATTER_GATHER : return DataMovementType.SCATTER_GATHER;
+      default : throw new IllegalArgumentException("unknown 'dataMovementType': " + type);
+    }
+  }
+  
+  public static PlanEdgeSchedulingType convertToDAGPlan(SchedulingType type){
+    switch(type){
+      case SEQUENTIAL : return PlanEdgeSchedulingType.SEQUENTIAL;  
+      case CONCURRENT : return PlanEdgeSchedulingType.CONCURRENT;
+      default : throw new RuntimeException("unknown 'SchedulingType': " + type);
+    }
+  }
+  
+  public static SchedulingType convertFromDAGPlan(PlanEdgeSchedulingType type){
+    switch(type){
+      case SEQUENTIAL : return SchedulingType.SEQUENTIAL;  
+      case CONCURRENT : return SchedulingType.CONCURRENT;
+      default : throw new IllegalArgumentException("unknown 'SchedulingType': " + type);
+    }
+  }
+  
+  public static PlanLocalResourceType convertToDAGPlan(LocalResourceType type) {
+    switch(type){
+    case ARCHIVE : return PlanLocalResourceType.ARCHIVE;
+    case FILE : return PlanLocalResourceType.FILE;
+    case PATTERN : return PlanLocalResourceType.PATTERN;
+    default : throw new IllegalArgumentException("unknown 'type': " + type);
+    }
+  }
+  
+  public static LocalResourceType convertFromDAGPlan(PlanLocalResourceType type) {
+    switch(type){
+    case ARCHIVE : return LocalResourceType.ARCHIVE;
+    case FILE : return LocalResourceType.FILE;
+    case PATTERN : return LocalResourceType.PATTERN;
+    default : throw new IllegalArgumentException("unknown 'type': " + type);
+    }
+  }
+
+  public static VertexLocationHint convertFromDAGPlan(
+      List<PlanTaskLocationHint> locationHints) {
+
+    List<TaskLocationHint> outputList = new ArrayList<TaskLocationHint>();  
+    
+    for(PlanTaskLocationHint inputHint : locationHints){
+      TaskLocationHint outputHint = new TaskLocationHint(
+          new HashSet<String>(inputHint.getHostList()),
+          new HashSet<String>(inputHint.getRackList()));
+      outputList.add(outputHint);
+    }
+    return new VertexLocationHint(outputList.size(), outputList);
+  }
+
+  // notes re HDFS URL handling:
+  //   Resource URLs in the protobuf message are strings of the form hdfs://host:port/path 
+  //   org.apache.hadoop.fs.Path.Path  is actually a URI type that allows any scheme
+  //   org.apache.hadoop.yarn.api.records.URL is a URL type used by YARN.
+  //   java.net.URL cannot be used out of the box as it rejects unknown schemes such as HDFS.
+  
+  public static String convertToDAGPlan(URL resource) {
+    // see above notes on HDFS URL handling
+    String out = resource.getScheme() + "://" + resource.getHost() + ":" + resource.getPort() 
+        + resource.getFile();
+    return out;
+  }
+
+  public static Map<String, LocalResource> createLocalResourceMapFromDAGPlan(
+      List<PlanLocalResource> localResourcesList) {
+    Map<String, LocalResource> map = new HashMap<String, LocalResource>();
+    for(PlanLocalResource res : localResourcesList){
+      LocalResource r = new LocalResourcePBImpl();
+      
+      //NOTE: have to check every optional field in protobuf generated classes for existence before accessing
+      //else we will receive a default value back, eg ""
+      if(res.hasPattern()){
+        r.setPattern(res.getPattern());
+      }
+      r.setResource(ConverterUtils.getYarnUrlFromPath(new Path(res.getUri())));  // see above notes on HDFS URL handling
+      r.setSize(res.getSize());
+      r.setTimestamp(res.getTimeStamp());
+      r.setType(DagTypeConverters.convertFromDAGPlan(res.getType()));
+      r.setVisibility(DagTypeConverters.convertFromDAGPlan(res.getVisibility()));
+      map.put(res.getName(), r);
+    }
+    return map;
+  }
+
+  public static Map<String, String> createEnvironmentMapFromDAGPlan(
+      List<PlanKeyValuePair> environmentSettingList) {  
+      
+    Map<String, String> map = new HashMap<String, String>();
+    for(PlanKeyValuePair setting : environmentSettingList){
+      map.put(setting.getKey(), setting.getValue());
+    }
+    
+    return map;
+  }
+  
+  public static Map<String, EdgePlan> createEdgePlanMapFromDAGPlan(List<EdgePlan> edgeList){
+    Map<String, EdgePlan> edgePlanMap =
+        new HashMap<String, EdgePlan>();
+    for(EdgePlan edgePlanItem : edgeList){
+      edgePlanMap.put(edgePlanItem.getId(), edgePlanItem);
+    }
+    return edgePlanMap;
+  }
+  
+  public static EdgeProperty createEdgePropertyMapFromDAGPlan(EdgePlan edge) {
+    return new EdgeProperty(
+        convertFromDAGPlan(edge.getDataMovementType()),
+        convertFromDAGPlan(edge.getDataSourceType()),
+        convertFromDAGPlan(edge.getSchedulingType()),
+        convertOutputDescriptorFromDAGPlan(edge.getEdgeSource()),
+        convertInputDescriptorFromDAGPlan(edge.getEdgeDestination())
+    );
+  }
+
+  public static Resource createResourceRequestFromTaskConfig(
+      PlanTaskConfiguration taskConfig) {
+    return Resource.newInstance(taskConfig.getMemoryMb(), taskConfig.getVirtualCores());
+  }
+
+  public static Map<String, String> convertConfFromProto(
+      ConfigurationProto confProto) {
+    List<PlanKeyValuePair> settingList = confProto.getConfKeyValuesList();
+    Map<String, String> map = new HashMap<String, String>();
+    for(PlanKeyValuePair setting: settingList){
+      map.put(setting.getKey(), setting.getValue());
+    }
+    return map;
+  }
+
+  public static TezEntityDescriptorProto convertToDAGPlan(
+      TezEntityDescriptor descriptor) {
+    TezEntityDescriptorProto.Builder builder = TezEntityDescriptorProto
+        .newBuilder();
+    builder.setClassName(descriptor.getClassName());
+    if (descriptor.getUserPayload() != null) {
+      builder
+          .setUserPayload(ByteString.copyFrom(descriptor.getUserPayload()));
+    }
+    return builder.build();
+  }
+
+  public static InputDescriptor convertInputDescriptorFromDAGPlan(
+      TezEntityDescriptorProto proto) {
+    String className = proto.getClassName();
+    byte[] bb = null;
+    if (proto.hasUserPayload()) {
+      bb = proto.getUserPayload().toByteArray();
+    }
+    return new InputDescriptor(className).setUserPayload(bb);
+  }
+
+  public static OutputDescriptor convertOutputDescriptorFromDAGPlan(
+      TezEntityDescriptorProto proto) {
+    String className = proto.getClassName();
+    byte[] bb = null;
+    if (proto.hasUserPayload()) {
+      bb =  proto.getUserPayload().toByteArray();
+    }
+    return new OutputDescriptor(className).setUserPayload(bb);
+  }
+
+  public static ProcessorDescriptor convertProcessorDescriptorFromDAGPlan(
+      TezEntityDescriptorProto proto) {
+    String className = proto.getClassName();
+    byte[] bb = null;
+    if (proto.hasUserPayload()) {
+      bb = proto.getUserPayload().toByteArray();
+    }
+    return new ProcessorDescriptor(className).setUserPayload(bb);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/src/main/java/org/apache/tez/dag/api/Edge.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/Edge.java b/tez-api/src/main/java/org/apache/tez/dag/api/Edge.java
new file mode 100644
index 0000000..a893bc3
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/dag/api/Edge.java
@@ -0,0 +1,59 @@
+/**
+ * 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.api;
+
+public class Edge{
+  
+  private final Vertex inputVertex;
+  private final Vertex outputVertex;
+  private final EdgeProperty edgeProperty;
+    
+  public Edge(Vertex inputVertex, 
+               Vertex outputVertex, 
+               EdgeProperty edgeProperty) {
+    this.inputVertex = inputVertex;
+    this.outputVertex = outputVertex;
+    this.edgeProperty = edgeProperty;
+  }
+  
+  // RENAME to source and destination
+  public Vertex getInputVertex() {
+    return inputVertex;
+  }
+  
+  public Vertex getOutputVertex() {
+    return outputVertex;
+  }
+  
+  public EdgeProperty getEdgeProperty() {
+    return edgeProperty;
+  }
+  
+  /*
+   * Used to identify the edge in the configuration
+   */
+  public String getId() {
+    return String.valueOf(this.hashCode());
+  }
+ 
+  @Override
+  public String toString() {
+    return inputVertex + " -> " + outputVertex + " (" + edgeProperty + ")";
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/src/main/java/org/apache/tez/dag/api/EdgeProperty.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/EdgeProperty.java b/tez-api/src/main/java/org/apache/tez/dag/api/EdgeProperty.java
new file mode 100644
index 0000000..326d3d0
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/dag/api/EdgeProperty.java
@@ -0,0 +1,147 @@
+/**
+ * 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.api;
+
+public class EdgeProperty {
+  
+  /**
+   * Defines the manner of data movement between source and destination tasks.
+   * Determines which destination tasks have access to data produced on this
+   * edge by a source task. A destination task may choose to read any portion of
+   * the data available to it.
+   */
+  public enum DataMovementType {
+    /**
+     * Output on this edge produced by the i-th source task is available to the 
+     * i-th destination task.
+     */
+    ONE_TO_ONE,
+    /**
+     * Output on this edge produced by any source task is available to all
+     * destination tasks.
+     */
+    BROADCAST,
+    /**
+     * The i-th output on this edge produced by all source tasks is available to
+     * the same destination task. Source tasks scatter their outputs and they
+     * are gathered by designated destination tasks.
+     */
+    SCATTER_GATHER
+  }
+  
+  /**
+   * Determines the lifetime of the data produced on this edge by a source task.
+   */
+  public enum DataSourceType {
+    /**
+     * Data produced by the source is persisted and available even when the
+     * task is not running. The data may become unavailable and may cause the 
+     * source task to be re-executed.
+     */
+    PERSISTED,
+    /**
+     * Source data is stored reliably and will always be available
+     */
+    PERSISTED_RELIABLE,
+    /**
+     * Data produced by the source task is available only while the source task
+     * is running. This requires the destination task to run concurrently with 
+     * the source task.
+     */
+    EPHEMERAL
+  }
+  
+  /**
+   * Determines when the destination task is eligible to run, once the source  
+   * task is eligible to run.
+   */
+  public enum SchedulingType {
+    /**
+     * Destination task is eligible to run after one or more of its source tasks 
+     * have started or completed.
+     */
+    SEQUENTIAL,
+    /**
+     * Destination task must run concurrently with the source task
+     */
+    CONCURRENT
+  }
+  
+  DataMovementType dataMovementType;
+  DataSourceType dataSourceType;
+  SchedulingType schedulingType;
+  InputDescriptor inputDescriptor;
+  OutputDescriptor outputDescriptor;
+  
+  /**
+   * @param dataMovementType
+   * @param dataSourceType
+   * @param edgeSource
+   *          The {@link OutputDescriptor} that generates data on the edge.
+   * @param edgeDestination
+   *          The {@link InputDescriptor} which will consume data from the edge.
+   */
+  public EdgeProperty(DataMovementType dataMovementType, 
+                       DataSourceType dataSourceType,
+                       SchedulingType schedulingType,
+                       OutputDescriptor edgeSource,
+                       InputDescriptor edgeDestination) {
+    this.dataMovementType = dataMovementType;
+    this.dataSourceType = dataSourceType;
+    this.schedulingType = schedulingType;
+    this.inputDescriptor = edgeDestination;
+    this.outputDescriptor = edgeSource;
+  }
+  
+  public DataMovementType getDataMovementType() {
+    return dataMovementType;
+  }
+  
+  public DataSourceType getDataSourceType() {
+    return dataSourceType;
+  }
+  
+  public SchedulingType getSchedulingType() {
+    return schedulingType;
+  }
+  
+  /**
+   * Returns the {@link InputDescriptor} which will consume data from the edge.
+   * 
+   * @return
+   */
+  public InputDescriptor getEdgeDestination() {
+    return inputDescriptor;
+  }
+  
+  /**
+   * Returns the {@link OutputDescriptor} which produces data on the edge.
+   * 
+   * @return
+   */
+  public OutputDescriptor getEdgeSource() {
+    return outputDescriptor;
+  }
+  
+  @Override
+  public String toString() {
+    return "{ " + dataMovementType + " : " + inputDescriptor.getClassName()
+        + " >> " + dataSourceType + " >> " + outputDescriptor.getClassName() + " }";
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/src/main/java/org/apache/tez/dag/api/InputDescriptor.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/InputDescriptor.java b/tez-api/src/main/java/org/apache/tez/dag/api/InputDescriptor.java
new file mode 100644
index 0000000..dea9001
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/dag/api/InputDescriptor.java
@@ -0,0 +1,32 @@
+/**
+ * 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.api;
+
+public class InputDescriptor extends TezEntityDescriptor {
+
+  public InputDescriptor(String inputClassName) {
+    super(inputClassName);
+  }
+
+  @Override
+  public InputDescriptor setUserPayload(byte[] userPayload) {
+    this.userPayload = userPayload;
+    return this;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/src/main/java/org/apache/tez/dag/api/OutputDescriptor.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/OutputDescriptor.java b/tez-api/src/main/java/org/apache/tez/dag/api/OutputDescriptor.java
new file mode 100644
index 0000000..16fb9b1
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/dag/api/OutputDescriptor.java
@@ -0,0 +1,32 @@
+/**
+ * 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.api;
+
+public class OutputDescriptor extends TezEntityDescriptor {
+
+  public OutputDescriptor(String outputClassName) {
+    super(outputClassName);
+  }
+
+  @Override
+  public OutputDescriptor setUserPayload(byte[] userPayload) {
+    this.userPayload = userPayload;
+    return this;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/src/main/java/org/apache/tez/dag/api/ProcessorDescriptor.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/ProcessorDescriptor.java b/tez-api/src/main/java/org/apache/tez/dag/api/ProcessorDescriptor.java
new file mode 100644
index 0000000..092147d
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/dag/api/ProcessorDescriptor.java
@@ -0,0 +1,31 @@
+/**
+ * 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.api;
+
+public class ProcessorDescriptor extends TezEntityDescriptor {
+
+  public ProcessorDescriptor(String processorClassName) {
+    super(processorClassName);
+  }
+
+  public ProcessorDescriptor setUserPayload(byte[] userPayload) {
+    this.userPayload = userPayload;
+    return this;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java b/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java
new file mode 100644
index 0000000..7447974
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.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.api;
+
+import org.apache.hadoop.conf.Configuration;
+
+public class TezConfiguration extends Configuration {
+
+  public final static String TEZ_SITE_XML = "tez-site.xml";
+
+  static {
+    addDefaultResource(TEZ_SITE_XML);
+  }
+
+  public TezConfiguration() {
+    super();
+  }
+
+  public TezConfiguration(Configuration conf) {
+    super(conf);
+  }
+
+  public static final String TEZ_PREFIX = "tez.";
+  public static final String TEZ_AM_PREFIX = TEZ_PREFIX + "am.";
+  public static final String TEZ_TASK_PREFIX = TEZ_PREFIX + "task.";
+
+  public static final String TEZ_AM_STAGING_DIR = TEZ_PREFIX + "staging-dir";
+  public static final String TEZ_AM_STAGING_DIR_DEFAULT = "/tmp/tez/staging";
+
+  // TODO Should not be required once all tokens are handled via AppSubmissionContext
+  public static final String JOB_SUBMIT_DIR = TEZ_PREFIX + "jobSubmitDir";
+  public static final String APPLICATION_TOKENS_FILE = "appTokens";
+  public static final String TEZ_APPLICATION_MASTER_CLASS =
+      "org.apache.tez.dag.app.DAGAppMaster";
+
+  /** Root Logging level passed to the Tez app master.*/
+  public static final String TEZ_AM_LOG_LEVEL = TEZ_AM_PREFIX+"log.level";
+  public static final String TEZ_AM_LOG_LEVEL_DEFAULT = "INFO";
+
+  public static final String TEZ_AM_JAVA_OPTS = TEZ_AM_PREFIX
+      + "java.opts";
+  public static final String DEFAULT_TEZ_AM_JAVA_OPTS = " -Xmx1024m ";
+
+  public static final String TEZ_AM_CANCEL_DELEGATION_TOKEN = TEZ_AM_PREFIX +
+      "am.complete.cancel.delegation.tokens";
+  public static final boolean TEZ_AM_CANCEL_DELEGATION_TOKEN_DEFAULT = true;
+
+  public static final String TEZ_AM_TASK_LISTENER_THREAD_COUNT =
+      TEZ_AM_PREFIX + "task.listener.thread-count";
+  public static final int TEZ_AM_TASK_LISTENER_THREAD_COUNT_DEFAULT = 30;
+
+  public static final String TEZ_AM_CONTAINER_LISTENER_THREAD_COUNT =
+      TEZ_AM_PREFIX + "container.listener.thread-count";
+  public static final int TEZ_AM_CONTAINER_LISTENER_THREAD_COUNT_DEFAULT = 30;
+
+  // TODO Some of the DAG properties are job specific and not AM specific. Rename accordingly.
+  // TODO Are any of these node blacklisting properties required. (other than for MR compat)
+  public static final String TEZ_AM_MAX_TASK_FAILURES_PER_NODE = TEZ_AM_PREFIX
+      + "maxtaskfailures.per.node";
+  public static final int TEZ_AM_MAX_TASK_FAILURES_PER_NODE_DEFAULT = 3;
+
+  public static final String TEZ_AM_MAX_TASK_ATTEMPTS =
+      TEZ_AM_PREFIX + "max.task.attempts";
+  public static final int TEZ_AM_MAX_TASK_ATTEMPTS_DEFAULT = 4;
+
+  public static final String TEZ_AM_NODE_BLACKLISTING_ENABLED = TEZ_AM_PREFIX
+      + "node-blacklisting.enabled";
+  public static final boolean TEZ_AM_NODE_BLACKLISTING_ENABLED_DEFAULT = true;
+  public static final String TEZ_AM_NODE_BLACKLISTING_IGNORE_THRESHOLD = TEZ_AM_PREFIX
+      + "node-blacklisting.ignore-threshold-node-percent";
+  public static final int TEZ_AM_NODE_BLACKLISTING_IGNORE_THRESHOLD_DEFAULT = 33;
+
+  /** Number of threads to handle job client RPC requests.*/
+  public static final String TEZ_AM_CLIENT_THREAD_COUNT =
+      TEZ_AM_PREFIX + "client.am.thread-count";
+  public static final int TEZ_AM_CLIENT_THREAD_COUNT_DEFAULT = 1;
+  /**
+   * Range of ports that the AM can use when binding. Leave blank
+   * if you want all possible ports.
+   */
+  public static final String TEZ_AM_CLIENT_AM_PORT_RANGE =
+      TEZ_AM_PREFIX + "client.am.port-range";
+
+
+  public static final String TEZ_AM_RESOURCE_MEMORY_MB = TEZ_AM_PREFIX
+      + "resource.memory.mb";
+  public static final int TEZ_AM_RESOURCE_MEMORY_MB_DEFAULT = 1536;
+
+  public static final String TEZ_AM_RESOURCE_CPU_VCORES = TEZ_AM_PREFIX
+      + "resource.cpu.vcores";
+  public static final int TEZ_AM_RESOURCE_CPU_VCORES_DEFAULT = 1;
+
+  public static final String
+          TEZ_AM_SHUFFLE_VERTEX_MANAGER_MIN_SRC_FRACTION = TEZ_AM_PREFIX
+          + "shuffle-vertex-manager.min-src-fraction";
+  public static final float
+          TEZ_AM_SHUFFLE_VERTEX_MANAGER_MIN_SRC_FRACTION_DEFAULT = 0.25f;
+
+  public static final String
+          TEZ_AM_SHUFFLE_VERTEX_MANAGER_MAX_SRC_FRACTION = TEZ_AM_PREFIX
+          + "shuffle-vertex-manager.max-src-fraction";
+  public static final float
+          TEZ_AM_SHUFFLE_VERTEX_MANAGER_MAX_SRC_FRACTION_DEFAULT = 0.75f;
+
+  public static final String
+          TEZ_AM_SHUFFLE_VERTEX_MANAGER_ENABLE_AUTO_PARALLEL = TEZ_AM_PREFIX +
+          "shuffle-vertex-manager.enable.auto-parallel";
+  public static final boolean
+          TEZ_AM_SHUFFLE_VERTEX_MANAGER_ENABLE_AUTO_PARALLEL_DEFAULT = false;
+
+  public static final String
+          TEZ_AM_SHUFFLE_VERTEX_MANAGER_DESIRED_TASK_INPUT_SIZE = TEZ_AM_PREFIX +
+          "shuffle-vertex-manager.desired-task-input-size";
+  public static final long
+          TEZ_AM_SHUFFLE_VERTEX_MANAGER_DESIRED_TASK_INPUT_SIZE_DEFAULT =
+          1024*1024*100L;
+
+  public static final String
+          TEZ_AM_SHUFFLE_VERTEX_MANAGER_MIN_TASK_PARALLELISM = TEZ_AM_PREFIX +
+          "shuffle-vertex-manager.min-task-parallelism";
+  public static final int
+          TEZ_AM_SHUFFLE_VERTEX_MANAGER_MIN_TASK_PARALLELISM_DEFAULT = 1;
+
+  public static final String
+          TEZ_AM_SLOWSTART_DAG_SCHEDULER_MIN_SHUFFLE_RESOURCE_FRACTION = TEZ_AM_PREFIX
+          + "slowstart-dag-scheduler.min-resource-fraction";
+  public static final float
+          TEZ_AM_SLOWSTART_DAG_SCHEDULER_MIN_SHUFFLE_RESOURCE_FRACTION_DEFAULT = 0.5f;
+
+  public static final String TEZ_AM_AGGRESSIVE_SCHEDULING = TEZ_AM_PREFIX +
+      "aggressive.scheduling";
+  public static boolean TEZ_AM_AGGRESSIVE_SCHEDULING_DEFAULT = false;
+
+  /**
+   * The complete path to the serialized dag plan file
+   * <code>TEZ_AM_PLAN_PB_BINARY</code>. Used to make the plan available to
+   * individual tasks if needed. This will typically be a path in the job submit
+   * directory.
+   */
+  public static final String TEZ_AM_PLAN_REMOTE_PATH = TEZ_AM_PREFIX
+      + "dag-am-plan.remote.path";
+
+  public static final String TEZ_AM_RM_HEARTBEAT_INTERVAL_MS_MAX = TEZ_AM_PREFIX
+      + "am-rm.heartbeat.interval-ms.max";
+  public static final int TEZ_AM_RM_HEARTBEAT_INTERVAL_MS_MAX_DEFAULT = 1000;
+
+  public static final String TEZ_TASK_GET_TASK_SLEEP_INTERVAL_MS_MAX = TEZ_TASK_PREFIX
+      + "get-task.sleep.interval-ms.max";
+  public static final int TEZ_TASK_GET_TASK_SLEEP_INTERVAL_MS_MAX_DEFAULT = 500;
+
+  public static final String TEZ_TASK_AM_HEARTBEAT_INTERVAL_MS = TEZ_TASK_PREFIX
+      + "am.heartbeat.interval-ms.max";
+  public static final int TEZ_TASK_AM_HEARTBEAT_INTERVAL_MS_DEFAULT = 100;
+
+  public static final String TEZ_TASK_MAX_EVENTS_PER_HEARTBEAT = TEZ_TASK_PREFIX
+      + "max-events-per-heartbeat.max";
+  public static final int TEZ_TASK_MAX_EVENTS_PER_HEARTBEAT_DEFAULT = 100;
+
+  /**
+   * Configuration to specify whether container should be reused.
+   */
+  public static final String TEZ_AM_CONTAINER_REUSE_ENABLED = TEZ_AM_PREFIX
+      + "container.reuse.enabled";
+  public static final boolean TEZ_AM_CONTAINER_REUSE_ENABLED_DEFAULT = true;
+
+  /**
+   * Whether to reuse containers for rack local tasks. Active only if reuse is
+   * enabled.
+   */
+  public static final String TEZ_AM_CONTAINER_REUSE_RACK_FALLBACK_ENABLED = TEZ_AM_PREFIX
+      + "container.reuse.rack-fallback.enabled";
+  public static final boolean TEZ_AM_CONTAINER_REUSE_RACK_FALLBACK_ENABLED_DEFAULT = true;
+
+  /**
+   * Whether to reuse containers for non-local tasks. Active only if reuse is
+   * enabled.
+   */
+  public static final String TEZ_AM_CONTAINER_REUSE_NON_LOCAL_FALLBACK_ENABLED = TEZ_AM_PREFIX
+      + "container.reuse.non-local-fallback.enabled";
+  public static final boolean TEZ_AM_CONTAINER_REUSE_NON_LOCAL_FALLBACK_ENABLED_DEFAULT = false;
+
+  public static final String TEZ_AM_CONTAINER_REUSE_DELAY_ALLOCATION_MILLIS = TEZ_AM_PREFIX
+      + "container.reuse.delay-allocation-millis";
+  public static final long TEZ_AM_CONTAINER_REUSE_DELAY_ALLOCATION_MILLIS_DEFAULT = 3000l;
+
+  public static final String TEZ_PB_BINARY_CONF_NAME = "tez-conf.pb";
+  public static final String TEZ_PB_PLAN_BINARY_NAME = "tez-dag.pb";
+  public static final String TEZ_PB_PLAN_TEXT_NAME = "tez-dag.pb.txt";
+
+  /*
+   * Logger properties
+   */
+  public static final String TEZ_CONTAINER_LOG4J_PROPERTIES_FILE = "tez-container-log4j.properties";
+  public static final String TEZ_CONTAINER_LOGGER_NAME = "CLA";
+  public static final String TEZ_ROOT_LOGGER_NAME = "tez.root.logger";
+  public static final String TEZ_CONTAINER_LOG_FILE_NAME = "syslog";
+  public static final String TEZ_CONTAINER_ERR_FILE_NAME = "stderr";
+  public static final String TEZ_CONTAINER_OUT_FILE_NAME = "stdout";
+
+
+  public static final String TEZ_LIB_URIS =
+      TEZ_PREFIX + "lib.uris";
+
+  public static final String TEZ_APPLICATION_TYPE = "TEZ-MR*";
+
+  public static final String LOCAL_FRAMEWORK_NAME = "local-tez";
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/src/main/java/org/apache/tez/dag/api/TezConstants.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/TezConstants.java b/tez-api/src/main/java/org/apache/tez/dag/api/TezConstants.java
new file mode 100644
index 0000000..5463d65
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/dag/api/TezConstants.java
@@ -0,0 +1,29 @@
+/**
+ * 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.api;
+
+/**
+ * Specifies all constant values in Tez
+ */
+public class TezConstants {
+
+  // Env variable names
+  public static final String TEZ_AM_IS_SESSION_ENV = "TEZ_AM_IS_SESSION";
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/src/main/java/org/apache/tez/dag/api/TezEntityDescriptor.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/TezEntityDescriptor.java b/tez-api/src/main/java/org/apache/tez/dag/api/TezEntityDescriptor.java
new file mode 100644
index 0000000..9d4b2c4
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/dag/api/TezEntityDescriptor.java
@@ -0,0 +1,42 @@
+/**
+ * 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.api;
+
+public abstract class TezEntityDescriptor {
+
+  protected byte[] userPayload;
+  private String className;
+
+  public TezEntityDescriptor(String className) {
+    this.className = className;
+  }
+
+  public byte[] getUserPayload() {
+    return this.userPayload;
+  }
+
+  public TezEntityDescriptor setUserPayload(byte[] userPayload) {
+    this.userPayload = userPayload;
+    return this;
+  }
+
+  public String getClassName() {
+    return this.className;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/src/main/java/org/apache/tez/dag/api/TezException.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/TezException.java b/tez-api/src/main/java/org/apache/tez/dag/api/TezException.java
new file mode 100644
index 0000000..e3b14e7
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/dag/api/TezException.java
@@ -0,0 +1,31 @@
+/**
+* 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.api;
+
+/**
+ *  Base TezException
+ */
+public class TezException extends Exception {
+  private static final long serialVersionUID = 6337442733802964447L;
+  public TezException(Throwable cause) { super(cause); }
+  public TezException(String message) { super(message); }
+  public TezException(String message, Throwable cause) {
+    super(message, cause);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/src/main/java/org/apache/tez/dag/api/TezUncheckedException.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/TezUncheckedException.java b/tez-api/src/main/java/org/apache/tez/dag/api/TezUncheckedException.java
new file mode 100644
index 0000000..f55f6dd
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/dag/api/TezUncheckedException.java
@@ -0,0 +1,33 @@
+/**
+* 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.api;
+
+/**
+ * Base Tez Unchecked Exception
+ */
+public class TezUncheckedException extends RuntimeException {
+
+  private static final long serialVersionUID = -4956339297375386184L;
+  
+  public TezUncheckedException(Throwable cause) { super(cause); }
+  public TezUncheckedException(String message) { super(message); }
+  public TezUncheckedException(String message, Throwable cause) {
+    super(message, cause);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/src/main/java/org/apache/tez/dag/api/Vertex.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/Vertex.java b/tez-api/src/main/java/org/apache/tez/dag/api/Vertex.java
new file mode 100644
index 0000000..900822b
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/dag/api/Vertex.java
@@ -0,0 +1,153 @@
+/**
+ * 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.api;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.hadoop.yarn.api.records.LocalResource;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.tez.dag.api.VertexLocationHint.TaskLocationHint;
+
+public class Vertex { // FIXME rename to Task
+
+  private final String vertexName;
+  private final ProcessorDescriptor processorDescriptor;
+
+  private final int parallelism;
+  private VertexLocationHint taskLocationsHint;
+  private final Resource taskResource;
+  private Map<String, LocalResource> taskLocalResources;
+  private Map<String, String> taskEnvironment;
+
+  private final List<Vertex> inputVertices = new ArrayList<Vertex>();
+  private final List<Vertex> outputVertices = new ArrayList<Vertex>();
+  private final List<String> inputEdgeIds = new ArrayList<String>();
+  private final List<String> outputEdgeIds = new ArrayList<String>();
+  private String javaOpts = "";
+
+
+  public Vertex(String vertexName,
+      ProcessorDescriptor processorDescriptor,
+      int parallelism,
+      Resource taskResource) {
+    this.vertexName = vertexName;
+    this.processorDescriptor = processorDescriptor;
+    this.parallelism = parallelism;
+    this.taskResource = taskResource;
+    if (parallelism == 0) {
+      throw new IllegalArgumentException("Parallelism cannot be 0");
+    }
+    if (taskResource == null) {
+      throw new IllegalArgumentException("Resource cannot be null");
+    }
+  }
+
+  public String getVertexName() { // FIXME rename to getName()
+    return vertexName;
+  }
+
+  public ProcessorDescriptor getProcessorDescriptor() {
+    return this.processorDescriptor;
+  }
+
+  public int getParallelism() {
+    return parallelism;
+  }
+
+  public Resource getTaskResource() {
+    return taskResource;
+  }
+
+  public Vertex setTaskLocationsHint(List<TaskLocationHint> locations) {
+    if (locations == null) {
+      return this;
+    }
+    assert locations.size() == parallelism;
+    taskLocationsHint = new VertexLocationHint(parallelism, locations);
+    return this;
+  }
+
+  // used internally to create parallelism location resource file
+  VertexLocationHint getTaskLocationsHint() {
+    return taskLocationsHint;
+  }
+
+  public Vertex setTaskLocalResources(Map<String, LocalResource> localResources) {
+    this.taskLocalResources = localResources;
+    return this;
+  }
+
+  public Map<String, LocalResource> getTaskLocalResources() {
+    return taskLocalResources;
+  }
+
+  public Vertex setTaskEnvironment(Map<String, String> environment) {
+    this.taskEnvironment = environment;
+    return this;
+  }
+
+  public Map<String, String> getTaskEnvironment() {
+    return taskEnvironment;
+  }
+
+  public Vertex setJavaOpts(String javaOpts){
+     this. javaOpts = javaOpts;
+     return this;
+  }
+
+  public String getJavaOpts(){
+	  return javaOpts;
+  }
+
+  @Override
+  public String toString() {
+    return "[" + vertexName + " : " + processorDescriptor.getClassName() + "]";
+  }
+
+  void addInputVertex(Vertex inputVertex, String edgeId) {
+    inputVertices.add(inputVertex);
+    inputEdgeIds.add(edgeId);
+  }
+
+  void addOutputVertex(Vertex outputVertex, String edgeId) {
+    outputVertices.add(outputVertex);
+    outputEdgeIds.add(edgeId);
+  }
+
+  List<Vertex> getInputVertices() {
+    return inputVertices;
+  }
+
+  List<Vertex> getOutputVertices() {
+    return outputVertices;
+  }
+
+  List<String> getInputEdgeIds() {
+    return inputEdgeIds;
+  }
+
+  List<String> getOutputEdgeIds() {
+    return outputEdgeIds;
+  }
+
+  // FIXME how do we support profiling? Can't profile all tasks.
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/src/main/java/org/apache/tez/dag/api/VertexLocationHint.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/VertexLocationHint.java b/tez-api/src/main/java/org/apache/tez/dag/api/VertexLocationHint.java
new file mode 100644
index 0000000..4f19314
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/dag/api/VertexLocationHint.java
@@ -0,0 +1,154 @@
+/**
+ * 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.api;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+
+public class VertexLocationHint  {
+
+  private final int numTasks;
+  private final List<TaskLocationHint> taskLocationHints;
+
+  public VertexLocationHint(int numTasks,
+      List<TaskLocationHint> taskLocationHints) {
+    this.numTasks = numTasks;
+    if (taskLocationHints != null) {
+      this.taskLocationHints = Collections.unmodifiableList(taskLocationHints);
+    } else {
+      this.taskLocationHints = null;
+    }
+  }
+
+  public int getNumTasks() {
+    return numTasks;
+  }
+
+  public List<TaskLocationHint> getTaskLocationHints() {
+    return taskLocationHints;
+  }
+
+  @Override
+  public int hashCode() {
+    final int prime = 7883;
+    int result = 1;
+    result = prime * result + numTasks;
+    if (taskLocationHints != null) {
+      result = prime * result + taskLocationHints.hashCode();
+    }
+    return result;
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (this == obj) {
+      return true;
+    }
+    if (obj == null) {
+      return false;
+    }
+    if (getClass() != obj.getClass()) {
+      return false;
+    }
+    VertexLocationHint other = (VertexLocationHint) obj;
+    if (numTasks != other.numTasks) {
+      return false;
+    }
+    if (taskLocationHints != null) {
+      if (!taskLocationHints.equals(other.taskLocationHints)) {
+        return false;
+      }
+    } else if (other.taskLocationHints != null) {
+      return false;
+    }
+    return true;
+  }
+
+  public static class TaskLocationHint {
+
+    // Host names if any to be used
+    private final Set<String> hosts;
+    // Rack names if any to be used
+    private final Set<String> racks;
+
+    public TaskLocationHint(Set<String> hosts, Set<String> racks) {
+      if (hosts != null) {
+        this.hosts = Collections.unmodifiableSet(hosts);
+      } else {
+        this.hosts = null;
+      }
+      if (racks != null) {
+        this.racks = Collections.unmodifiableSet(racks);
+      } else {
+        this.racks = null;
+      }
+    }
+
+    public Set<String> getDataLocalHosts() {
+      return hosts;
+    }
+
+    public Set<String> getRacks() {
+      return racks;
+    }
+
+    @Override
+    public int hashCode() {
+      final int prime = 9397;
+      int result = 1;
+      result = ( hosts != null) ?
+          prime * result + hosts.hashCode() :
+          result + prime;
+      result = ( racks != null) ?
+          prime * result + racks.hashCode() :
+          result + prime;
+      return result;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+      if (this == obj) {
+        return true;
+      }
+      if (obj == null) {
+        return false;
+      }
+      if (getClass() != obj.getClass()) {
+        return false;
+      }
+      TaskLocationHint other = (TaskLocationHint) obj;
+      if (hosts != null) {
+        if (!hosts.equals(other.hosts)) {
+          return false;
+        }
+      } else if (other.hosts != null) {
+        return false;
+      }
+      if (racks != null) {
+        if (!racks.equals(other.racks)) {
+          return false;
+        }
+      } else if (other.racks != null) {
+        return false;
+      }
+      return true;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGClient.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGClient.java b/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGClient.java
new file mode 100644
index 0000000..9062e8e
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGClient.java
@@ -0,0 +1,67 @@
+/**
+* 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.api.client;
+
+import java.io.Closeable;
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.ApplicationReport;
+import org.apache.tez.dag.api.TezException;
+
+/*
+ * Interface class for monitoring the <code>DAG</code> running in a Tez DAG
+ * Application Master.
+ */
+public interface DAGClient extends Closeable {
+
+  /**
+   * Get the YARN ApplicationId for the app running the DAG
+   * @return <code>ApplicationId</code>
+   */
+  public ApplicationId getApplicationId();
+
+  @Private
+  /**
+   * Get the YARN ApplicationReport for the app running the DAG. For performance
+   * reasons this may be stale copy and should be used to access static info. It
+   * may be null.
+   * @return <code>ApplicationReport</code> or null
+   */
+  public ApplicationReport getApplicationReport();
+
+  /**
+   * Get the status of the specified DAG
+   */
+  public DAGStatus getDAGStatus() throws IOException, TezException;
+
+  /**
+   * Get the status of a Vertex of a DAG
+   */
+  public VertexStatus getVertexStatus(String vertexName)
+      throws IOException, TezException;
+
+  /**
+   * Kill a running DAG
+   *
+   */
+  public void tryKillDAG() throws TezException, IOException;
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/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
new file mode 100644
index 0000000..d61173d
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGStatus.java
@@ -0,0 +1,130 @@
+/**
+* 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.api.client;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.tez.dag.api.records.DAGProtos.DAGStatusProtoOrBuilder;
+import org.apache.tez.dag.api.records.DAGProtos.StringProgressPairProto;
+import org.apache.tez.dag.api.TezUncheckedException;
+
+public class DAGStatus {
+
+  public enum State {
+    SUBMITTED,
+    INITING,
+    RUNNING,
+    SUCCEEDED,
+    KILLED,
+    FAILED,
+    ERROR,
+  };
+
+  DAGStatusProtoOrBuilder proxy = null;
+  Progress progress = null;
+  Map<String, Progress> vertexProgress = null;
+
+  public DAGStatus(DAGStatusProtoOrBuilder proxy) {
+    this.proxy = proxy;
+  }
+
+  public State getState() {
+    switch(proxy.getState()) {
+    case DAG_SUBMITTED:
+      return DAGStatus.State.SUBMITTED;
+    // For simplicity, initing/terminating states are presented as running
+    case DAG_INITING:
+    case DAG_TERMINATING:
+    case DAG_RUNNING:
+      return DAGStatus.State.RUNNING;
+    case DAG_SUCCEEDED:
+      return DAGStatus.State.SUCCEEDED;
+    case DAG_FAILED:
+      return DAGStatus.State.FAILED;
+    case DAG_KILLED:
+      return DAGStatus.State.KILLED;
+    case DAG_ERROR:
+      return DAGStatus.State.ERROR;
+    default:
+      throw new TezUncheckedException("Unsupported value for DAGStatus.State : " +
+                              proxy.getState());
+    }
+  }
+
+  public boolean isCompleted() {
+    State state = getState();
+    return (state == State.SUCCEEDED ||
+             state == State.FAILED ||
+             state == State.KILLED ||
+             state == State.ERROR);
+  }
+
+  public List<String> getDiagnostics() {
+    return proxy.getDiagnosticsList();
+  }
+
+  /**
+   * Gets overall progress value of the DAG.
+   *
+   * @return Progress of the DAG. Maybe null when the DAG is not running. Maybe
+   *         null when the DAG is running and the application master cannot be
+   *         reached - e.g. when the execution platform has restarted the
+   *         application master.
+   * @see Progress
+   */
+  public Progress getDAGProgress() {
+    if(progress == null && proxy.hasDAGProgress()) {
+      progress = new Progress(proxy.getDAGProgress());
+    }
+    return progress;
+  }
+
+  /**
+   * Get the progress of a vertex in the DAG
+   *
+   * @return Progress of the vertex. May be null when the DAG is not running.
+   *         Maybe null when the DAG is running and the application master
+   *         cannot be reached - e.g. when the execution platform has restarted
+   *         the application master.
+   * @see Progress
+   */
+  public Map<String, Progress> getVertexProgress() {
+    if(vertexProgress == null) {
+      if(proxy.getVertexProgressList() != null) {
+        List<StringProgressPairProto> kvList = proxy.getVertexProgressList();
+        vertexProgress = new HashMap<String, Progress>(kvList.size());
+        for(StringProgressPairProto kv : kvList){
+          vertexProgress.put(kv.getKey(), new Progress(kv.getProgress()));
+        }
+      }
+    }
+    return vertexProgress;
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder();
+    sb.append("status=" + getState()
+        + ", progress=" + getDAGProgress());
+    return sb.toString();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/src/main/java/org/apache/tez/dag/api/client/Progress.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/client/Progress.java b/tez-api/src/main/java/org/apache/tez/dag/api/client/Progress.java
new file mode 100644
index 0000000..9577320
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/dag/api/client/Progress.java
@@ -0,0 +1,67 @@
+/**
+* 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.api.client;
+
+import org.apache.tez.dag.api.records.DAGProtos.ProgressProtoOrBuilder;
+
+public class Progress {
+  
+  ProgressProtoOrBuilder proxy = null;
+  
+  Progress(ProgressProtoOrBuilder proxy) {
+    this.proxy = proxy;
+  }
+  
+  public int getTotalTaskCount() {
+    return proxy.getTotalTaskCount();
+  }
+
+  public int getSucceededTaskCount() {
+    return proxy.getSucceededTaskCount();
+  }
+
+  public int getRunningTaskCount() {
+    return proxy.getRunningTaskCount();
+  }
+
+  public int getFailedTaskCount() {
+    return proxy.getFailedTaskCount();
+  }
+
+  public int getKilledTaskCount() {
+    return proxy.getKilledTaskCount();
+  }
+  
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder();
+    sb.append("TotalTasks: ");
+    sb.append(getTotalTaskCount());
+    sb.append(" Succeeded: ");
+    sb.append(getSucceededTaskCount());
+    sb.append(" Running: ");
+    sb.append(getRunningTaskCount());
+    sb.append(" Failed: "); 
+    sb.append(getFailedTaskCount());
+    sb.append(" Killed: "); 
+    sb.append(getKilledTaskCount());
+    return sb.toString();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/src/main/java/org/apache/tez/dag/api/client/VertexStatus.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/client/VertexStatus.java b/tez-api/src/main/java/org/apache/tez/dag/api/client/VertexStatus.java
new file mode 100644
index 0000000..ce5dbe0
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/dag/api/client/VertexStatus.java
@@ -0,0 +1,78 @@
+/**
+* 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.api.client;
+
+import java.util.List;
+
+import org.apache.tez.dag.api.records.DAGProtos.VertexStatusProtoOrBuilder;
+import org.apache.tez.dag.api.TezUncheckedException;
+
+public class VertexStatus {
+  
+  public enum State {
+    INITED,
+    RUNNING,
+    SUCCEEDED,
+    KILLED,
+    FAILED,
+    ERROR,
+    TERMINATING,
+  };
+  
+  VertexStatusProtoOrBuilder proxy = null;
+  Progress progress = null;
+  
+  public VertexStatus(VertexStatusProtoOrBuilder proxy) {
+    this.proxy = proxy;
+  }
+
+  public State getState() {
+    switch(proxy.getState()) {
+    case VERTEX_INITED:
+      return VertexStatus.State.INITED;
+    case VERTEX_RUNNING:
+      return VertexStatus.State.RUNNING;
+    case VERTEX_SUCCEEDED:
+      return VertexStatus.State.SUCCEEDED;
+    case VERTEX_FAILED:
+      return VertexStatus.State.FAILED;
+    case VERTEX_KILLED:
+      return VertexStatus.State.KILLED;
+    case VERTEX_ERROR:
+      return VertexStatus.State.ERROR;
+    case VERTEX_TERMINATING:
+      return VertexStatus.State.TERMINATING;
+    default:
+      throw new TezUncheckedException("Unsupported value for VertexStatus.State : " + 
+                              proxy.getState());
+    }    
+  }
+
+  public List<String> getDiagnostics() {
+    return proxy.getDiagnosticsList();
+  }
+
+  public Progress getProgress() {
+    if(progress == null && proxy.hasProgress()) {
+      progress = new Progress(proxy.getProgress());
+    }
+    return progress;    
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/src/main/java/org/apache/tez/dag/api/client/rpc/DAGClientAMProtocolBlockingPB.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/client/rpc/DAGClientAMProtocolBlockingPB.java b/tez-api/src/main/java/org/apache/tez/dag/api/client/rpc/DAGClientAMProtocolBlockingPB.java
new file mode 100644
index 0000000..a1ee18f
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/dag/api/client/rpc/DAGClientAMProtocolBlockingPB.java
@@ -0,0 +1,30 @@
+/**
+ * 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.api.client.rpc;
+
+import org.apache.hadoop.ipc.ProtocolInfo;
+import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolRPC.DAGClientAMProtocol;
+
+@ProtocolInfo(
+    protocolName = "org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolPB",
+    protocolVersion = 1)
+public interface DAGClientAMProtocolBlockingPB 
+                              extends DAGClientAMProtocol.BlockingInterface {
+
+}


[22/50] [abbrv] TEZ-443. Merge tez-dag-api and tez-engine-api into a single module - tez-api (part of TEZ-398). (sseth)

Posted by ss...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/src/main/java/org/apache/tez/dag/api/client/rpc/DAGClientRPCImpl.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/client/rpc/DAGClientRPCImpl.java b/tez-api/src/main/java/org/apache/tez/dag/api/client/rpc/DAGClientRPCImpl.java
new file mode 100644
index 0000000..dae5625
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/dag/api/client/rpc/DAGClientRPCImpl.java
@@ -0,0 +1,291 @@
+/**
+ * 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.api.client.rpc;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.Collections;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.ipc.ProtobufRpcEngine;
+import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.ApplicationReport;
+import org.apache.hadoop.yarn.api.records.YarnApplicationState;
+import org.apache.hadoop.yarn.client.api.YarnClient;
+import org.apache.hadoop.yarn.client.api.impl.YarnClientImpl;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.tez.dag.api.TezConfiguration;
+import org.apache.tez.dag.api.TezException;
+import org.apache.tez.dag.api.TezUncheckedException;
+import org.apache.tez.dag.api.client.DAGClient;
+import org.apache.tez.dag.api.client.DAGStatus;
+import org.apache.tez.dag.api.client.VertexStatus;
+import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolRPC.GetDAGStatusRequestProto;
+import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolRPC.GetVertexStatusRequestProto;
+import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolRPC.TryKillDAGRequestProto;
+import org.apache.tez.dag.api.records.DAGProtos.DAGStatusProto;
+import org.apache.tez.dag.api.records.DAGProtos.DAGStatusStateProto;
+
+import com.google.protobuf.ServiceException;
+
+public class DAGClientRPCImpl implements DAGClient {
+  private static final Log LOG = LogFactory.getLog(DAGClientRPCImpl.class);
+
+  private final ApplicationId appId;
+  private final String dagId;
+  private final TezConfiguration conf;
+  private ApplicationReport appReport;
+  private YarnClient yarnClient;
+  private DAGClientAMProtocolBlockingPB proxy = null;
+
+  public DAGClientRPCImpl(ApplicationId appId, String dagId,
+      TezConfiguration conf) {
+    this.appId = appId;
+    this.dagId = dagId;
+    this.conf = conf;
+    yarnClient = new YarnClientImpl();
+    yarnClient.init(new YarnConfiguration(conf));
+    yarnClient.start();
+    appReport = null;
+  }
+
+  @Override
+  public ApplicationId getApplicationId() {
+    return appId;
+  }
+
+  @Override
+  public DAGStatus getDAGStatus() throws IOException, TezException {
+    if(createAMProxyIfNeeded()) {
+      try {
+        return getDAGStatusViaAM();
+      } catch (TezException e) {
+        resetProxy(e); // create proxy again
+      }
+    }
+
+    // Later maybe from History
+    return getDAGStatusViaRM();
+  }
+
+  @Override
+  public VertexStatus getVertexStatus(String vertexName)
+                                    throws IOException, TezException {
+    if(createAMProxyIfNeeded()) {
+      try {
+        return getVertexStatusViaAM(vertexName);
+      } catch (TezException e) {
+        resetProxy(e); // create proxy again
+      }
+    }
+
+    // need AM for this. Later maybe from History
+    return null;
+  }
+
+  @Override
+  public void tryKillDAG() throws TezException, IOException {
+    if(LOG.isDebugEnabled()) {
+      LOG.debug("TryKill for app: " + appId + " dag:" + dagId);
+    }
+    if(createAMProxyIfNeeded()) {
+      TryKillDAGRequestProto requestProto =
+          TryKillDAGRequestProto.newBuilder().setDagId(dagId).build();
+      try {
+        proxy.tryKillDAG(null, requestProto);
+      } catch (ServiceException e) {
+        resetProxy(e);
+      }
+    }
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (this.proxy != null) {
+      RPC.stopProxy(this.proxy);
+    }
+    if(yarnClient != null) {
+      yarnClient.stop();
+    }
+  }
+
+  @Override
+  public ApplicationReport getApplicationReport() {
+    return appReport;
+  }
+
+  void resetProxy(Exception e) {
+    if(LOG.isDebugEnabled()) {
+      LOG.debug("Resetting AM proxy for app: " + appId + " dag:" + dagId +
+          " due to exception :", e);
+    }
+    proxy = null;
+  }
+
+  DAGStatus getDAGStatusViaAM() throws IOException, TezException {
+    if(LOG.isDebugEnabled()) {
+      LOG.debug("GetDAGStatus via AM for app: " + appId + " dag:" + dagId);
+    }
+    GetDAGStatusRequestProto requestProto =
+        GetDAGStatusRequestProto.newBuilder().setDagId(dagId).build();
+    try {
+      return new DAGStatus(
+                 proxy.getDAGStatus(null, requestProto).getDagStatus());
+    } catch (ServiceException e) {
+      // TEZ-151 retrieve wrapped TezException
+      throw new TezException(e);
+    }
+  }
+
+
+
+  DAGStatus getDAGStatusViaRM() throws TezException, IOException {
+    if(LOG.isDebugEnabled()) {
+      LOG.debug("GetDAGStatus via AM for app: " + appId + " dag:" + dagId);
+    }
+    ApplicationReport appReport;
+    try {
+      appReport = yarnClient.getApplicationReport(appId);
+    } catch (YarnException e) {
+      throw new TezException(e);
+    }
+
+    if(appReport == null) {
+      throw new TezException("Unknown/Invalid appId: " + appId);
+    }
+
+    DAGStatusProto.Builder builder = DAGStatusProto.newBuilder();
+    DAGStatus dagStatus = new DAGStatus(builder);
+    DAGStatusStateProto dagState = null;
+    switch (appReport.getYarnApplicationState()) {
+    case NEW:
+    case NEW_SAVING:
+    case SUBMITTED:
+    case ACCEPTED:
+      dagState = DAGStatusStateProto.DAG_SUBMITTED;
+      break;
+    case RUNNING:
+      dagState = DAGStatusStateProto.DAG_RUNNING;
+      break;
+    case FAILED:
+      dagState = DAGStatusStateProto.DAG_FAILED;
+      break;
+    case KILLED:
+      dagState = DAGStatusStateProto.DAG_KILLED;
+      break;
+    case FINISHED:
+      switch(appReport.getFinalApplicationStatus()) {
+      case UNDEFINED:
+      case FAILED:
+        dagState = DAGStatusStateProto.DAG_FAILED;
+        break;
+      case KILLED:
+        dagState = DAGStatusStateProto.DAG_KILLED;
+        break;
+      case SUCCEEDED:
+        dagState = DAGStatusStateProto.DAG_SUCCEEDED;
+        break;
+      default:
+        throw new TezUncheckedException("Encountered unknown final application"
+          + " status from YARN"
+          + ", appState=" + appReport.getYarnApplicationState()
+          + ", finalStatus=" + appReport.getFinalApplicationStatus());
+      }
+      break;
+    default:
+      throw new TezUncheckedException("Encountered unknown application state"
+          + " from YARN, appState=" + appReport.getYarnApplicationState());
+    }
+
+    builder.setState(dagState);
+    if(appReport.getDiagnostics() != null) {
+      builder.addAllDiagnostics(Collections.singleton(appReport.getDiagnostics()));
+    }
+
+    return dagStatus;
+  }
+
+  VertexStatus getVertexStatusViaAM(String vertexName) throws TezException {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("GetVertexStatus via AM for app: " + appId + " dag: " + dagId
+          + " vertex: " + vertexName);
+    }
+    GetVertexStatusRequestProto requestProto =
+        GetVertexStatusRequestProto.newBuilder().
+                        setDagId(dagId).setVertexName(vertexName).build();
+
+    try {
+      return new VertexStatus(
+                 proxy.getVertexStatus(null, requestProto).getVertexStatus());
+    } catch (ServiceException e) {
+      // TEZ-151 retrieve wrapped TezException
+      throw new TezException(e);
+    }
+  }
+
+  ApplicationReport getAppReport() throws IOException, TezException {
+    try {
+      ApplicationReport appReport = yarnClient.getApplicationReport(appId);
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("App: " + appId + " in state: "
+            + appReport.getYarnApplicationState());
+      }
+      return appReport;
+    } catch (YarnException e) {
+      throw new TezException(e);
+    }
+  }
+
+  boolean createAMProxyIfNeeded() throws IOException, TezException {
+    if(proxy != null) {
+      // if proxy exist optimistically use it assuming there is no retry
+      return true;
+    }
+    appReport = getAppReport();
+
+    if(appReport == null) {
+      return false;
+    }
+    YarnApplicationState appState = appReport.getYarnApplicationState();
+    if(appState != YarnApplicationState.RUNNING) {
+      return false;
+    }
+
+    // YARN-808. Cannot ascertain if AM is ready until we connect to it.
+    // workaround check the default string set by YARN
+    if(appReport.getHost() == null || appReport.getHost().equals("N/A") ||
+        appReport.getRpcPort() == 0){
+      // attempt not running
+      return false;
+    }
+
+    InetSocketAddress addr = new InetSocketAddress(appReport.getHost(),
+        appReport.getRpcPort());
+
+    RPC.setProtocolEngine(conf, DAGClientAMProtocolBlockingPB.class,
+        ProtobufRpcEngine.class);
+    proxy = (DAGClientAMProtocolBlockingPB) RPC.getProxy(
+        DAGClientAMProtocolBlockingPB.class, 0, addr, conf);
+    return true;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/src/main/java/org/apache/tez/engine/api/Event.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/engine/api/Event.java b/tez-api/src/main/java/org/apache/tez/engine/api/Event.java
new file mode 100644
index 0000000..80da655
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/engine/api/Event.java
@@ -0,0 +1,28 @@
+/**
+ * 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.engine.api;
+
+/**
+ * Base class for all events generated within the Tez execution engine.
+ * Used as the primary mode of communication between the AM, Inputs, Processors
+ * and Outputs.
+ */
+public abstract class Event {
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/src/main/java/org/apache/tez/engine/api/Input.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/engine/api/Input.java b/tez-api/src/main/java/org/apache/tez/engine/api/Input.java
new file mode 100644
index 0000000..e333075
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/engine/api/Input.java
@@ -0,0 +1,71 @@
+/**
+ * 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.engine.api;
+
+import java.util.List;
+
+/**
+ * Represents an input through which a TezProcessor receives data on an edge.
+ * </p>
+ *
+ * <code>Input</code> classes must have a 0 argument public constructor for Tez
+ * to construct the <code>Input</code>. Tez will take care of initializing and
+ * closing the Input after a {@link Processor} completes. </p>
+ */
+public interface Input {
+
+  /**
+   * Initializes the <code>Input</code>.
+   *
+   * @param inputContext
+   *          the {@link TezInputContext}
+   * @return
+   * @throws Exception
+   *           if an error occurs
+   */
+  public List<Event> initialize(TezInputContext inputContext)
+      throws Exception;
+
+  /**
+   * Gets an instance of the {@link Reader} for this <code>Output</code>
+   *
+   * @return
+   * @throws Exception
+   *           if an error occurs
+   */
+  public Reader getReader() throws Exception;
+
+  /**
+   * Handles user and system generated {@link Events}s, which typically carry
+   * information such as an output being available on the previous vertex.
+   *
+   * @param inputEvents
+   *          the list of {@link Event}s
+   */
+  public void handleEvents(List<Event> inputEvents);
+
+  /**
+   * Closes the <code>Input</code>
+   *
+   * @return
+   * @throws Exception
+   *           if an error occurs
+   */
+  public List<Event> close() throws Exception;
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/src/main/java/org/apache/tez/engine/api/LogicalIOProcessor.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/engine/api/LogicalIOProcessor.java b/tez-api/src/main/java/org/apache/tez/engine/api/LogicalIOProcessor.java
new file mode 100644
index 0000000..90be09e
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/engine/api/LogicalIOProcessor.java
@@ -0,0 +1,43 @@
+/**
+ * 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.engine.api;
+
+import java.util.Map;
+
+/**
+ * Represents a processor which consumes {@link LogicalInput}s and produces
+ * {@link LogicalOutput}s
+ */
+public interface LogicalIOProcessor extends Processor {
+
+  /**
+   * Runs the {@link LogicalProcessor}
+   * 
+   * @param inputs
+   *          a map of the source vertex name to {@link LogicalInput} - one per
+   *          incoming edge.
+   * @param outputs
+   *          a map of the destination vertex name to {@link LogicalOutput} -
+   *          one per outgoing edge
+   * @throws Exception TODO
+   */
+  public void run(Map<String, LogicalInput> inputs,
+      Map<String, LogicalOutput> outputs) throws Exception;
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/src/main/java/org/apache/tez/engine/api/LogicalInput.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/engine/api/LogicalInput.java b/tez-api/src/main/java/org/apache/tez/engine/api/LogicalInput.java
new file mode 100644
index 0000000..4a47ccf
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/engine/api/LogicalInput.java
@@ -0,0 +1,37 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.engine.api;
+
+/**
+ * An @link {@link Input} which handles all incoming physical connections on an
+ * edge. A {@link LogicalIOProcessor} sees a single Logical Input per incoming
+ * edge.
+ */
+public interface LogicalInput extends Input {
+
+  /**
+   * Sets the number of physical inputs that this <code>LogicalInput</code> will
+   * receive. This will be called by the Tez framework before initializing the
+   * <code>LogicalInput</code>
+   * 
+   * @param numInputs
+   *          the number of physical inputs.
+   */
+  public void setNumPhysicalInputs(int numInputs);
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/src/main/java/org/apache/tez/engine/api/LogicalOutput.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/engine/api/LogicalOutput.java b/tez-api/src/main/java/org/apache/tez/engine/api/LogicalOutput.java
new file mode 100644
index 0000000..4626fbd
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/engine/api/LogicalOutput.java
@@ -0,0 +1,36 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.engine.api;
+
+/**
+ * An @link {@link Output} which handles all outgoing physical connections on an
+ * edge. A {@link LogicalIOProcessor} sees a single Logical Output per outgoing
+ * edge.
+ */
+public interface LogicalOutput extends Output {
+  /**
+   * Sets the number of physical ouputs that this <code>LogicalOutput</code>
+   * will receive. This will be called by the Tez framework before initializing
+   * the <code>LogicalOutput</code>
+   * 
+   * @param numOutputs
+   *          the number of physical outputs
+   */
+  public void setNumPhysicalOutputs(int numOutputs);
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/src/main/java/org/apache/tez/engine/api/Output.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/engine/api/Output.java b/tez-api/src/main/java/org/apache/tez/engine/api/Output.java
new file mode 100644
index 0000000..ec679ed
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/engine/api/Output.java
@@ -0,0 +1,71 @@
+/**
+ * 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.engine.api;
+
+import java.util.List;
+
+/**
+ * Represents an Output through which a TezProcessor writes information on an
+ * edge. </p>
+ *
+ * <code>Output</code> implementations must have a 0 argument public constructor
+ * for Tez to construct the <code>Output</code>. Tez will take care of
+ * initializing and closing the Input after a {@link Processor} completes. </p>
+ */
+public interface Output {
+
+  /**
+   * Initializes the <code>Output</code>
+   *
+   * @param outputContext
+   *          the {@link TezOutputContext}
+   * @return
+   * @throws Exception
+   *           if an error occurs
+   */
+  public List<Event> initialize(TezOutputContext outputContext)
+      throws Exception;
+
+  /**
+   * Gets an instance of the {@link Writer} in an <code>Output</code>
+   *
+   * @return
+   * @throws Exception
+   *           if an error occurs
+   */
+  public Writer getWriter() throws Exception;
+
+  /**
+   * Handles user and system generated {@link Events}s, which typically carry
+   * information such as a downstream vertex being ready to consume input.
+   *
+   * @param outputEvents
+   *          the list of {@link Event}s
+   */
+  public void handleEvents(List<Event> outputEvents);
+
+  /**
+   * Closes the <code>Output</code>
+   *
+   * @return
+   * @throws Exception
+   *           if an error occurs
+   */
+  public List<Event> close() throws Exception;
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/src/main/java/org/apache/tez/engine/api/Processor.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/engine/api/Processor.java b/tez-api/src/main/java/org/apache/tez/engine/api/Processor.java
new file mode 100644
index 0000000..05e6b84
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/engine/api/Processor.java
@@ -0,0 +1,55 @@
+/**
+ * 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.engine.api;
+
+import java.io.IOException;
+import java.util.List;
+
+/**
+ * {@link Processor} represents the <em>Tez</em> entity responsible for
+ * consuming {@link Input} and producing {@link Output}.
+ */
+public interface Processor {
+
+  /**
+   * Initializes the <code>Processor</code>
+   *
+   * @param processorContext
+   * @throws IOException
+   *           if an error occurs
+   */
+  public void initialize(TezProcessorContext processorContext)
+      throws Exception;
+
+  /**
+   * Handles user and system generated {@link Events}s.
+   *
+   * @param processorEvents
+   *          the list of {@link Event}s
+   */
+  public void handleEvents(List<Event> processorEvents);
+
+  /**
+   * Closes the <code>Processor</code>
+   *
+   * @throws IOException
+   *           if an error occurs
+   */
+  public void close() throws Exception;
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/src/main/java/org/apache/tez/engine/api/Reader.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/engine/api/Reader.java b/tez-api/src/main/java/org/apache/tez/engine/api/Reader.java
new file mode 100644
index 0000000..502c5f2
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/engine/api/Reader.java
@@ -0,0 +1,26 @@
+/**
+ * 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.engine.api;
+
+/**
+ * A <code>Reader</code> represents the data being read in an {@link Input}
+ */
+public interface Reader {
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/src/main/java/org/apache/tez/engine/api/TezInputContext.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/engine/api/TezInputContext.java b/tez-api/src/main/java/org/apache/tez/engine/api/TezInputContext.java
new file mode 100644
index 0000000..ddf1ff8
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/engine/api/TezInputContext.java
@@ -0,0 +1,32 @@
+/**
+ * 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.engine.api;
+
+/**
+ * Context handle for the Input to initialize itself.
+ */
+public interface TezInputContext extends TezTaskContext {
+
+  /**
+   * Get the Vertex Name of the Source that generated data for this Input
+   * @return Name of the Source Vertex
+   */
+  public String getSourceVertexName();
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/src/main/java/org/apache/tez/engine/api/TezOutputContext.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/engine/api/TezOutputContext.java b/tez-api/src/main/java/org/apache/tez/engine/api/TezOutputContext.java
new file mode 100644
index 0000000..791a0f0
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/engine/api/TezOutputContext.java
@@ -0,0 +1,33 @@
+/**
+ * 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.engine.api;
+
+/**
+ * Context handle for the Output to initialize itself.
+ */
+public interface TezOutputContext extends TezTaskContext {
+
+  /**
+   * Get the Vertex Name of the Destination that is the recipient of this
+   * Output's data
+   * @return Name of the Destination Vertex
+   */
+  public String getDestinationVertexName();
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/src/main/java/org/apache/tez/engine/api/TezProcessorContext.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/engine/api/TezProcessorContext.java b/tez-api/src/main/java/org/apache/tez/engine/api/TezProcessorContext.java
new file mode 100644
index 0000000..2bbbe81
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/engine/api/TezProcessorContext.java
@@ -0,0 +1,41 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.engine.api;
+
+import java.io.IOException;
+
+/**
+ * Context handle for the Processor to initialize itself.
+ */
+public interface TezProcessorContext extends TezTaskContext {
+
+  /**
+   * Set the overall progress of this Task Attempt
+   * @param progress Progress in the range from [0.0 - 1.0f]
+   */
+  public void setProgress(float progress);
+
+  /**
+   * Check whether this attempt can commit its output
+   * @return true if commit allowed
+   * @throws IOException
+   */
+  public boolean canCommit() throws IOException;
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/src/main/java/org/apache/tez/engine/api/TezTaskContext.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/engine/api/TezTaskContext.java b/tez-api/src/main/java/org/apache/tez/engine/api/TezTaskContext.java
new file mode 100644
index 0000000..706e646
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/engine/api/TezTaskContext.java
@@ -0,0 +1,130 @@
+/**
+ * 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.engine.api;
+
+import java.nio.ByteBuffer;
+import java.util.List;
+
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.tez.common.counters.TezCounters;
+
+/**
+ * Base interface for Context classes used to initialize the Input, Output
+ * and Processor instances.
+ */
+public interface TezTaskContext {
+
+  // TODO NEWTEZ
+  // Scale the maximum events we fetch per RPC call to mitigate OOM issues
+  // on the ApplicationMaster when a thundering herd of reducers fetch events
+  // This should not be necessary after HADOOP-8942
+
+  /**
+   * Get the {@link ApplicationId} for the running app
+   * @return the {@link ApplicationId}
+   */
+  public ApplicationId getApplicationId();
+
+  /**
+   * Get the current DAG Attempt Number
+   * @return DAG Attempt Number
+   */
+  public int getDAGAttemptNumber();
+
+  /**
+   * Get the index of this Task
+   * @return Task Index
+   */
+  public int getTaskIndex();
+
+  /**
+   * Get the current Task Attempt Number
+   * @return Task Attempt Number
+   */
+  public int getTaskAttemptNumber();
+
+  /**
+   * Get the name of the DAG
+   * @return the DAG name
+   */
+  public String getDAGName();
+
+  /**
+   * Get the name of the Vertex in which the task is running
+   * @return Vertex Name
+   */
+  public String getTaskVertexName();
+
+  public TezCounters getCounters();
+
+  /**
+   * Send Events to the AM and/or dependent Vertices
+   * @param events Events to be sent
+   */
+  public void sendEvents(List<Event> events);
+
+  /**
+   * Get the User Payload for the Input/Output/Processor
+   * @return User Payload
+   */
+  public byte[] getUserPayload();
+
+  /**
+   * Get the work diectories for the Input/Output/Processor
+   * @return an array of work dirs
+   */
+  public String[] getWorkDirs();
+
+  /**
+   * Returns an identifier which is unique to the specific Input, Processor or
+   * Output
+   *
+   * @return
+   */
+  public String getUniqueIdentifier();
+
+  /**
+   * Report a fatal error to the framework. This will cause the entire task to
+   * fail and should not be used for reporting temporary or recoverable errors
+   *
+   * @param exception an exception representing the error
+   */
+  public void fatalError(Throwable exception, String message);
+
+  /**
+   * Returns meta-data for the specified service. As an example, when the MR
+   * ShuffleHandler is used - this would return the jobToken serialized as bytes
+   *
+   * @param serviceName
+   *          the name of the service for which meta-data is required
+   * @return a ByteBuffer representing the meta-data
+   */
+  public ByteBuffer getServiceConsumerMetaData(String serviceName);
+
+  /**
+   * Return Provider meta-data for the specified service As an example, when the
+   * MR ShuffleHandler is used - this would return the shuffle port serialized
+   * as bytes
+   *
+   * @param serviceName
+   *          the name of the service for which provider meta-data is required
+   * @return a ByteBuffer representing the meta-data
+   */
+  public ByteBuffer getServiceProviderMetaData(String serviceName);
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/src/main/java/org/apache/tez/engine/api/Writer.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/engine/api/Writer.java b/tez-api/src/main/java/org/apache/tez/engine/api/Writer.java
new file mode 100644
index 0000000..c9503a3
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/engine/api/Writer.java
@@ -0,0 +1,26 @@
+/**
+ * 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.engine.api;
+
+/**
+ * A <code>Writer</code> represents the data being written by an {@link Output}
+ */
+public interface Writer {
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/src/main/java/org/apache/tez/engine/api/events/DataMovementEvent.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/engine/api/events/DataMovementEvent.java b/tez-api/src/main/java/org/apache/tez/engine/api/events/DataMovementEvent.java
new file mode 100644
index 0000000..3f35555
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/engine/api/events/DataMovementEvent.java
@@ -0,0 +1,109 @@
+/**
+ * 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.engine.api.events;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.tez.engine.api.Event;
+
+/**
+ * Event used by user code to send information between tasks. An output can
+ * generate an Event of this type to sending information regarding output data
+ * ( such as URI for file-based output data, port info in case of
+ * streaming-based data transfers ) to the Input on the destination vertex.
+ */
+public final class DataMovementEvent extends Event {
+
+  /**
+   * Index(i) of the i-th (physical) Input or Output that generated an Event.
+   * For a Processor-generated event, this is ignored.
+   */
+  private final int sourceIndex;
+
+  /**
+   * Index(i) of the i-th (physical) Input or Output that is meant to receive
+   * this Event. For a Processor event, this is ignored.
+   */
+  private int targetIndex;
+
+  /**
+   * User Payload for this Event
+   */
+  private final byte[] userPayload;
+
+  /**
+   * Version number to indicate what attempt generated this Event
+   */
+  private int version;
+
+  /**
+   * User Event constructor
+   * @param sourceIndex Index to identify the physical edge of the input/output
+   * that generated the event
+   * @param userPayload User Payload of the User Event
+   */
+  public DataMovementEvent(int sourceIndex,
+      byte[] userPayload) {
+    this.userPayload = userPayload;
+    this.sourceIndex = sourceIndex;
+  }
+
+  @Private
+  public DataMovementEvent(int sourceIndex,
+      int targetIndex,
+      byte[] userPayload) {
+    this.userPayload = userPayload;
+    this.sourceIndex = sourceIndex;
+    this.targetIndex = targetIndex;
+  }
+
+  /**
+   * Constructor for Processor-generated User Events
+   * @param userPayload
+   */
+  public DataMovementEvent(byte[] userPayload) {
+    this(-1, userPayload);
+  }
+
+  public byte[] getUserPayload() {
+    return userPayload;
+  }
+
+  public int getSourceIndex() {
+    return sourceIndex;
+  }
+
+  public int getTargetIndex() {
+    return targetIndex;
+  }
+
+  @Private
+  public void setTargetIndex(int targetIndex) {
+    this.targetIndex = targetIndex;
+  }
+
+  public int getVersion() {
+    return version;
+  }
+
+  @Private
+  public void setVersion(int version) {
+    this.version = version;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/src/main/java/org/apache/tez/engine/api/events/InputFailedEvent.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/engine/api/events/InputFailedEvent.java b/tez-api/src/main/java/org/apache/tez/engine/api/events/InputFailedEvent.java
new file mode 100644
index 0000000..57de09b
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/engine/api/events/InputFailedEvent.java
@@ -0,0 +1,89 @@
+/**
+ * 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.engine.api.events;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.tez.engine.api.Event;
+
+/**
+ * Event sent from the AM to an Input to indicate that one of it's sources has
+ * failed - effectively the input is no longer available from the particular
+ * source.
+ * Users are not expected to send this event.
+ */
+public class InputFailedEvent extends Event{
+
+  /**
+   * Index(i) of the i-th (physical) Input or Output that generated the data.
+   * For a Processor-generated event, this is ignored.
+   */
+  private final int sourceIndex;
+
+  /**
+   * Index(i) of the i-th (physical) Input or Output that is meant to receive
+   * this Event. For a Processor event, this is ignored.
+   */
+  private int targetIndex;
+
+  /**
+   * Version number to indicate what attempt generated this Event
+   */
+  private int version;
+
+  /**
+   * User Event constructor
+   * @param sourceIndex Index to identify the physical edge of the input/output
+   * that generated the event
+   * @param userPayload User Payload of the User Event
+   */
+  public InputFailedEvent(int sourceIndex) {
+    this.sourceIndex = sourceIndex;
+  }
+
+  @Private
+  public InputFailedEvent(int sourceIndex,
+      int targetIndex,
+      int version) {
+    this.sourceIndex = sourceIndex;
+    this.targetIndex = targetIndex;
+    this.version = version;
+  }
+
+  public int getSourceIndex() {
+    return sourceIndex;
+  }
+
+  public int getTargetIndex() {
+    return targetIndex;
+  }
+
+  @Private
+  public void setTargetIndex(int targetIndex) {
+    this.targetIndex = targetIndex;
+  }
+
+  public int getVersion() {
+    return version;
+  }
+
+  @Private
+  public void setVersion(int version) {
+    this.version = version;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/src/main/java/org/apache/tez/engine/api/events/InputInformationEvent.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/engine/api/events/InputInformationEvent.java b/tez-api/src/main/java/org/apache/tez/engine/api/events/InputInformationEvent.java
new file mode 100644
index 0000000..3656d45
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/engine/api/events/InputInformationEvent.java
@@ -0,0 +1,41 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.engine.api.events;
+
+import org.apache.tez.engine.api.Event;
+
+/**
+ * Event used to send user specific data from the user 
+ * code in the AM to the task input
+ */
+public class InputInformationEvent extends Event {
+
+  /**
+   * User Payload for this Event
+   */
+  private final byte[] userPayload;
+  public InputInformationEvent(byte[] userPayload) {
+    this.userPayload = userPayload;
+  }
+
+  public byte[] getUserPayload() {
+    return userPayload;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/src/main/java/org/apache/tez/engine/api/events/InputReadErrorEvent.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/engine/api/events/InputReadErrorEvent.java b/tez-api/src/main/java/org/apache/tez/engine/api/events/InputReadErrorEvent.java
new file mode 100644
index 0000000..fa49b79
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/engine/api/events/InputReadErrorEvent.java
@@ -0,0 +1,65 @@
+/**
+ * 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.engine.api.events;
+
+import org.apache.tez.engine.api.Event;
+
+/**
+ * Event generated by an Input to indicate error when trying to retrieve data.
+ * This is not necessarily a fatal event - it's an indication to the AM to retry
+ * source data generation.
+ */
+public final class InputReadErrorEvent extends Event {
+
+  /**
+   * Diagnostics/trace of the error that occurred on the Input's edge.
+   */
+  private final String diagnostics;
+
+  /**
+   * Index of the physical edge on which the error occurred.
+   */
+  private final int index;
+
+  /**
+   * Version of the data on which the error occurred.
+   */
+  private final int version;
+
+  public InputReadErrorEvent(String diagnostics, int index,
+      int version) {
+    super();
+    this.diagnostics = diagnostics;
+    this.index = index;
+    this.version = version;
+  }
+
+  public String getDiagnostics() {
+    return diagnostics;
+  }
+
+  public int getIndex() {
+    return index;
+  }
+
+  public int getVersion() {
+    return version;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/src/main/java/org/apache/tez/engine/common/objectregistry/ObjectLifeCycle.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/engine/common/objectregistry/ObjectLifeCycle.java b/tez-api/src/main/java/org/apache/tez/engine/common/objectregistry/ObjectLifeCycle.java
new file mode 100644
index 0000000..7099299
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/engine/common/objectregistry/ObjectLifeCycle.java
@@ -0,0 +1,37 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.engine.common.objectregistry;
+
+/**
+ * Defines the valid lifecycle and scope of Objects stored in ObjectRegistry.
+ * Objects are guaranteed to not be valid outside of their defined life-cycle
+ * period. Objects are not guaranteed to be retained through the defined period
+ * as they may be evicted for various reasons.
+ */
+public enum ObjectLifeCycle {
+  /** Objects are valid for the lifetime of the Tez JVM/Session
+   */
+  SESSION,
+  /** Objects are valid for the lifetime of the DAG.
+   */
+  DAG,
+  /** Objects are valid for the lifetime of the Vertex.
+   */
+  VERTEX,
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/src/main/java/org/apache/tez/engine/common/objectregistry/ObjectRegistry.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/engine/common/objectregistry/ObjectRegistry.java b/tez-api/src/main/java/org/apache/tez/engine/common/objectregistry/ObjectRegistry.java
new file mode 100644
index 0000000..a27903d
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/engine/common/objectregistry/ObjectRegistry.java
@@ -0,0 +1,56 @@
+/**
+ * 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.engine.common.objectregistry;
+
+/**
+ * Preliminary version of a simple shared object cache to re-use
+ * objects across multiple tasks within the same container/JVM.
+ */
+public interface ObjectRegistry {
+
+  /**
+   * Insert or update object into the registry. This will remove an object
+   * associated with the same key with a different life-cycle as there is only
+   * one instance of an Object stored for a given key irrespective of the
+   * life-cycle attached to the Object.
+   * @param lifeCycle What life-cycle is the Object valid for
+   * @param key Key to identify the Object
+   * @param value Object to be inserted
+   * @return Previous Object associated with the key attached if present
+   * else null. Could return the same object if the object was associated with
+   * the same key for a different life-cycle.
+   */
+  public Object add(ObjectLifeCycle lifeCycle, String key, Object value);
+
+  /**
+   * Return the object associated with the provided key
+   * @param key Key to find object
+   * @return Object if found else null
+   */
+  public Object get(String key);
+
+  /**
+   * Delete the object associated with the provided key
+   * @param lifeCycle What life-cycle is the Object valid for
+   * @param key Key to find object
+   * @return True if an object was found and removed
+   */
+  public boolean delete(String key);
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/src/main/java/org/apache/tez/engine/common/objectregistry/ObjectRegistryFactory.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/engine/common/objectregistry/ObjectRegistryFactory.java b/tez-api/src/main/java/org/apache/tez/engine/common/objectregistry/ObjectRegistryFactory.java
new file mode 100644
index 0000000..94352b3
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/engine/common/objectregistry/ObjectRegistryFactory.java
@@ -0,0 +1,32 @@
+/**
+ * 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.engine.common.objectregistry;
+
+import com.google.inject.Inject;
+
+public class ObjectRegistryFactory {
+
+  @Inject
+  private static ObjectRegistry objectRegistry;
+
+  public static ObjectRegistry getObjectRegistry() {
+    return objectRegistry;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/src/main/proto/DAGApiRecords.proto
----------------------------------------------------------------------
diff --git a/tez-api/src/main/proto/DAGApiRecords.proto b/tez-api/src/main/proto/DAGApiRecords.proto
new file mode 100644
index 0000000..4385749
--- /dev/null
+++ b/tez-api/src/main/proto/DAGApiRecords.proto
@@ -0,0 +1,183 @@
+/**
+ * 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.
+ */
+
+option java_package = "org.apache.tez.dag.api.records";
+option java_outer_classname = "DAGProtos";
+option java_generate_equals_and_hash = true;
+
+// DAG plan messages
+
+// Many of these types have a dual in the Tez-api.  To reduce confusion, these types have prefix or suffix 
+// of "Plan" to indicate they are to be used in the dag-plan.
+// The big types use a suffix:  JobPlan, VertexPlan, EdgePlan 
+//   --> these get more direct use in the runtime and the naming is natural.
+// The enums and utility types use prefix: PlanVertexType, PlanEdgeConnectionPaatern, etc
+//   --> there is not great naming choice for these that avoids ambiguity, but this one seems acceptable. 
+
+enum PlanVertexType {
+  INPUT = 0;
+  NORMAL = 1;
+  OUTPUT = 2;
+}
+
+enum PlanEdgeDataMovementType {
+  ONE_TO_ONE = 0;
+  BROADCAST = 1;
+  SCATTER_GATHER = 2;
+}
+
+enum PlanEdgeDataSourceType {
+  PERSISTED = 0;
+  PERSISTED_RELIABLE = 1;
+  EPHEMERAL = 2;
+}
+
+enum PlanEdgeSchedulingType {
+  SEQUENTIAL = 0;
+  CONCURRENT = 1;
+}
+
+message PlanKeyValuePair {
+  required string key = 1;
+  required string value = 2;
+}
+
+enum PlanLocalResourceType {
+  FILE = 0;
+  ARCHIVE = 1;
+  PATTERN = 2;
+}
+
+enum PlanLocalResourceVisibility {
+  PUBLIC = 0;
+  PRIVATE = 1;
+  APPLICATION = 2;
+}
+
+message PlanLocalResource {
+  required string name = 1;
+  required string uri = 2;
+  required int64 size = 3;
+  required int64 timeStamp = 4;
+  required PlanLocalResourceType type = 5;
+  required PlanLocalResourceVisibility visibility = 6;
+  optional string pattern = 7; // only used if type=PATTERN
+}
+
+// Each taskLocationHint represents a single split in in the input.
+// It is the list of [{rack,machines}] that host a replica of each particular split.
+// For now it is represented as pair-of-arrays rather than array-of-pairs.
+message PlanTaskLocationHint {
+  repeated string rack = 1;
+  repeated string host = 2;
+}
+
+message PlanTaskConfiguration {
+  required int32 numTasks = 1;
+  required int32 memoryMb = 2;
+  required int32 virtualCores = 3;
+  required string javaOpts = 4;
+  required string taskModule = 5;
+  repeated PlanLocalResource localResource = 6;
+  repeated PlanKeyValuePair environmentSetting = 8;  
+}
+
+message TezEntityDescriptorProto {
+  optional string class_name = 1;
+  optional bytes user_payload = 2;
+}
+
+message VertexPlan {
+  required string name = 1;
+  required PlanVertexType type = 2;
+  optional TezEntityDescriptorProto processor_descriptor = 3;
+  required PlanTaskConfiguration taskConfig = 4;
+  repeated PlanTaskLocationHint taskLocationHint = 7;
+  repeated string inEdgeId = 8;
+  repeated string outEdgeId = 9;
+}
+
+message EdgePlan {
+  required string id = 1;
+  required string inputVertexName = 2;
+  required string outputVertexName = 3;
+  required PlanEdgeDataMovementType dataMovementType = 4;
+  required PlanEdgeDataSourceType dataSourceType = 5;
+  required PlanEdgeSchedulingType schedulingType = 6;
+  optional TezEntityDescriptorProto edge_source = 7;
+  optional TezEntityDescriptorProto edge_destination = 8;
+}
+
+message ConfigurationProto {
+  repeated PlanKeyValuePair confKeyValues = 1;
+}
+
+message DAGPlan {
+  required string name = 1;
+  repeated VertexPlan vertex = 2;
+  repeated EdgePlan edge = 3;
+  optional ConfigurationProto dagKeyValues = 4;
+}
+
+// DAG monitoring messages
+message ProgressProto {
+  optional int32 totalTaskCount = 1;
+  optional int32 succeededTaskCount = 2;
+  optional int32 runningTaskCount = 3;
+  optional int32 failedTaskCount = 4;
+  optional int32 killedTaskCount = 5;
+}
+
+enum VertexStatusStateProto {
+  VERTEX_INITED = 0;
+  VERTEX_RUNNING = 1;
+  VERTEX_SUCCEEDED = 2;
+  VERTEX_KILLED = 3;
+  VERTEX_FAILED = 4;
+  VERTEX_ERROR = 5;
+  VERTEX_TERMINATING = 6;
+}
+
+message VertexStatusProto {
+  optional VertexStatusStateProto state = 1;
+  repeated string diagnostics = 2;
+  optional ProgressProto progress = 3;
+}
+
+enum DAGStatusStateProto {
+  DAG_SUBMITTED = 0;
+  DAG_INITING = 1;
+  DAG_RUNNING = 2;
+  DAG_SUCCEEDED = 3;
+  DAG_KILLED = 4;
+  DAG_FAILED = 5;
+  DAG_ERROR = 6;
+  DAG_TERMINATING = 7;
+}
+
+message StringProgressPairProto {
+  required string key = 1;
+  required ProgressProto progress = 2;
+}
+
+message DAGStatusProto {
+  optional DAGStatusStateProto state = 1;
+  repeated string diagnostics = 2;
+  optional ProgressProto DAGProgress = 3;
+  repeated StringProgressPairProto vertexProgress = 4;  
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/src/main/proto/DAGClientAMProtocol.proto
----------------------------------------------------------------------
diff --git a/tez-api/src/main/proto/DAGClientAMProtocol.proto b/tez-api/src/main/proto/DAGClientAMProtocol.proto
new file mode 100644
index 0000000..6fcd1f8
--- /dev/null
+++ b/tez-api/src/main/proto/DAGClientAMProtocol.proto
@@ -0,0 +1,81 @@
+/**
+ * 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.
+ */
+
+option java_package = "org.apache.tez.dag.api.client.rpc";
+option java_outer_classname = "DAGClientAMProtocolRPC";
+option java_generic_services = true;
+option java_generate_equals_and_hash = true;
+
+//import "DAGClientAMProtocolRecords.proto";
+
+import "DAGApiRecords.proto";
+
+message GetAllDAGsRequestProto {
+}
+
+message GetAllDAGsResponseProto {
+  repeated string dagId = 1;
+}
+
+message GetDAGStatusRequestProto {
+  optional string dagId = 1;
+}
+
+message GetDAGStatusResponseProto {
+  optional DAGStatusProto dagStatus = 1;
+}
+
+message GetVertexStatusRequestProto {
+  optional string dagId = 1;
+  optional string vertexName = 2;
+}
+
+message GetVertexStatusResponseProto {
+  optional VertexStatusProto vertexStatus = 1;
+}
+
+message TryKillDAGRequestProto {
+  optional string dagId = 1;
+}
+
+message TryKillDAGResponseProto {
+  //nothing yet
+}
+
+message SubmitDAGRequestProto {
+  optional DAGPlan d_a_g_plan = 1;
+}
+
+message SubmitDAGResponseProto {
+  optional string dagId = 1;
+}
+
+message ShutdownSessionRequestProto {
+}
+
+message ShutdownSessionResponseProto {
+}
+
+service DAGClientAMProtocol {
+  rpc getAllDAGs (GetAllDAGsRequestProto) returns (GetAllDAGsResponseProto);
+  rpc getDAGStatus (GetDAGStatusRequestProto) returns (GetDAGStatusResponseProto);
+  rpc getVertexStatus (GetVertexStatusRequestProto) returns (GetVertexStatusResponseProto);
+  rpc tryKillDAG (TryKillDAGRequestProto) returns (TryKillDAGResponseProto);
+  rpc submitDAG (SubmitDAGRequestProto) returns (SubmitDAGResponseProto);
+  rpc shutdownSession (ShutdownSessionRequestProto) returns (ShutdownSessionResponseProto);
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/src/main/proto/Events.proto
----------------------------------------------------------------------
diff --git a/tez-api/src/main/proto/Events.proto b/tez-api/src/main/proto/Events.proto
new file mode 100644
index 0000000..21cacf6
--- /dev/null
+++ b/tez-api/src/main/proto/Events.proto
@@ -0,0 +1,44 @@
+/**
+ * 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.
+ */
+
+option java_package = "org.apache.tez.engine.api.events";
+option java_outer_classname = "EventProtos";
+option java_generate_equals_and_hash = true;
+
+message DataMovementEventProto {
+  optional int32 source_index = 1;
+  optional int32 target_index = 2;
+  optional bytes user_payload = 3;
+  optional int32 version = 4;
+}
+
+message InputReadErrorEventProto {
+  optional int32 index = 1;
+  optional string diagnostics = 2;
+  optional int32 version = 3;
+}
+
+message InputFailedEventProto {
+  optional int32 source_index = 1;
+  optional int32 target_index = 2;
+  optional int32 version = 4;
+}
+
+message InputInformationEventProto {
+  optional bytes user_payload = 1;
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/src/test/java/org/apache/tez/dag/api/TestDAGPlan.java
----------------------------------------------------------------------
diff --git a/tez-api/src/test/java/org/apache/tez/dag/api/TestDAGPlan.java b/tez-api/src/test/java/org/apache/tez/dag/api/TestDAGPlan.java
new file mode 100644
index 0000000..53ec357
--- /dev/null
+++ b/tez-api/src/test/java/org/apache/tez/dag/api/TestDAGPlan.java
@@ -0,0 +1,155 @@
+/**
+ * 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.api;
+
+import static org.junit.Assert.*;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.util.HashMap;
+
+import org.apache.hadoop.yarn.api.records.LocalResource;
+import org.apache.hadoop.yarn.api.records.Resource;
+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.records.DAGProtos.DAGPlan;
+import org.apache.tez.dag.api.records.DAGProtos.EdgePlan;
+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.VertexPlan;
+import org.junit.Assert;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+// based on TestDAGLocationHint
+public class TestDAGPlan {
+  @Rule
+  public TemporaryFolder tempFolder = new TemporaryFolder(); //TODO: doesn't seem to be deleting this folder automatically as expected.
+
+  @Test
+  public void testBasicJobPlanSerde() throws IOException {
+
+    DAGPlan job = DAGPlan.newBuilder()
+       .setName("test")
+       .addVertex(
+           VertexPlan.newBuilder()
+             .setName("vertex1")
+             .setType(PlanVertexType.NORMAL)
+             .addTaskLocationHint(PlanTaskLocationHint.newBuilder().addHost("machineName").addRack("rack1").build())
+             .setTaskConfig(
+                 PlanTaskConfiguration.newBuilder()
+                   .setNumTasks(2)
+                   .setVirtualCores(4)
+                   .setMemoryMb(1024)
+                   .setJavaOpts("")
+                   .setTaskModule("x.y")
+                   .build())
+             .build())
+        .build();
+   File file = tempFolder.newFile("jobPlan");
+   FileOutputStream outStream = null;
+   try {
+     outStream = new FileOutputStream(file);
+     job.writeTo(outStream);
+   }
+   finally {
+     if(outStream != null){
+       outStream.close();
+     }
+   }
+
+   DAGPlan inJob;
+   FileInputStream inputStream;
+   try {
+     inputStream = new FileInputStream(file);
+     inJob = DAGPlan.newBuilder().mergeFrom(inputStream).build();
+   }
+   finally {
+     outStream.close();
+   }
+
+   Assert.assertEquals(job, inJob);
+  }
+
+  @Test
+  public void testUserPayloadSerde() {
+    DAG dag = new DAG("testDag");
+    ProcessorDescriptor pd1 = new ProcessorDescriptor("processor1").
+        setUserPayload("processor1Bytes".getBytes());
+    ProcessorDescriptor pd2 = new ProcessorDescriptor("processor2").
+        setUserPayload("processor2Bytes".getBytes());
+    Vertex v1 = new Vertex("v1", pd1, 10, Resource.newInstance(1024, 1));
+    Vertex v2 = new Vertex("v2", pd2, 1, Resource.newInstance(1024, 1));
+    v1.setJavaOpts("").setTaskEnvironment(new HashMap<String, String>())
+        .setTaskLocalResources(new HashMap<String, LocalResource>());
+    v2.setJavaOpts("").setTaskEnvironment(new HashMap<String, String>())
+        .setTaskLocalResources(new HashMap<String, LocalResource>());
+
+    InputDescriptor inputDescriptor = new InputDescriptor("input").
+        setUserPayload("inputBytes".getBytes());
+    OutputDescriptor outputDescriptor = new OutputDescriptor("output").
+        setUserPayload("outputBytes".getBytes());
+    Edge edge = new Edge(v1, v2, new EdgeProperty(
+        DataMovementType.SCATTER_GATHER, DataSourceType.PERSISTED,
+        SchedulingType.SEQUENTIAL, outputDescriptor, inputDescriptor));
+
+    dag.addVertex(v1).addVertex(v2).addEdge(edge);
+
+    DAGPlan dagProto = dag.createDag(new TezConfiguration());
+
+    assertEquals(2, dagProto.getVertexCount());
+    assertEquals(1, dagProto.getEdgeCount());
+
+    VertexPlan v1Proto = dagProto.getVertex(0);
+    VertexPlan v2Proto = dagProto.getVertex(1);
+    EdgePlan edgeProto = dagProto.getEdge(0);
+
+    assertEquals("processor1Bytes", new String(v1Proto.getProcessorDescriptor()
+        .getUserPayload().toByteArray()));
+    assertEquals("processor1", v1Proto.getProcessorDescriptor().getClassName());
+
+    assertEquals("processor2Bytes", new String(v2Proto.getProcessorDescriptor()
+        .getUserPayload().toByteArray()));
+    assertEquals("processor2", v2Proto.getProcessorDescriptor().getClassName());
+
+    assertEquals("inputBytes", new String(edgeProto.getEdgeDestination()
+        .getUserPayload().toByteArray()));
+    assertEquals("input", edgeProto.getEdgeDestination().getClassName());
+
+    assertEquals("outputBytes", new String(edgeProto.getEdgeSource()
+        .getUserPayload().toByteArray()));
+    assertEquals("output", edgeProto.getEdgeSource().getClassName());
+
+    EdgeProperty edgeProperty = DagTypeConverters
+        .createEdgePropertyMapFromDAGPlan(dagProto.getEdgeList().get(0));
+
+    byte[] ib = edgeProperty.getEdgeDestination().getUserPayload();
+    assertEquals("inputBytes", new String(ib));
+    assertEquals("input", edgeProperty.getEdgeDestination().getClassName());
+
+    byte[] ob = edgeProperty.getEdgeSource().getUserPayload();
+    assertEquals("outputBytes", new String(ob));
+    assertEquals("output", edgeProperty.getEdgeSource().getClassName());
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/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
new file mode 100644
index 0000000..b33f3a6
--- /dev/null
+++ b/tez-api/src/test/java/org/apache/tez/dag/api/TestDAGVerify.java
@@ -0,0 +1,417 @@
+/**
+ * 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.api;
+
+import org.apache.hadoop.yarn.api.records.Resource;
+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.junit.Assert;
+import org.junit.Test;
+
+public class TestDAGVerify {
+
+  private final String dummyProcessorClassName = TestDAGVerify.class.getName();
+  private final String dummyInputClassName = TestDAGVerify.class.getName();
+  private final String dummyOutputClassName = TestDAGVerify.class.getName();
+  private final int dummyTaskCount = 2;
+  private final Resource dummyTaskResource = Resource.newInstance(1, 1);
+
+  //    v1
+  //    |
+  //    v2
+  @Test
+  public void testVerify1() {
+    Vertex v1 = new Vertex("v1",
+        new ProcessorDescriptor(dummyProcessorClassName),
+        dummyTaskCount, dummyTaskResource);
+    Vertex v2 = new Vertex("v2",
+        new ProcessorDescriptor("MapProcessor"),
+        dummyTaskCount, dummyTaskResource);
+    Edge e1 = new Edge(v1, v2,
+        new EdgeProperty(DataMovementType.SCATTER_GATHER, 
+            DataSourceType.PERSISTED, SchedulingType.SEQUENTIAL, 
+            new OutputDescriptor(dummyOutputClassName),
+            new InputDescriptor(dummyInputClassName)));
+    DAG dag = new DAG("testDag");
+    dag.addVertex(v1);
+    dag.addVertex(v2);
+    dag.addEdge(e1);
+    dag.verify();
+  }
+
+  @Test(expected = IllegalStateException.class)  
+  public void testVerify2() {
+    Vertex v1 = new Vertex("v1",
+        new ProcessorDescriptor(dummyProcessorClassName),
+        dummyTaskCount, dummyTaskResource);
+    Vertex v2 = new Vertex("v2",
+        new ProcessorDescriptor("MapProcessor"),
+        dummyTaskCount, dummyTaskResource);
+    Edge e1 = new Edge(v1, v2,
+        new EdgeProperty(DataMovementType.ONE_TO_ONE, 
+            DataSourceType.PERSISTED, SchedulingType.SEQUENTIAL, 
+            new OutputDescriptor(dummyOutputClassName),
+            new InputDescriptor(dummyInputClassName)));
+    DAG dag = new DAG("testDag");
+    dag.addVertex(v1);
+    dag.addVertex(v2);
+    dag.addEdge(e1);
+    dag.verify();
+  }
+
+  @Test(expected = IllegalStateException.class)  
+  public void testVerify3() {
+    Vertex v1 = new Vertex("v1",
+        new ProcessorDescriptor(dummyProcessorClassName),
+        dummyTaskCount, dummyTaskResource);
+    Vertex v2 = new Vertex("v2",
+        new ProcessorDescriptor("MapProcessor"),
+        dummyTaskCount, dummyTaskResource);
+    Edge e1 = new Edge(v1, v2,
+        new EdgeProperty(DataMovementType.SCATTER_GATHER, 
+            DataSourceType.EPHEMERAL, SchedulingType.SEQUENTIAL, 
+            new OutputDescriptor(dummyOutputClassName),
+            new InputDescriptor(dummyInputClassName)));
+    DAG dag = new DAG("testDag");
+    dag.addVertex(v1);
+    dag.addVertex(v2);
+    dag.addEdge(e1);
+    dag.verify();
+  }
+
+  @Test(expected = IllegalStateException.class)  
+  public void testVerify4() {
+    Vertex v1 = new Vertex("v1",
+        new ProcessorDescriptor(dummyProcessorClassName),
+        dummyTaskCount, dummyTaskResource);
+    Vertex v2 = new Vertex("v2",
+        new ProcessorDescriptor("MapProcessor"),
+        dummyTaskCount, dummyTaskResource);
+    Edge e1 = new Edge(v1, v2,
+        new EdgeProperty(DataMovementType.SCATTER_GATHER, 
+            DataSourceType.EPHEMERAL, SchedulingType.CONCURRENT, 
+            new OutputDescriptor(dummyOutputClassName),
+            new InputDescriptor(dummyInputClassName)));
+    DAG dag = new DAG("testDag");
+    dag.addVertex(v1);
+    dag.addVertex(v2);
+    dag.addEdge(e1);
+    dag.verify();
+  }
+
+  //    v1 <----
+  //      |     ^
+  //       v2   ^
+  //      |  |  ^
+  //    v3    v4
+  @Test
+  public void testCycle1() {
+    IllegalStateException ex=null;
+    Vertex v1 = new Vertex("v1",
+        new ProcessorDescriptor("MapProcessor"),
+        dummyTaskCount, dummyTaskResource);
+    Vertex v2 = new Vertex("v2",
+        new ProcessorDescriptor("MapProcessor"),
+        dummyTaskCount, dummyTaskResource);
+    Vertex v3 = new Vertex("v3",
+        new ProcessorDescriptor("MapProcessor"),
+        dummyTaskCount, dummyTaskResource);
+    Vertex v4 = new Vertex("v4",
+        new ProcessorDescriptor("MapProcessor"),
+        dummyTaskCount, dummyTaskResource);
+    Edge e1 = new Edge(v1, v2,
+        new EdgeProperty(DataMovementType.SCATTER_GATHER, 
+            DataSourceType.PERSISTED, SchedulingType.SEQUENTIAL, 
+            new OutputDescriptor("dummy output class"),
+            new InputDescriptor("dummy input class")));
+    Edge e2 = new Edge(v2, v3,
+        new EdgeProperty(DataMovementType.SCATTER_GATHER, 
+            DataSourceType.PERSISTED, SchedulingType.SEQUENTIAL, 
+            new OutputDescriptor("dummy output class"),
+            new InputDescriptor("dummy input class")));
+    Edge e3 = new Edge(v2, v4,
+        new EdgeProperty(DataMovementType.SCATTER_GATHER, 
+            DataSourceType.PERSISTED, SchedulingType.SEQUENTIAL, 
+            new OutputDescriptor("dummy output class"),
+            new InputDescriptor("dummy input class")));
+    Edge e4 = new Edge(v4, v1,
+        new EdgeProperty(DataMovementType.SCATTER_GATHER, 
+            DataSourceType.PERSISTED, SchedulingType.SEQUENTIAL, 
+            new OutputDescriptor("dummy output class"),
+            new InputDescriptor("dummy input class")));
+    DAG dag = new DAG("testDag");
+    dag.addVertex(v1);
+    dag.addVertex(v2);
+    dag.addVertex(v3);
+    dag.addVertex(v4);
+    dag.addEdge(e1);
+    dag.addEdge(e2);
+    dag.addEdge(e3);
+    dag.addEdge(e4);
+    try{
+      dag.verify();
+    }
+    catch (IllegalStateException e){
+      ex = e;
+    }
+    Assert.assertNotNull(ex);
+    System.out.println(ex.getMessage());
+    Assert.assertTrue(ex.getMessage().startsWith("DAG contains a cycle"));
+  }
+
+  //     v1
+  //      |
+  //    -> v2
+  //    ^  | |
+  //    v3    v4
+  @Test
+  public void testCycle2() {
+    IllegalStateException ex=null;
+    Vertex v1 = new Vertex("v1",
+        new ProcessorDescriptor("MapProcessor"),
+        dummyTaskCount, dummyTaskResource);
+    Vertex v2 = new Vertex("v2",
+        new ProcessorDescriptor("MapProcessor"),
+        dummyTaskCount, dummyTaskResource);
+    Vertex v3 = new Vertex("v3",
+        new ProcessorDescriptor("MapProcessor"),
+        dummyTaskCount, dummyTaskResource);
+    Vertex v4 = new Vertex("v4",
+        new ProcessorDescriptor("MapProcessor"),
+        dummyTaskCount, dummyTaskResource);
+    Edge e1 = new Edge(v1, v2,
+        new EdgeProperty(DataMovementType.SCATTER_GATHER, 
+            DataSourceType.PERSISTED, SchedulingType.SEQUENTIAL, 
+            new OutputDescriptor("dummy output class"),
+            new InputDescriptor("dummy input class")));
+    Edge e2 = new Edge(v2, v3,
+        new EdgeProperty(DataMovementType.SCATTER_GATHER, 
+            DataSourceType.PERSISTED, SchedulingType.SEQUENTIAL, 
+            new OutputDescriptor("dummy output class"),
+            new InputDescriptor("dummy input class")));
+    Edge e3 = new Edge(v2, v4,
+        new EdgeProperty(DataMovementType.SCATTER_GATHER, 
+            DataSourceType.PERSISTED, SchedulingType.SEQUENTIAL, 
+            new OutputDescriptor("dummy output class"),
+            new InputDescriptor("dummy input class")));
+    Edge e4 = new Edge(v3, v2,
+        new EdgeProperty(DataMovementType.SCATTER_GATHER, 
+            DataSourceType.PERSISTED, SchedulingType.SEQUENTIAL, 
+            new OutputDescriptor("dummy output class"),
+            new InputDescriptor("dummy input class")));
+    DAG dag = new DAG("testDag");
+    dag.addVertex(v1);
+    dag.addVertex(v2);
+    dag.addVertex(v3);
+    dag.addVertex(v4);
+    dag.addEdge(e1);
+    dag.addEdge(e2);
+    dag.addEdge(e3);
+    dag.addEdge(e4);
+    try{
+      dag.verify();
+    }
+    catch (IllegalStateException e){
+      ex = e;
+    }
+    Assert.assertNotNull(ex);
+    System.out.println(ex.getMessage());
+    Assert.assertTrue(ex.getMessage().startsWith("DAG contains a cycle"));
+  }
+
+  @Test
+  public void repeatedVertexName() {
+    IllegalStateException ex=null;
+    Vertex v1 = new Vertex("v1",
+        new ProcessorDescriptor("MapProcessor"),
+        dummyTaskCount, dummyTaskResource);
+    Vertex v1repeat = new Vertex("v1",
+        new ProcessorDescriptor("MapProcessor"),
+        dummyTaskCount, dummyTaskResource);
+    DAG dag = new DAG("testDag");
+    dag.addVertex(v1);
+    dag.addVertex(v1repeat);
+    try {
+      dag.verify();
+    }
+    catch (IllegalStateException e){
+      ex = e;
+    }
+    Assert.assertNotNull(ex);
+    System.out.println(ex.getMessage());
+    Assert.assertTrue(ex.getMessage().startsWith("DAG contains multiple vertices with name"));
+  }
+
+  //  v1  v2
+  //   |  |
+  //    v3
+  @Test
+  public void BinaryInputDisallowed() {
+    IllegalStateException ex=null;
+    try {
+      Vertex v1 = new Vertex("v1",
+          new ProcessorDescriptor("MapProcessor"),
+          dummyTaskCount, dummyTaskResource);
+      Vertex v2 = new Vertex("v2",
+          new ProcessorDescriptor("MapProcessor"),
+          dummyTaskCount, dummyTaskResource);
+      Vertex v3 = new Vertex("v3",
+          new ProcessorDescriptor("ReduceProcessor"),
+          dummyTaskCount, dummyTaskResource);
+      Edge e1 = new Edge(v1, v3,
+          new EdgeProperty(DataMovementType.ONE_TO_ONE, 
+              DataSourceType.PERSISTED, SchedulingType.SEQUENTIAL, 
+              new OutputDescriptor("dummy output class"),
+              new InputDescriptor("dummy input class")));
+      Edge e2 = new Edge(v2, v3,
+          new EdgeProperty(DataMovementType.SCATTER_GATHER, 
+              DataSourceType.PERSISTED, SchedulingType.SEQUENTIAL, 
+              new OutputDescriptor("dummy output class"),
+              new InputDescriptor("dummy input class")));
+      DAG dag = new DAG("testDag");
+      dag.addVertex(v1);
+      dag.addVertex(v2);
+      dag.addVertex(v3);
+      dag.addEdge(e1);
+      dag.addEdge(e2);
+      dag.verify();
+    }
+    catch (IllegalStateException e){
+      ex = e;
+    }
+    Assert.assertNotNull(ex);
+    System.out.println(ex.getMessage());
+    Assert.assertTrue(ex.getMessage().startsWith(
+        "Unsupported connection pattern on edge"));
+  }
+
+  //  v1  v2
+  //   |  |
+  //    v3
+  @Test
+  public void BinaryInputAllowed() {
+    Vertex v1 = new Vertex("v1",
+        new ProcessorDescriptor("MapProcessor"),
+        dummyTaskCount, dummyTaskResource);
+    Vertex v2 = new Vertex("v2",
+        new ProcessorDescriptor("MapProcessor"),
+        dummyTaskCount, dummyTaskResource);
+    Vertex v3 = new Vertex("v3",
+        new ProcessorDescriptor("ReduceProcessor"),
+        dummyTaskCount, dummyTaskResource);
+    Edge e1 = new Edge(v1, v3,
+        new EdgeProperty(DataMovementType.SCATTER_GATHER, 
+            DataSourceType.PERSISTED, SchedulingType.SEQUENTIAL,
+            new OutputDescriptor("dummy output class"),
+            new InputDescriptor("dummy input class")));
+    Edge e2 = new Edge(v2, v3,
+        new EdgeProperty(DataMovementType.SCATTER_GATHER, 
+            DataSourceType.PERSISTED, SchedulingType.SEQUENTIAL, 
+            new OutputDescriptor("dummy output class"),
+            new InputDescriptor("dummy input class")));
+    DAG dag = new DAG("testDag");
+    dag.addVertex(v1);
+    dag.addVertex(v2);
+    dag.addVertex(v3);
+    dag.addEdge(e1);
+    dag.addEdge(e2);
+    dag.verify();
+  }
+
+  //   v1
+  //  |  |
+  //  v2  v3
+  @Test
+  public void BinaryOutput() {
+    IllegalStateException ex=null;
+    try {
+      Vertex v1 = new Vertex("v1",
+          new ProcessorDescriptor("MapProcessor"),
+          dummyTaskCount, dummyTaskResource);
+      Vertex v2 = new Vertex("v2",
+          new ProcessorDescriptor("MapProcessor"),
+          dummyTaskCount, dummyTaskResource);
+      Vertex v3 = new Vertex("v3",
+          new ProcessorDescriptor("MapProcessor"),
+          dummyTaskCount, dummyTaskResource);
+      Edge e1 = new Edge(v1, v2,
+          new EdgeProperty(DataMovementType.SCATTER_GATHER, 
+              DataSourceType.PERSISTED, SchedulingType.SEQUENTIAL, 
+              new OutputDescriptor("dummy output class"),
+              new InputDescriptor("dummy input class")));
+      Edge e2 = new Edge(v1, v2,
+          new EdgeProperty(DataMovementType.SCATTER_GATHER, 
+              DataSourceType.PERSISTED, SchedulingType.SEQUENTIAL, 
+              new OutputDescriptor("dummy output class"),
+              new InputDescriptor("dummy input class")));
+      DAG dag = new DAG("testDag");
+      dag.addVertex(v1);
+      dag.addVertex(v2);
+      dag.addVertex(v3);
+      dag.addEdge(e1);
+      dag.addEdge(e2);
+      dag.verify();
+    }
+    catch (IllegalStateException e){
+      ex = e;
+    }
+    Assert.assertNotNull(ex);
+    System.out.println(ex.getMessage());
+    Assert.assertTrue(ex.getMessage().startsWith("Vertex has outDegree>1"));
+  }
+
+  @Test
+  public void testDagWithNoVertices() {
+    IllegalStateException ex=null;
+    try {
+      DAG dag = new DAG("testDag");
+      dag.verify();
+    }
+    catch (IllegalStateException e){
+      ex = e;
+    }
+    Assert.assertNotNull(ex);
+    System.out.println(ex.getMessage());
+    Assert.assertTrue(ex.getMessage()
+        .startsWith("Invalid dag containing 0 vertices"));
+  }
+
+  @SuppressWarnings("unused")
+  @Test
+  public void testInvalidVertexConstruction() {
+    try {
+      Vertex v1 = new Vertex("v1",
+          new ProcessorDescriptor("MapProcessor"),
+          0, dummyTaskResource);
+      Assert.fail("Expected exception for 0 parallelism");
+    } catch (IllegalArgumentException e) {
+      Assert.assertTrue(e.getMessage().startsWith("Parallelism cannot be 0"));
+    }
+    try {
+      Vertex v1 = new Vertex("v1",
+          new ProcessorDescriptor("MapProcessor"),
+          1, null);
+      Assert.fail("Expected exception for 0 parallelism");
+    } catch (IllegalArgumentException e) {
+      Assert.assertTrue(e.getMessage().startsWith("Resource cannot be null"));
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-common/pom.xml
----------------------------------------------------------------------
diff --git a/tez-common/pom.xml b/tez-common/pom.xml
index 87f18d6..b4882bb 100644
--- a/tez-common/pom.xml
+++ b/tez-common/pom.xml
@@ -39,7 +39,7 @@
     </dependency>
     <dependency>
       <groupId>org.apache.tez</groupId>
-      <artifactId>tez-dag-api</artifactId>
+      <artifactId>tez-api</artifactId>
     </dependency>
   </dependencies>
 


[50/50] [abbrv] git commit: Merge branch 'TEZ-398' into master.

Posted by ss...@apache.org.
Merge branch 'TEZ-398' into master.


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

Branch: refs/heads/master
Commit: f1cbb1c60a73add41f93c0f455a803ea0d165aee
Parents: 92f458b 3749a18
Author: Siddharth Seth <ss...@apache.org>
Authored: Wed Sep 25 00:30:12 2013 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Wed Sep 25 00:30:12 2013 -0700

----------------------------------------------------------------------
 pom.xml                                         |   14 +-
 tez-api/findbugs-exclude.xml                    |   16 +
 tez-api/pom.xml                                 |   90 +
 .../org/apache/tez/client/AMConfiguration.java  |  100 +
 .../java/org/apache/tez/client/TezClient.java   |  144 ++
 .../org/apache/tez/client/TezClientUtils.java   |  560 ++++++
 .../java/org/apache/tez/client/TezSession.java  |  184 ++
 .../tez/client/TezSessionConfiguration.java     |   57 +
 .../org/apache/tez/common/TezJobConfig.java     |  316 +++
 .../tez/common/counters/AbstractCounter.java    |   52 +
 .../common/counters/AbstractCounterGroup.java   |  208 ++
 .../tez/common/counters/AbstractCounters.java   |  385 ++++
 .../tez/common/counters/CounterGroup.java       |   32 +
 .../tez/common/counters/CounterGroupBase.java   |  108 ++
 .../common/counters/CounterGroupFactory.java    |  180 ++
 .../apache/tez/common/counters/DAGCounter.java  |   39 +
 .../tez/common/counters/FileSystemCounter.java  |   30 +
 .../common/counters/FileSystemCounterGroup.java |  327 ++++
 .../common/counters/FrameworkCounterGroup.java  |  275 +++
 .../tez/common/counters/GenericCounter.java     |  109 ++
 .../apache/tez/common/counters/JobCounter.java  |   45 +
 .../common/counters/LimitExceededException.java |   36 +
 .../org/apache/tez/common/counters/Limits.java  |  112 ++
 .../tez/common/counters/ResourceBundles.java    |   94 +
 .../apache/tez/common/counters/TaskCounter.java |   66 +
 .../apache/tez/common/counters/TezCounter.java  |   83 +
 .../apache/tez/common/counters/TezCounters.java |  144 ++
 .../main/java/org/apache/tez/dag/api/DAG.java   |  377 ++++
 .../apache/tez/dag/api/DagTypeConverters.java   |  278 +++
 .../main/java/org/apache/tez/dag/api/Edge.java  |   59 +
 .../org/apache/tez/dag/api/EdgeProperty.java    |  147 ++
 .../org/apache/tez/dag/api/InputDescriptor.java |   32 +
 .../apache/tez/dag/api/OutputDescriptor.java    |   32 +
 .../apache/tez/dag/api/ProcessorDescriptor.java |   31 +
 .../apache/tez/dag/api/TezConfiguration.java    |  223 +++
 .../org/apache/tez/dag/api/TezConstants.java    |   29 +
 .../apache/tez/dag/api/TezEntityDescriptor.java |   42 +
 .../org/apache/tez/dag/api/TezException.java    |   31 +
 .../tez/dag/api/TezUncheckedException.java      |   33 +
 .../java/org/apache/tez/dag/api/Vertex.java     |  153 ++
 .../apache/tez/dag/api/VertexLocationHint.java  |  154 ++
 .../apache/tez/dag/api/client/DAGClient.java    |   67 +
 .../apache/tez/dag/api/client/DAGStatus.java    |  130 ++
 .../org/apache/tez/dag/api/client/Progress.java |   67 +
 .../apache/tez/dag/api/client/VertexStatus.java |   78 +
 .../rpc/DAGClientAMProtocolBlockingPB.java      |   30 +
 .../dag/api/client/rpc/DAGClientRPCImpl.java    |  291 +++
 .../java/org/apache/tez/runtime/api/Event.java  |   28 +
 .../java/org/apache/tez/runtime/api/Input.java  |   71 +
 .../tez/runtime/api/LogicalIOProcessor.java     |   43 +
 .../apache/tez/runtime/api/LogicalInput.java    |   37 +
 .../apache/tez/runtime/api/LogicalOutput.java   |   36 +
 .../java/org/apache/tez/runtime/api/Output.java |   71 +
 .../org/apache/tez/runtime/api/Processor.java   |   55 +
 .../java/org/apache/tez/runtime/api/Reader.java |   26 +
 .../apache/tez/runtime/api/TezInputContext.java |   32 +
 .../tez/runtime/api/TezOutputContext.java       |   33 +
 .../tez/runtime/api/TezProcessorContext.java    |   41 +
 .../apache/tez/runtime/api/TezTaskContext.java  |  130 ++
 .../java/org/apache/tez/runtime/api/Writer.java |   26 +
 .../runtime/api/events/DataMovementEvent.java   |  109 ++
 .../runtime/api/events/InputFailedEvent.java    |   89 +
 .../api/events/InputInformationEvent.java       |   41 +
 .../runtime/api/events/InputReadErrorEvent.java |   65 +
 .../common/objectregistry/ObjectLifeCycle.java  |   37 +
 .../common/objectregistry/ObjectRegistry.java   |   56 +
 .../objectregistry/ObjectRegistryFactory.java   |   32 +
 tez-api/src/main/proto/DAGApiRecords.proto      |  183 ++
 .../src/main/proto/DAGClientAMProtocol.proto    |   81 +
 tez-api/src/main/proto/Events.proto             |   44 +
 .../org/apache/tez/dag/api/TestDAGPlan.java     |  155 ++
 .../org/apache/tez/dag/api/TestDAGVerify.java   |  417 ++++
 tez-common/pom.xml                              |    2 +-
 .../java/org/apache/tez/common/Constants.java   |   54 -
 .../org/apache/tez/common/ContainerContext.java |   64 -
 .../org/apache/tez/common/ContainerTask.java    |  102 -
 .../java/org/apache/tez/common/IDUtils.java     |    3 +-
 .../java/org/apache/tez/common/InputSpec.java   |   85 -
 .../java/org/apache/tez/common/OutputSpec.java  |   84 -
 .../org/apache/tez/common/TezJobConfig.java     |  304 ---
 .../org/apache/tez/common/TezTaskContext.java   |   88 -
 .../org/apache/tez/common/TezTaskStatus.java    |  104 -
 .../tez/common/counters/AbstractCounter.java    |   52 -
 .../common/counters/AbstractCounterGroup.java   |  208 --
 .../tez/common/counters/AbstractCounters.java   |  385 ----
 .../tez/common/counters/CounterGroup.java       |   32 -
 .../tez/common/counters/CounterGroupBase.java   |  108 --
 .../common/counters/CounterGroupFactory.java    |  180 --
 .../apache/tez/common/counters/DAGCounter.java  |   39 -
 .../tez/common/counters/FileSystemCounter.java  |   30 -
 .../common/counters/FileSystemCounterGroup.java |  327 ----
 .../common/counters/FrameworkCounterGroup.java  |  275 ---
 .../tez/common/counters/GenericCounter.java     |  109 --
 .../apache/tez/common/counters/JobCounter.java  |   45 -
 .../common/counters/LimitExceededException.java |   36 -
 .../org/apache/tez/common/counters/Limits.java  |  112 --
 .../tez/common/counters/ResourceBundles.java    |   94 -
 .../apache/tez/common/counters/TaskCounter.java |   65 -
 .../apache/tez/common/counters/TezCounter.java  |   83 -
 .../apache/tez/common/counters/TezCounters.java |  144 --
 .../org/apache/tez/records/TezContainerId.java  |   78 -
 tez-dag-api/findbugs-exclude.xml                |   16 -
 tez-dag-api/pom.xml                             |   88 -
 .../org/apache/tez/client/AMConfiguration.java  |  100 -
 .../java/org/apache/tez/client/TezClient.java   |  144 --
 .../org/apache/tez/client/TezClientUtils.java   |  560 ------
 .../java/org/apache/tez/client/TezSession.java  |  184 --
 .../tez/client/TezSessionConfiguration.java     |   57 -
 .../main/java/org/apache/tez/dag/api/DAG.java   |  377 ----
 .../apache/tez/dag/api/DagTypeConverters.java   |  287 ---
 .../main/java/org/apache/tez/dag/api/Edge.java  |   58 -
 .../org/apache/tez/dag/api/EdgeProperty.java    |  147 --
 .../org/apache/tez/dag/api/InputDescriptor.java |   32 -
 .../apache/tez/dag/api/OutputDescriptor.java    |   32 -
 .../apache/tez/dag/api/ProcessorDescriptor.java |   31 -
 .../apache/tez/dag/api/TezConfiguration.java    |  215 ---
 .../org/apache/tez/dag/api/TezConstants.java    |   29 -
 .../apache/tez/dag/api/TezEntityDescriptor.java |   42 -
 .../org/apache/tez/dag/api/TezException.java    |   31 -
 .../tez/dag/api/TezUncheckedException.java      |   33 -
 .../java/org/apache/tez/dag/api/Vertex.java     |  153 --
 .../apache/tez/dag/api/VertexLocationHint.java  |  154 --
 .../apache/tez/dag/api/client/DAGClient.java    |   67 -
 .../apache/tez/dag/api/client/DAGStatus.java    |  130 --
 .../org/apache/tez/dag/api/client/Progress.java |   67 -
 .../apache/tez/dag/api/client/VertexStatus.java |   78 -
 .../rpc/DAGClientAMProtocolBlockingPB.java      |   30 -
 .../dag/api/client/rpc/DAGClientRPCImpl.java    |  291 ---
 tez-dag-api/src/main/proto/DAGApiRecords.proto  |  183 --
 .../src/main/proto/DAGClientAMProtocol.proto    |   81 -
 .../org/apache/tez/dag/api/TestDAGPlan.java     |  158 --
 .../org/apache/tez/dag/api/TestDAGVerify.java   |  417 ----
 tez-dag/pom.xml                                 |   18 +-
 .../apache/hadoop/mapred/YarnOutputFiles.java   |  236 ---
 .../apache/hadoop/mapred/YarnTezDagChild.java   |  461 +++--
 .../dag/api/oldrecords/TaskAttemptState.java    |    1 -
 .../org/apache/tez/dag/app/DAGAppMaster.java    |    2 +-
 .../dag/app/TaskAttemptListenerImpTezDag.java   |  306 ++-
 .../tez/dag/app/TaskHeartbeatHandler.java       |    9 +-
 .../java/org/apache/tez/dag/app/dag/DAG.java    |    4 +
 .../tez/dag/app/dag/DAGTerminationCause.java    |    4 +-
 .../org/apache/tez/dag/app/dag/EdgeManager.java |   52 +
 .../java/org/apache/tez/dag/app/dag/Task.java   |    7 +
 .../org/apache/tez/dag/app/dag/TaskAttempt.java |   45 +-
 .../dag/app/dag/TaskAttemptStateInternal.java   |    1 -
 .../tez/dag/app/dag/TaskTerminationCause.java   |    7 +-
 .../java/org/apache/tez/dag/app/dag/Vertex.java |   37 +-
 .../apache/tez/dag/app/dag/VertexScheduler.java |    6 +-
 .../tez/dag/app/dag/VertexTerminationCause.java |    4 +-
 .../tez/dag/app/dag/event/DAGEventType.java     |    1 +
 .../app/dag/event/DAGEventVertexReRunning.java  |   37 +
 .../event/TaskAttemptEventAttemptFailed.java    |   39 +
 .../dag/event/TaskAttemptEventNodeFailed.java   |    4 +-
 .../event/TaskAttemptEventOutputConsumable.java |   36 -
 .../dag/event/TaskAttemptEventOutputFailed.java |   44 +
 .../event/TaskAttemptEventStartedRemotely.java  |    7 +-
 .../dag/event/TaskAttemptEventStatusUpdate.java |   21 +-
 .../dag/app/dag/event/TaskAttemptEventType.java |    5 +-
 .../dag/app/dag/event/TaskEventAddTezEvent.java |   36 +
 .../tez/dag/app/dag/event/TaskEventType.java    |    4 +-
 .../app/dag/event/VertexEventRouteEvent.java    |   39 +
 .../VertexEventSourceTaskAttemptCompleted.java  |    7 +-
 .../event/VertexEventTaskAttemptCompleted.java  |   25 +-
 .../VertexEventTaskAttemptFetchFailure.java     |   46 -
 .../tez/dag/app/dag/event/VertexEventType.java  |    7 +-
 .../dag/app/dag/impl/BroadcastEdgeManager.java  |   74 +
 .../apache/tez/dag/app/dag/impl/DAGImpl.java    |  144 +-
 .../org/apache/tez/dag/app/dag/impl/Edge.java   |  220 +++
 .../dag/impl/ImmediateStartVertexScheduler.java |    4 +-
 .../dag/app/dag/impl/OneToOneEdgeManager.java   |   70 +
 .../app/dag/impl/ScatterGatherEdgeManager.java  |   68 +
 .../dag/app/dag/impl/ShuffleVertexManager.java  |  121 +-
 .../tez/dag/app/dag/impl/TaskAttemptImpl.java   |  243 ++-
 .../apache/tez/dag/app/dag/impl/TaskImpl.java   |  285 ++-
 .../apache/tez/dag/app/dag/impl/VertexImpl.java |  430 +++--
 .../dag/app/launcher/ContainerLauncherImpl.java |   19 +-
 .../app/rm/AMSchedulerEventTALaunchRequest.java |   13 +-
 .../dag/app/rm/TaskSchedulerEventHandler.java   |    2 +-
 .../tez/dag/app/rm/container/AMContainer.java   |    2 -
 .../rm/container/AMContainerEventAssignTA.java  |   22 +-
 .../rm/container/AMContainerEventLaunched.java  |    9 +-
 .../app/rm/container/AMContainerHelpers.java    |    8 +-
 .../dag/app/rm/container/AMContainerImpl.java   |   25 +-
 .../dag/app/rm/container/AMContainerTask.java   |    8 +-
 .../dag/app/speculate/DefaultSpeculator.java    |    6 +-
 ...ponentiallySmoothedTaskRuntimeEstimator.java |    4 +-
 .../speculate/LegacyTaskRuntimeEstimator.java   |    4 +-
 .../app/speculate/NullTaskRuntimesEngine.java   |    6 +-
 .../tez/dag/app/speculate/Speculator.java       |    4 +-
 .../tez/dag/app/speculate/SpeculatorEvent.java  |   10 +-
 .../dag/app/speculate/StartEndTimesBase.java    |    6 +-
 .../dag/app/speculate/TaskRuntimeEstimator.java |    6 +-
 .../apache/tez/dag/utils/TezEngineChildJVM.java |  122 --
 .../tez/dag/utils/TezRuntimeChildJVM.java       |  122 ++
 .../tez/dag/app/dag/impl/TestDAGImpl.java       |    2 +-
 .../tez/dag/app/dag/impl/TestTaskAttempt.java   |   98 +-
 .../tez/dag/app/dag/impl/TestTaskImpl.java      |   63 +-
 .../tez/dag/app/dag/impl/TestVertexImpl.java    |  196 +-
 .../dag/app/dag/impl/TestVertexScheduler.java   |  119 +-
 .../tez/dag/app/rm/TestContainerReuse.java      |   65 +-
 .../dag/app/rm/container/TestAMContainer.java   |   24 +-
 tez-dist/src/main/assembly/tez-dist-full.xml    |    6 +-
 tez-dist/src/main/assembly/tez-dist.xml         |    6 +-
 tez-engine-api/findbugs-exclude.xml             |   16 -
 tez-engine-api/pom.xml                          |   60 -
 .../java/org/apache/tez/engine/api/Input.java   |   83 -
 .../java/org/apache/tez/engine/api/Master.java  |   39 -
 .../java/org/apache/tez/engine/api/Output.java  |   65 -
 .../org/apache/tez/engine/api/Partitioner.java  |   35 -
 .../org/apache/tez/engine/api/Processor.java    |   62 -
 .../java/org/apache/tez/engine/api/Task.java    |   79 -
 .../common/objectregistry/ObjectLifeCycle.java  |   37 -
 .../common/objectregistry/ObjectRegistry.java   |   56 -
 .../objectregistry/ObjectRegistryFactory.java   |   32 -
 .../tez/engine/records/OutputContext.java       |   61 -
 .../TezDependentTaskCompletionEvent.java        |  250 ---
 ...TezTaskDependencyCompletionEventsUpdate.java |   64 -
 tez-engine/findbugs-exclude.xml                 |   16 -
 tez-engine/pom.xml                              |   64 -
 .../java/org/apache/hadoop/io/BufferUtils.java  |   81 -
 .../org/apache/hadoop/io/HashComparator.java    |   24 -
 .../apache/tez/common/RunningTaskContext.java   |   70 -
 .../apache/tez/common/TezEngineTaskContext.java |  141 --
 .../org/apache/tez/common/TezTaskReporter.java  |  116 --
 .../tez/common/TezTaskUmbilicalProtocol.java    |   79 -
 .../apache/tez/engine/common/ConfigUtils.java   |  144 --
 .../apache/tez/engine/common/YARNMaster.java    |   57 -
 .../tez/engine/common/combine/CombineInput.java |  176 --
 .../engine/common/combine/CombineOutput.java    |   55 -
 .../common/localshuffle/LocalShuffle.java       |  128 --
 .../objectregistry/ObjectRegistryImpl.java      |   65 -
 .../objectregistry/ObjectRegistryModule.java    |   43 -
 .../common/security/DelegationTokenRenewal.java |  318 ---
 .../common/security/JobTokenIdentifier.java     |   98 -
 .../common/security/JobTokenSecretManager.java  |  137 --
 .../common/security/JobTokenSelector.java       |   53 -
 .../tez/engine/common/security/Master.java      |   57 -
 .../common/security/SecureShuffleUtils.java     |  142 --
 .../tez/engine/common/security/TokenCache.java  |  205 --
 .../common/shuffle/impl/EventFetcher.java       |  212 --
 .../common/shuffle/impl/ExceptionReporter.java  |   25 -
 .../tez/engine/common/shuffle/impl/Fetcher.java |  627 ------
 .../common/shuffle/impl/InMemoryReader.java     |  157 --
 .../common/shuffle/impl/InMemoryWriter.java     |  100 -
 .../tez/engine/common/shuffle/impl/MapHost.java |  122 --
 .../engine/common/shuffle/impl/MapOutput.java   |  226 ---
 .../common/shuffle/impl/MergeManager.java       |  807 --------
 .../engine/common/shuffle/impl/MergeThread.java |  108 --
 .../tez/engine/common/shuffle/impl/Shuffle.java |  259 ---
 .../shuffle/impl/ShuffleClientMetrics.java      |   93 -
 .../common/shuffle/impl/ShuffleHeader.java      |   86 -
 .../common/shuffle/impl/ShuffleScheduler.java   |  481 -----
 .../common/shuffle/server/ShuffleHandler.java   |  569 ------
 .../tez/engine/common/sort/SortingOutput.java   |   32 -
 .../engine/common/sort/impl/ExternalSorter.java |  246 ---
 .../tez/engine/common/sort/impl/IFile.java      |  559 ------
 .../common/sort/impl/IFileInputStream.java      |  276 ---
 .../common/sort/impl/IFileOutputStream.java     |  131 --
 .../common/sort/impl/PipelinedSorter.java       |  957 ---------
 .../engine/common/sort/impl/TezIndexRecord.java |   45 -
 .../tez/engine/common/sort/impl/TezMerger.java  |  797 --------
 .../sort/impl/TezRawKeyValueIterator.java       |   70 -
 .../engine/common/sort/impl/TezSpillRecord.java |  146 --
 .../common/sort/impl/dflt/DefaultSorter.java    | 1134 -----------
 .../sort/impl/dflt/InMemoryShuffleSorter.java   |  138 --
 .../sort/impl/dflt/SortBufferInputStream.java   |  271 ---
 .../common/task/impl/CombineValuesIterator.java |   51 -
 .../engine/common/task/impl/ValuesIterator.java |  142 --
 .../local/output/TezLocalTaskOutputFiles.java   |  237 ---
 .../common/task/local/output/TezTaskOutput.java |  165 --
 .../task/local/output/TezTaskOutputFiles.java   |  241 ---
 .../tez/engine/lib/input/LocalMergedInput.java  |   82 -
 .../engine/lib/input/ShuffledMergedInput.java   |  107 --
 .../engine/lib/output/InMemorySortedOutput.java |   70 -
 .../lib/output/LocalOnFileSorterOutput.java     |   57 -
 .../engine/lib/output/OnFileSortedOutput.java   |   69 -
 .../apache/tez/engine/runtime/RuntimeUtils.java |  164 --
 .../org/apache/tez/engine/task/RuntimeTask.java |   92 -
 .../objectregistry/TestObjectRegistry.java      |   56 -
 .../tez/mapreduce/examples/ExampleDriver.java   |    3 +
 .../tez/mapreduce/examples/MRRSleepJob.java     |   12 +-
 .../tez/mapreduce/examples/MapredWordCount.java |  163 ++
 .../mapreduce/examples/OrderedWordCount.java    |    8 +-
 .../tez/mapreduce/examples/WordCount.java       |   14 +-
 .../apache/tez/mapreduce/TestMRRJobsDAGApi.java |   51 +-
 tez-mapreduce/pom.xml                           |    9 +-
 .../mapred/LocalClientProtocolProviderTez.java  |  108 +-
 .../hadoop/mapred/LocalJobRunnerMetricsTez.java |  196 +-
 .../apache/hadoop/mapred/LocalJobRunnerTez.java | 1818 +++++++++---------
 .../mapreduce/split/SplitMetaInfoReaderTez.java |    8 +-
 .../org/apache/tez/common/TezTaskStatus.java    |  105 +
 .../tez/mapreduce/combine/MRCombiner.java       |  446 ++---
 .../org/apache/tez/mapreduce/common/Utils.java  |   47 +
 .../tez/mapreduce/hadoop/DeprecatedKeys.java    |   97 +-
 .../apache/tez/mapreduce/hadoop/MRConfig.java   |   27 +-
 .../apache/tez/mapreduce/hadoop/MRHelpers.java  |   16 +
 .../hadoop/MultiStageMRConfToTezTranslator.java |   29 +-
 .../hadoop/MultiStageMRConfigUtil.java          |    2 +-
 .../tez/mapreduce/hadoop/TezTypeConverters.java |    9 -
 .../tez/mapreduce/hadoop/mapred/MRReporter.java |   76 +
 .../hadoop/mapred/TaskAttemptContextImpl.java   |   21 +-
 .../hadoop/mapreduce/MapContextImpl.java        |    6 +-
 .../hadoop/mapreduce/ReduceContextImpl.java     |  359 ----
 .../mapreduce/TaskAttemptContextImpl.java       |  102 +-
 .../mapreduce/TaskInputOutputContextImpl.java   |    6 +-
 .../org/apache/tez/mapreduce/input/MRInput.java |  439 +++++
 .../tez/mapreduce/input/MRInputLegacy.java      |   36 +
 .../apache/tez/mapreduce/input/SimpleInput.java |  387 ----
 .../apache/tez/mapreduce/output/MROutput.java   |  326 ++++
 .../tez/mapreduce/output/SimpleOutput.java      |  193 --
 .../tez/mapreduce/partition/MRPartitioner.java  |  105 +-
 .../apache/tez/mapreduce/processor/MRTask.java  |  646 +++----
 .../tez/mapreduce/processor/MRTaskReporter.java |   77 +-
 .../processor/TezTaskReporterImpl.java          |  270 ---
 .../mapreduce/processor/map/MapProcessor.java   |  288 ++-
 .../processor/reduce/ReduceProcessor.java       |  342 ++--
 .../tez/mapreduce/task/MRRuntimeTask.java       |  270 ---
 .../mapreduce/task/impl/YarnOutputFiles.java    |  239 ---
 .../org/apache/tez/mapreduce/TestUmbilical.java |   62 +
 .../tez/mapreduce/TestUmbilicalProtocol.java    |  148 --
 .../hadoop/TestConfigTranslationMRToTez.java    |    2 +-
 .../mapreduce/hadoop/TestDeprecatedKeys.java    |   18 +-
 .../tez/mapreduce/processor/MapUtils.java       |   57 +-
 .../processor/map/TestMapProcessor.java         |  229 +--
 .../processor/reduce/TestReduceProcessor.java   |  112 +-
 tez-runtime-internals/findbugs-exclude.xml      |   16 +
 tez-runtime-internals/pom.xml                   |   95 +
 .../org/apache/tez/common/ContainerContext.java |   64 +
 .../org/apache/tez/common/ContainerTask.java    |   74 +
 .../tez/common/TezTaskUmbilicalProtocol.java    |   57 +
 .../runtime/LogicalIOProcessorRuntimeTask.java  |  475 +++++
 .../org/apache/tez/runtime/RuntimeTask.java     |  106 +
 .../org/apache/tez/runtime/RuntimeUtils.java    |   62 +
 .../api/events/TaskAttemptCompletedEvent.java   |   28 +
 .../api/events/TaskAttemptFailedEvent.java      |   35 +
 .../api/events/TaskStatusUpdateEvent.java       |   70 +
 .../tez/runtime/api/impl/EventMetaData.java     |  152 ++
 .../apache/tez/runtime/api/impl/EventType.java  |   29 +
 .../apache/tez/runtime/api/impl/InputSpec.java  |   88 +
 .../apache/tez/runtime/api/impl/OutputSpec.java |   87 +
 .../apache/tez/runtime/api/impl/TaskSpec.java   |  146 ++
 .../apache/tez/runtime/api/impl/TezEvent.java   |  248 +++
 .../runtime/api/impl/TezHeartbeatRequest.java   |  137 ++
 .../runtime/api/impl/TezHeartbeatResponse.java  |  105 +
 .../runtime/api/impl/TezInputContextImpl.java   |   84 +
 .../runtime/api/impl/TezOutputContextImpl.java  |   85 +
 .../api/impl/TezProcessorContextImpl.java       |   86 +
 .../runtime/api/impl/TezTaskContextImpl.java    |  145 ++
 .../tez/runtime/api/impl/TezUmbilical.java      |   36 +
 .../objectregistry/ObjectRegistryImpl.java      |   68 +
 .../objectregistry/ObjectRegistryModule.java    |   46 +
 .../src/main/proto/Events.proto                 |   28 +
 .../objectregistry/TestObjectRegistry.java      |   60 +
 tez-runtime-library/pom.xml                     |   79 +
 .../java/org/apache/hadoop/io/BufferUtils.java  |   81 +
 .../org/apache/hadoop/io/HashComparator.java    |   24 +
 .../tez/runtime/library/api/KVReader.java       |   81 +
 .../tez/runtime/library/api/KVWriter.java       |   40 +
 .../tez/runtime/library/api/Partitioner.java    |   52 +
 .../broadcast/input/BroadcastInputManager.java  |  138 ++
 .../broadcast/input/BroadcastKVReader.java      |  225 +++
 .../BroadcastShuffleInputEventHandler.java      |   88 +
 .../input/BroadcastShuffleManager.java          |  489 +++++
 .../broadcast/output/FileBasedKVWriter.java     |  125 ++
 .../tez/runtime/library/common/ConfigUtils.java |  148 ++
 .../tez/runtime/library/common/Constants.java   |   61 +
 .../library/common/InputAttemptIdentifier.java  |   95 +
 .../runtime/library/common/InputIdentifier.java |   56 +
 .../runtime/library/common/TezRuntimeUtils.java |  152 ++
 .../runtime/library/common/ValuesIterator.java  |  194 ++
 .../tez/runtime/library/common/YARNMaster.java  |   57 +
 .../library/common/combine/Combiner.java        |   42 +
 .../common/localshuffle/LocalShuffle.java       |  120 ++
 .../common/security/JobTokenIdentifier.java     |   98 +
 .../common/security/JobTokenSecretManager.java  |  137 ++
 .../common/security/JobTokenSelector.java       |   53 +
 .../runtime/library/common/security/Master.java |   57 +
 .../common/security/SecureShuffleUtils.java     |  142 ++
 .../library/common/security/TokenCache.java     |  205 ++
 .../common/shuffle/impl/ExceptionReporter.java  |   25 +
 .../library/common/shuffle/impl/Fetcher.java    |  624 ++++++
 .../common/shuffle/impl/InMemoryReader.java     |  156 ++
 .../common/shuffle/impl/InMemoryWriter.java     |  100 +
 .../library/common/shuffle/impl/MapHost.java    |  124 ++
 .../library/common/shuffle/impl/MapOutput.java  |  227 +++
 .../common/shuffle/impl/MergeManager.java       |  782 ++++++++
 .../common/shuffle/impl/MergeThread.java        |  108 ++
 .../library/common/shuffle/impl/Shuffle.java    |  278 +++
 .../shuffle/impl/ShuffleClientMetrics.java      |   91 +
 .../common/shuffle/impl/ShuffleHeader.java      |   94 +
 .../shuffle/impl/ShuffleInputEventHandler.java  |  134 ++
 .../common/shuffle/impl/ShuffleScheduler.java   |  521 +++++
 .../common/shuffle/server/ShuffleHandler.java   |  572 ++++++
 .../common/sort/impl/ExternalSorter.java        |  194 ++
 .../runtime/library/common/sort/impl/IFile.java |  559 ++++++
 .../common/sort/impl/IFileInputStream.java      |  276 +++
 .../common/sort/impl/IFileOutputStream.java     |  129 ++
 .../common/sort/impl/PipelinedSorter.java       |  932 +++++++++
 .../common/sort/impl/TezIndexRecord.java        |   45 +
 .../library/common/sort/impl/TezMerger.java     |  798 ++++++++
 .../sort/impl/TezRawKeyValueIterator.java       |   70 +
 .../common/sort/impl/TezSpillRecord.java        |  146 ++
 .../common/sort/impl/dflt/DefaultSorter.java    | 1108 +++++++++++
 .../sort/impl/dflt/InMemoryShuffleSorter.java   |  126 ++
 .../sort/impl/dflt/SortBufferInputStream.java   |  271 +++
 .../common/task/impl/ValuesIterator.java        |  149 ++
 .../local/output/TezLocalTaskOutputFiles.java   |  249 +++
 .../common/task/local/output/TezTaskOutput.java |  165 ++
 .../task/local/output/TezTaskOutputFiles.java   |  246 +++
 .../library/hadoop/compat/NullProgressable.java |   33 +
 .../runtime/library/input/LocalMergedInput.java |   52 +
 .../library/input/ShuffledMergedInput.java      |  179 ++
 .../input/ShuffledMergedInputLegacy.java        |   30 +
 .../library/input/ShuffledUnorderedKVInput.java |   76 +
 .../library/output/InMemorySortedOutput.java    |   81 +
 .../library/output/LocalOnFileSorterOutput.java |   63 +
 .../library/output/OnFileSortedOutput.java      |  123 ++
 .../library/output/OnFileUnorderedKVOutput.java |   98 +
 .../shuffle/common/DiskFetchedInput.java        |  111 ++
 .../library/shuffle/common/FetchResult.java     |   70 +
 .../library/shuffle/common/FetchedInput.java    |  144 ++
 .../shuffle/common/FetchedInputAllocator.java   |   31 +
 .../shuffle/common/FetchedInputCallback.java    |   29 +
 .../runtime/library/shuffle/common/Fetcher.java |  608 ++++++
 .../library/shuffle/common/FetcherCallback.java |   31 +
 .../library/shuffle/common/InputHost.java       |   90 +
 .../shuffle/common/MemoryFetchedInput.java      |   89 +
 .../library/shuffle/common/ShuffleUtils.java    |   81 +
 .../src/main/proto/ShufflePayloads.proto        |   33 +
 tez-yarn-client/pom.xml                         |    2 +-
 .../org/apache/tez/mapreduce/DAGJobStatus.java  |   12 +-
 .../org/apache/tez/mapreduce/YARNRunner.java    |    6 +-
 432 files changed, 31100 insertions(+), 27207 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/f1cbb1c6/tez-dag/src/main/java/org/apache/tez/dag/app/dag/Vertex.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/f1cbb1c6/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java
----------------------------------------------------------------------


[19/50] [abbrv] TEZ-443. Merge tez-dag-api and tez-engine-api into a single module - tez-api (part of TEZ-398). (sseth)

Posted by ss...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-dag-api/src/main/java/org/apache/tez/dag/api/DAG.java
----------------------------------------------------------------------
diff --git a/tez-dag-api/src/main/java/org/apache/tez/dag/api/DAG.java b/tez-dag-api/src/main/java/org/apache/tez/dag/api/DAG.java
deleted file mode 100644
index 9cb602c..0000000
--- a/tez-dag-api/src/main/java/org/apache/tez/dag/api/DAG.java
+++ /dev/null
@@ -1,377 +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.api;
-
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Stack;
-
-import org.apache.hadoop.classification.InterfaceAudience.Private;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.yarn.api.records.LocalResource;
-import org.apache.hadoop.yarn.api.records.LocalResourceType;
-import org.apache.hadoop.yarn.api.records.Resource;
-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.VertexLocationHint.TaskLocationHint;
-import org.apache.tez.dag.api.records.DAGProtos.ConfigurationProto;
-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.PlanKeyValuePair;
-import org.apache.tez.dag.api.records.DAGProtos.PlanLocalResource;
-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.VertexPlan;
-
-
-public class DAG { // FIXME rename to Topology
-  final List<Vertex> vertices;
-  final List<Edge> edges;
-  final String name;
-
-  public DAG(String name) {
-    this.vertices = new ArrayList<Vertex>();
-    this.edges = new ArrayList<Edge>();
-    this.name = name;
-  }
-
-  public synchronized DAG addVertex(Vertex vertex) {
-    if (vertices.contains(vertex)) {
-      throw new IllegalArgumentException(
-          "Vertex " + vertex + " already defined!");
-    }
-    vertices.add(vertex);
-    return this;
-  }
-
-  @Private
-  public synchronized List<Vertex> getVertices() {
-    return Collections.unmodifiableList(this.vertices);
-  }
-
-  public synchronized DAG addEdge(Edge edge) {
-    // Sanity checks
-    if (!vertices.contains(edge.getInputVertex())) {
-      throw new IllegalArgumentException(
-          "Input vertex " + edge.getInputVertex() + " doesn't exist!");
-    }
-    if (!vertices.contains(edge.getOutputVertex())) {
-      throw new IllegalArgumentException(
-          "Output vertex " + edge.getOutputVertex() + " doesn't exist!");
-    }
-    if (edges.contains(edge)) {
-      throw new IllegalArgumentException(
-          "Edge " + edge + " already defined!");
-    }
-
-    // Inform the vertices
-    edge.getInputVertex().addOutputVertex(edge.getOutputVertex(), edge.getId());
-    edge.getOutputVertex().addInputVertex(edge.getInputVertex(), edge.getId());
-
-    edges.add(edge);
-    return this;
-  }
-
-  public String getName() {
-    return this.name;
-  }
-
-  // AnnotatedVertex is used by verify()
-  private static class AnnotatedVertex {
-    Vertex v;
-
-    int index; //for Tarjan's algorithm
-    int lowlink; //for Tarjan's algorithm
-    boolean onstack; //for Tarjan's algorithm
-
-    int outDegree;
-
-    private AnnotatedVertex(Vertex v){
-       this.v = v;
-       index = -1;
-       lowlink = -1;
-       outDegree = 0;
-    }
-  }
-
-  // verify()
-  //
-  // Default rules
-  //   Illegal:
-  //     - duplicate vertex id
-  //     - cycles
-  //
-  //   Ok:
-  //     - orphaned vertex.  Occurs in map-only
-  //     - islands.  Occurs if job has unrelated workflows.
-  //
-  //   Not yet categorized:
-  //     - orphaned vertex in DAG of >1 vertex.  Could be unrelated map-only job.
-  //     - v1->v2 via two edges.  perhaps some self-join job would use this?
-  //
-  // "restricted" mode:
-  //   In short term, the supported DAGs are limited. Call with restricted=true for these verifications.
-  //   Illegal:
-  //     - any vertex with more than one input or output edge. (n-ary input, n-ary merge)
-  public void verify() throws IllegalStateException {
-    verify(true);
-  }
-
-  public void verify(boolean restricted) throws IllegalStateException  {
-    if (vertices.isEmpty()) {
-      throw new IllegalStateException("Invalid dag containing 0 vertices");
-    }
-
-    Map<Vertex, List<Edge>> edgeMap = new HashMap<Vertex, List<Edge>>();
-    for(Edge e : edges){
-      Vertex inputVertex = e.getInputVertex();
-      List<Edge> edgeList = edgeMap.get(inputVertex);
-      if(edgeList == null){
-        edgeList = new ArrayList<Edge>();
-        edgeMap.put(inputVertex, edgeList);
-      }
-      edgeList.add(e);
-    }
-
-    // check for valid vertices, duplicate vertex names,
-    // and prepare for cycle detection
-    Map<String, AnnotatedVertex> vertexMap = new HashMap<String, AnnotatedVertex>();
-    for(Vertex v : vertices){
-      if(vertexMap.containsKey(v.getVertexName())){
-         throw new IllegalStateException("DAG contains multiple vertices"
-             + " with name: " + v.getVertexName());
-      }
-      vertexMap.put(v.getVertexName(), new AnnotatedVertex(v));
-    }
-
-    detectCycles(edgeMap, vertexMap);
-
-    if(restricted){
-      for(Edge e : edges){
-        vertexMap.get(e.getInputVertex().getVertexName()).outDegree++;
-        if (e.getEdgeProperty().getDataMovementType() !=
-            DataMovementType.SCATTER_GATHER) {
-          throw new IllegalStateException(
-              "Unsupported connection pattern on edge. " + e);
-        }
-        if (e.getEdgeProperty().getDataSourceType() !=
-            DataSourceType.PERSISTED) {
-          throw new IllegalStateException(
-              "Unsupported source type on edge. " + e);
-        }
-        if (e.getEdgeProperty().getSchedulingType() !=
-            SchedulingType.SEQUENTIAL) {
-          throw new IllegalStateException(
-              "Unsupported scheduling type on edge. " + e);
-        }
-      }
-      for(AnnotatedVertex av: vertexMap.values()){
-        if (av.outDegree > 1) {
-          throw new IllegalStateException("Vertex has outDegree>1: "
-              + av.v.getVertexName());
-        }
-      }
-    }
-  }
-
-  // Adaptation of Tarjan's algorithm for connected components.
-  // http://en.wikipedia.org/wiki/Tarjan%27s_strongly_connected_components_algorithm
-  private void detectCycles(Map<Vertex, List<Edge>> edgeMap, Map<String, AnnotatedVertex> vertexMap)
-      throws IllegalStateException{
-    Integer nextIndex = 0; // boxed integer so it is passed by reference.
-    Stack<AnnotatedVertex> stack = new Stack<DAG.AnnotatedVertex>();
-    for(AnnotatedVertex av: vertexMap.values()){
-      if(av.index == -1){
-        assert stack.empty();
-        strongConnect(av, vertexMap, edgeMap, stack, nextIndex);
-      }
-    }
-  }
-
-  // part of Tarjan's algorithm for connected components.
-  // http://en.wikipedia.org/wiki/Tarjan%27s_strongly_connected_components_algorithm
-  private void strongConnect(
-          AnnotatedVertex av,
-          Map<String, AnnotatedVertex> vertexMap,
-          Map<Vertex, List<Edge>> edgeMap,
-          Stack<AnnotatedVertex> stack, Integer nextIndex) throws IllegalStateException{
-    av.index = nextIndex;
-    av.lowlink = nextIndex;
-    nextIndex++;
-    stack.push(av);
-    av.onstack = true;
-
-    List<Edge> edges = edgeMap.get(av.v);
-    if(edges != null){
-      for(Edge e : edgeMap.get(av.v)){
-        AnnotatedVertex outVertex = vertexMap.get(e.getOutputVertex().getVertexName());
-        if(outVertex.index == -1){
-          strongConnect(outVertex, vertexMap, edgeMap, stack, nextIndex);
-          av.lowlink = Math.min(av.lowlink, outVertex.lowlink);
-        }
-        else if(outVertex.onstack){
-          // strongly connected component detected, but we will wait till later so that the full cycle can be displayed.
-          // update lowlink in case outputVertex should be considered the root of this component.
-          av.lowlink = Math.min(av.lowlink, outVertex.index);
-        }
-      }
-    }
-
-    if(av.lowlink == av.index ){
-       AnnotatedVertex pop = stack.pop();
-       pop.onstack = false;
-       if(pop != av){
-         // there was something on the stack other than this "av".
-         // this indicates there is a scc/cycle. It comprises all nodes from top of stack to "av"
-         StringBuilder message = new StringBuilder();
-         message.append(av.v.getVertexName() + " <- ");
-         for( ; pop != av; pop = stack.pop()){
-           message.append(pop.v.getVertexName() + " <- ");
-           pop.onstack = false;
-         }
-         message.append(av.v.getVertexName());
-         throw new IllegalStateException("DAG contains a cycle: " + message);
-       }
-    }
-  }
-
-
-  // create protobuf message describing DAG
-  @Private
-  public DAGPlan createDag(Configuration dagConf) {
-    verify(true);
-
-    DAGPlan.Builder dagBuilder = DAGPlan.newBuilder();
-
-    dagBuilder.setName(this.name);
-
-    for(Vertex vertex : vertices){
-      VertexPlan.Builder vertexBuilder = VertexPlan.newBuilder();
-      vertexBuilder.setName(vertex.getVertexName());
-      vertexBuilder.setType(PlanVertexType.NORMAL); // vertex type is implicitly NORMAL until  TEZ-46.
-      vertexBuilder.setProcessorDescriptor(DagTypeConverters
-          .convertToDAGPlan(vertex.getProcessorDescriptor()));
-
-      //task config
-      PlanTaskConfiguration.Builder taskConfigBuilder = PlanTaskConfiguration.newBuilder();
-      Resource resource = vertex.getTaskResource();
-      taskConfigBuilder.setNumTasks(vertex.getParallelism());
-      taskConfigBuilder.setMemoryMb(resource.getMemory());
-      taskConfigBuilder.setVirtualCores(resource.getVirtualCores());
-      taskConfigBuilder.setJavaOpts(vertex.getJavaOpts());
-
-      taskConfigBuilder.setTaskModule(vertex.getVertexName());
-      PlanLocalResource.Builder localResourcesBuilder = PlanLocalResource.newBuilder();
-      Map<String,LocalResource> lrs = vertex.getTaskLocalResources();
-      for(Entry<String, LocalResource> entry : lrs.entrySet()){
-        String key = entry.getKey();
-        LocalResource lr = entry.getValue();
-        localResourcesBuilder.setName(key);
-        localResourcesBuilder.setUri(
-            DagTypeConverters.convertToDAGPlan(lr.getResource()));
-        localResourcesBuilder.setSize(lr.getSize());
-        localResourcesBuilder.setTimeStamp(lr.getTimestamp());
-        localResourcesBuilder.setType(
-            DagTypeConverters.convertToDAGPlan(lr.getType()));
-        localResourcesBuilder.setVisibility(
-            DagTypeConverters.convertToDAGPlan(lr.getVisibility()));
-        if(lr.getType() == LocalResourceType.PATTERN){
-          if (lr.getPattern() == null || lr.getPattern().isEmpty()) {
-            throw new TezUncheckedException("LocalResource type set to pattern"
-                + " but pattern is null or empty");
-          }
-          localResourcesBuilder.setPattern(lr.getPattern());
-        }
-        taskConfigBuilder.addLocalResource(localResourcesBuilder);
-      }
-
-      if(vertex.getTaskEnvironment() != null){
-        for(String key : vertex.getTaskEnvironment().keySet()){
-          PlanKeyValuePair.Builder envSettingBuilder = PlanKeyValuePair.newBuilder();
-          envSettingBuilder.setKey(key);
-          envSettingBuilder.setValue(vertex.getTaskEnvironment().get(key));
-          taskConfigBuilder.addEnvironmentSetting(envSettingBuilder);
-        }
-      }
-
-      if(vertex.getTaskLocationsHint() != null ){
-        if(vertex.getTaskLocationsHint().getTaskLocationHints() != null){
-          for(TaskLocationHint hint : vertex.getTaskLocationsHint().getTaskLocationHints()){
-            PlanTaskLocationHint.Builder taskLocationHintBuilder = PlanTaskLocationHint.newBuilder();
-
-            if(hint.getDataLocalHosts() != null){
-              taskLocationHintBuilder.addAllHost(hint.getDataLocalHosts());
-            }
-            if(hint.getRacks() != null){
-              taskLocationHintBuilder.addAllRack(hint.getRacks());
-            }
-
-            vertexBuilder.addTaskLocationHint(taskLocationHintBuilder);
-          }
-        }
-      }
-
-      for(String inEdgeId : vertex.getInputEdgeIds()){
-        vertexBuilder.addInEdgeId(inEdgeId);
-      }
-
-      for(String outEdgeId : vertex.getOutputEdgeIds()){
-        vertexBuilder.addOutEdgeId(outEdgeId);
-      }
-
-      vertexBuilder.setTaskConfig(taskConfigBuilder);
-      dagBuilder.addVertex(vertexBuilder);
-    }
-
-    for(Edge edge : edges){
-      EdgePlan.Builder edgeBuilder = EdgePlan.newBuilder();
-      edgeBuilder.setId(edge.getId());
-      edgeBuilder.setInputVertexName(edge.getInputVertex().getVertexName());
-      edgeBuilder.setOutputVertexName(edge.getOutputVertex().getVertexName());
-      edgeBuilder.setDataMovementType(DagTypeConverters.convertToDAGPlan(edge.getEdgeProperty().getDataMovementType()));
-      edgeBuilder.setDataSourceType(DagTypeConverters.convertToDAGPlan(edge.getEdgeProperty().getDataSourceType()));
-      edgeBuilder.setSchedulingType(DagTypeConverters.convertToDAGPlan(edge.getEdgeProperty().getSchedulingType()));
-      edgeBuilder.setEdgeSource(DagTypeConverters.convertToDAGPlan(edge.getEdgeProperty().getEdgeSource()));
-      edgeBuilder.setEdgeDestination(DagTypeConverters.convertToDAGPlan(edge.getEdgeProperty().getEdgeDestination()));
-      dagBuilder.addEdge(edgeBuilder);
-    }
-
-    if(dagConf != null) {
-      Iterator<Entry<String, String>> iter = dagConf.iterator();
-      ConfigurationProto.Builder confProtoBuilder =
-          ConfigurationProto.newBuilder();
-      while (iter.hasNext()) {
-        Entry<String, String> entry = iter.next();
-        PlanKeyValuePair.Builder kvp = PlanKeyValuePair.newBuilder();
-        kvp.setKey(entry.getKey());
-        kvp.setValue(entry.getValue());
-        confProtoBuilder.addConfKeyValues(kvp);
-      }
-      dagBuilder.setDagKeyValues(confProtoBuilder);
-    }
-
-    return dagBuilder.build();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-dag-api/src/main/java/org/apache/tez/dag/api/DagTypeConverters.java
----------------------------------------------------------------------
diff --git a/tez-dag-api/src/main/java/org/apache/tez/dag/api/DagTypeConverters.java b/tez-dag-api/src/main/java/org/apache/tez/dag/api/DagTypeConverters.java
deleted file mode 100644
index 1fd78f1..0000000
--- a/tez-dag-api/src/main/java/org/apache/tez/dag/api/DagTypeConverters.java
+++ /dev/null
@@ -1,278 +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.api;
-
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.yarn.api.records.LocalResource;
-import org.apache.hadoop.yarn.api.records.LocalResourceType;
-import org.apache.hadoop.yarn.api.records.LocalResourceVisibility;
-import org.apache.hadoop.yarn.api.records.Resource;
-import org.apache.hadoop.yarn.api.records.URL;
-import org.apache.hadoop.yarn.api.records.impl.pb.LocalResourcePBImpl;
-import org.apache.hadoop.yarn.util.ConverterUtils;
-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.VertexLocationHint.TaskLocationHint;
-import org.apache.tez.dag.api.records.DAGProtos.ConfigurationProto;
-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.PlanKeyValuePair;
-import org.apache.tez.dag.api.records.DAGProtos.PlanLocalResource;
-import org.apache.tez.dag.api.records.DAGProtos.PlanLocalResourceType;
-import org.apache.tez.dag.api.records.DAGProtos.PlanLocalResourceVisibility;
-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.TezEntityDescriptorProto;
-
-import com.google.protobuf.ByteString;
-
-
-public class DagTypeConverters {
-  
-  public static PlanLocalResourceVisibility convertToDAGPlan(LocalResourceVisibility visibility){
-    switch(visibility){
-      case PUBLIC : return PlanLocalResourceVisibility.PUBLIC;  
-      case PRIVATE : return PlanLocalResourceVisibility.PRIVATE;
-      case APPLICATION : return PlanLocalResourceVisibility.APPLICATION;
-      default : throw new RuntimeException("unknown 'visibility': " + visibility);
-    }
-  }
-  
-  public static LocalResourceVisibility convertFromDAGPlan(PlanLocalResourceVisibility visibility){
-    switch(visibility){
-      case PUBLIC : return LocalResourceVisibility.PUBLIC;  
-      case PRIVATE : return LocalResourceVisibility.PRIVATE;
-      case APPLICATION : return LocalResourceVisibility.APPLICATION;
-      default : throw new RuntimeException("unknown 'visibility': " + visibility);
-    }
-  }
-  
-  public static PlanEdgeDataSourceType convertToDAGPlan(DataSourceType sourceType){
-    switch(sourceType){
-      case PERSISTED : return PlanEdgeDataSourceType.PERSISTED;  
-      case PERSISTED_RELIABLE : return PlanEdgeDataSourceType.PERSISTED_RELIABLE;
-      case EPHEMERAL :  return PlanEdgeDataSourceType.EPHEMERAL;
-      default : throw new RuntimeException("unknown 'dataSourceType': " + sourceType);
-    }
-  }
-  
-  public static DataSourceType convertFromDAGPlan(PlanEdgeDataSourceType sourceType){
-    switch(sourceType){
-      case PERSISTED : return DataSourceType.PERSISTED;  
-      case PERSISTED_RELIABLE : return DataSourceType.PERSISTED_RELIABLE;
-      case EPHEMERAL :  return DataSourceType.EPHEMERAL;
-      default : throw new RuntimeException("unknown 'dataSourceType': " + sourceType);
-    }
-  }
-  
-  public static PlanEdgeDataMovementType convertToDAGPlan(DataMovementType type){
-    switch(type){
-      case ONE_TO_ONE : return PlanEdgeDataMovementType.ONE_TO_ONE;  
-      case BROADCAST : return PlanEdgeDataMovementType.BROADCAST;
-      case SCATTER_GATHER : return PlanEdgeDataMovementType.SCATTER_GATHER;
-      default : throw new RuntimeException("unknown 'dataMovementType': " + type);
-    }
-  }
-  
-  public static DataMovementType convertFromDAGPlan(PlanEdgeDataMovementType type){
-    switch(type){
-      case ONE_TO_ONE : return DataMovementType.ONE_TO_ONE;  
-      case BROADCAST : return DataMovementType.BROADCAST;
-      case SCATTER_GATHER : return DataMovementType.SCATTER_GATHER;
-      default : throw new IllegalArgumentException("unknown 'dataMovementType': " + type);
-    }
-  }
-  
-  public static PlanEdgeSchedulingType convertToDAGPlan(SchedulingType type){
-    switch(type){
-      case SEQUENTIAL : return PlanEdgeSchedulingType.SEQUENTIAL;  
-      case CONCURRENT : return PlanEdgeSchedulingType.CONCURRENT;
-      default : throw new RuntimeException("unknown 'SchedulingType': " + type);
-    }
-  }
-  
-  public static SchedulingType convertFromDAGPlan(PlanEdgeSchedulingType type){
-    switch(type){
-      case SEQUENTIAL : return SchedulingType.SEQUENTIAL;  
-      case CONCURRENT : return SchedulingType.CONCURRENT;
-      default : throw new IllegalArgumentException("unknown 'SchedulingType': " + type);
-    }
-  }
-  
-  public static PlanLocalResourceType convertToDAGPlan(LocalResourceType type) {
-    switch(type){
-    case ARCHIVE : return PlanLocalResourceType.ARCHIVE;
-    case FILE : return PlanLocalResourceType.FILE;
-    case PATTERN : return PlanLocalResourceType.PATTERN;
-    default : throw new IllegalArgumentException("unknown 'type': " + type);
-    }
-  }
-  
-  public static LocalResourceType convertFromDAGPlan(PlanLocalResourceType type) {
-    switch(type){
-    case ARCHIVE : return LocalResourceType.ARCHIVE;
-    case FILE : return LocalResourceType.FILE;
-    case PATTERN : return LocalResourceType.PATTERN;
-    default : throw new IllegalArgumentException("unknown 'type': " + type);
-    }
-  }
-
-  public static VertexLocationHint convertFromDAGPlan(
-      List<PlanTaskLocationHint> locationHints) {
-
-    List<TaskLocationHint> outputList = new ArrayList<TaskLocationHint>();  
-    
-    for(PlanTaskLocationHint inputHint : locationHints){
-      TaskLocationHint outputHint = new TaskLocationHint(
-          new HashSet<String>(inputHint.getHostList()),
-          new HashSet<String>(inputHint.getRackList()));
-      outputList.add(outputHint);
-    }
-    return new VertexLocationHint(outputList.size(), outputList);
-  }
-
-  // notes re HDFS URL handling:
-  //   Resource URLs in the protobuf message are strings of the form hdfs://host:port/path 
-  //   org.apache.hadoop.fs.Path.Path  is actually a URI type that allows any scheme
-  //   org.apache.hadoop.yarn.api.records.URL is a URL type used by YARN.
-  //   java.net.URL cannot be used out of the box as it rejects unknown schemes such as HDFS.
-  
-  public static String convertToDAGPlan(URL resource) {
-    // see above notes on HDFS URL handling
-    String out = resource.getScheme() + "://" + resource.getHost() + ":" + resource.getPort() 
-        + resource.getFile();
-    return out;
-  }
-
-  public static Map<String, LocalResource> createLocalResourceMapFromDAGPlan(
-      List<PlanLocalResource> localResourcesList) {
-    Map<String, LocalResource> map = new HashMap<String, LocalResource>();
-    for(PlanLocalResource res : localResourcesList){
-      LocalResource r = new LocalResourcePBImpl();
-      
-      //NOTE: have to check every optional field in protobuf generated classes for existence before accessing
-      //else we will receive a default value back, eg ""
-      if(res.hasPattern()){
-        r.setPattern(res.getPattern());
-      }
-      r.setResource(ConverterUtils.getYarnUrlFromPath(new Path(res.getUri())));  // see above notes on HDFS URL handling
-      r.setSize(res.getSize());
-      r.setTimestamp(res.getTimeStamp());
-      r.setType(DagTypeConverters.convertFromDAGPlan(res.getType()));
-      r.setVisibility(DagTypeConverters.convertFromDAGPlan(res.getVisibility()));
-      map.put(res.getName(), r);
-    }
-    return map;
-  }
-
-  public static Map<String, String> createEnvironmentMapFromDAGPlan(
-      List<PlanKeyValuePair> environmentSettingList) {  
-      
-    Map<String, String> map = new HashMap<String, String>();
-    for(PlanKeyValuePair setting : environmentSettingList){
-      map.put(setting.getKey(), setting.getValue());
-    }
-    
-    return map;
-  }
-  
-  public static Map<String, EdgePlan> createEdgePlanMapFromDAGPlan(List<EdgePlan> edgeList){
-    Map<String, EdgePlan> edgePlanMap =
-        new HashMap<String, EdgePlan>();
-    for(EdgePlan edgePlanItem : edgeList){
-      edgePlanMap.put(edgePlanItem.getId(), edgePlanItem);
-    }
-    return edgePlanMap;
-  }
-  
-  public static EdgeProperty createEdgePropertyMapFromDAGPlan(EdgePlan edge) {
-    return new EdgeProperty(
-        convertFromDAGPlan(edge.getDataMovementType()),
-        convertFromDAGPlan(edge.getDataSourceType()),
-        convertFromDAGPlan(edge.getSchedulingType()),
-        convertOutputDescriptorFromDAGPlan(edge.getEdgeSource()),
-        convertInputDescriptorFromDAGPlan(edge.getEdgeDestination())
-    );
-  }
-
-  public static Resource createResourceRequestFromTaskConfig(
-      PlanTaskConfiguration taskConfig) {
-    return Resource.newInstance(taskConfig.getMemoryMb(), taskConfig.getVirtualCores());
-  }
-
-  public static Map<String, String> convertConfFromProto(
-      ConfigurationProto confProto) {
-    List<PlanKeyValuePair> settingList = confProto.getConfKeyValuesList();
-    Map<String, String> map = new HashMap<String, String>();
-    for(PlanKeyValuePair setting: settingList){
-      map.put(setting.getKey(), setting.getValue());
-    }
-    return map;
-  }
-
-  public static TezEntityDescriptorProto convertToDAGPlan(
-      TezEntityDescriptor descriptor) {
-    TezEntityDescriptorProto.Builder builder = TezEntityDescriptorProto
-        .newBuilder();
-    builder.setClassName(descriptor.getClassName());
-    if (descriptor.getUserPayload() != null) {
-      builder
-          .setUserPayload(ByteString.copyFrom(descriptor.getUserPayload()));
-    }
-    return builder.build();
-  }
-
-  public static InputDescriptor convertInputDescriptorFromDAGPlan(
-      TezEntityDescriptorProto proto) {
-    String className = proto.getClassName();
-    byte[] bb = null;
-    if (proto.hasUserPayload()) {
-      bb = proto.getUserPayload().toByteArray();
-    }
-    return new InputDescriptor(className).setUserPayload(bb);
-  }
-
-  public static OutputDescriptor convertOutputDescriptorFromDAGPlan(
-      TezEntityDescriptorProto proto) {
-    String className = proto.getClassName();
-    byte[] bb = null;
-    if (proto.hasUserPayload()) {
-      bb =  proto.getUserPayload().toByteArray();
-    }
-    return new OutputDescriptor(className).setUserPayload(bb);
-  }
-
-  public static ProcessorDescriptor convertProcessorDescriptorFromDAGPlan(
-      TezEntityDescriptorProto proto) {
-    String className = proto.getClassName();
-    byte[] bb = null;
-    if (proto.hasUserPayload()) {
-      bb = proto.getUserPayload().toByteArray();
-    }
-    return new ProcessorDescriptor(className).setUserPayload(bb);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-dag-api/src/main/java/org/apache/tez/dag/api/Edge.java
----------------------------------------------------------------------
diff --git a/tez-dag-api/src/main/java/org/apache/tez/dag/api/Edge.java b/tez-dag-api/src/main/java/org/apache/tez/dag/api/Edge.java
deleted file mode 100644
index a893bc3..0000000
--- a/tez-dag-api/src/main/java/org/apache/tez/dag/api/Edge.java
+++ /dev/null
@@ -1,59 +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.api;
-
-public class Edge{
-  
-  private final Vertex inputVertex;
-  private final Vertex outputVertex;
-  private final EdgeProperty edgeProperty;
-    
-  public Edge(Vertex inputVertex, 
-               Vertex outputVertex, 
-               EdgeProperty edgeProperty) {
-    this.inputVertex = inputVertex;
-    this.outputVertex = outputVertex;
-    this.edgeProperty = edgeProperty;
-  }
-  
-  // RENAME to source and destination
-  public Vertex getInputVertex() {
-    return inputVertex;
-  }
-  
-  public Vertex getOutputVertex() {
-    return outputVertex;
-  }
-  
-  public EdgeProperty getEdgeProperty() {
-    return edgeProperty;
-  }
-  
-  /*
-   * Used to identify the edge in the configuration
-   */
-  public String getId() {
-    return String.valueOf(this.hashCode());
-  }
- 
-  @Override
-  public String toString() {
-    return inputVertex + " -> " + outputVertex + " (" + edgeProperty + ")";
-  }
-  
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-dag-api/src/main/java/org/apache/tez/dag/api/EdgeProperty.java
----------------------------------------------------------------------
diff --git a/tez-dag-api/src/main/java/org/apache/tez/dag/api/EdgeProperty.java b/tez-dag-api/src/main/java/org/apache/tez/dag/api/EdgeProperty.java
deleted file mode 100644
index 326d3d0..0000000
--- a/tez-dag-api/src/main/java/org/apache/tez/dag/api/EdgeProperty.java
+++ /dev/null
@@ -1,147 +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.api;
-
-public class EdgeProperty {
-  
-  /**
-   * Defines the manner of data movement between source and destination tasks.
-   * Determines which destination tasks have access to data produced on this
-   * edge by a source task. A destination task may choose to read any portion of
-   * the data available to it.
-   */
-  public enum DataMovementType {
-    /**
-     * Output on this edge produced by the i-th source task is available to the 
-     * i-th destination task.
-     */
-    ONE_TO_ONE,
-    /**
-     * Output on this edge produced by any source task is available to all
-     * destination tasks.
-     */
-    BROADCAST,
-    /**
-     * The i-th output on this edge produced by all source tasks is available to
-     * the same destination task. Source tasks scatter their outputs and they
-     * are gathered by designated destination tasks.
-     */
-    SCATTER_GATHER
-  }
-  
-  /**
-   * Determines the lifetime of the data produced on this edge by a source task.
-   */
-  public enum DataSourceType {
-    /**
-     * Data produced by the source is persisted and available even when the
-     * task is not running. The data may become unavailable and may cause the 
-     * source task to be re-executed.
-     */
-    PERSISTED,
-    /**
-     * Source data is stored reliably and will always be available
-     */
-    PERSISTED_RELIABLE,
-    /**
-     * Data produced by the source task is available only while the source task
-     * is running. This requires the destination task to run concurrently with 
-     * the source task.
-     */
-    EPHEMERAL
-  }
-  
-  /**
-   * Determines when the destination task is eligible to run, once the source  
-   * task is eligible to run.
-   */
-  public enum SchedulingType {
-    /**
-     * Destination task is eligible to run after one or more of its source tasks 
-     * have started or completed.
-     */
-    SEQUENTIAL,
-    /**
-     * Destination task must run concurrently with the source task
-     */
-    CONCURRENT
-  }
-  
-  DataMovementType dataMovementType;
-  DataSourceType dataSourceType;
-  SchedulingType schedulingType;
-  InputDescriptor inputDescriptor;
-  OutputDescriptor outputDescriptor;
-  
-  /**
-   * @param dataMovementType
-   * @param dataSourceType
-   * @param edgeSource
-   *          The {@link OutputDescriptor} that generates data on the edge.
-   * @param edgeDestination
-   *          The {@link InputDescriptor} which will consume data from the edge.
-   */
-  public EdgeProperty(DataMovementType dataMovementType, 
-                       DataSourceType dataSourceType,
-                       SchedulingType schedulingType,
-                       OutputDescriptor edgeSource,
-                       InputDescriptor edgeDestination) {
-    this.dataMovementType = dataMovementType;
-    this.dataSourceType = dataSourceType;
-    this.schedulingType = schedulingType;
-    this.inputDescriptor = edgeDestination;
-    this.outputDescriptor = edgeSource;
-  }
-  
-  public DataMovementType getDataMovementType() {
-    return dataMovementType;
-  }
-  
-  public DataSourceType getDataSourceType() {
-    return dataSourceType;
-  }
-  
-  public SchedulingType getSchedulingType() {
-    return schedulingType;
-  }
-  
-  /**
-   * Returns the {@link InputDescriptor} which will consume data from the edge.
-   * 
-   * @return
-   */
-  public InputDescriptor getEdgeDestination() {
-    return inputDescriptor;
-  }
-  
-  /**
-   * Returns the {@link OutputDescriptor} which produces data on the edge.
-   * 
-   * @return
-   */
-  public OutputDescriptor getEdgeSource() {
-    return outputDescriptor;
-  }
-  
-  @Override
-  public String toString() {
-    return "{ " + dataMovementType + " : " + inputDescriptor.getClassName()
-        + " >> " + dataSourceType + " >> " + outputDescriptor.getClassName() + " }";
-  }
-  
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-dag-api/src/main/java/org/apache/tez/dag/api/InputDescriptor.java
----------------------------------------------------------------------
diff --git a/tez-dag-api/src/main/java/org/apache/tez/dag/api/InputDescriptor.java b/tez-dag-api/src/main/java/org/apache/tez/dag/api/InputDescriptor.java
deleted file mode 100644
index dea9001..0000000
--- a/tez-dag-api/src/main/java/org/apache/tez/dag/api/InputDescriptor.java
+++ /dev/null
@@ -1,32 +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.api;
-
-public class InputDescriptor extends TezEntityDescriptor {
-
-  public InputDescriptor(String inputClassName) {
-    super(inputClassName);
-  }
-
-  @Override
-  public InputDescriptor setUserPayload(byte[] userPayload) {
-    this.userPayload = userPayload;
-    return this;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-dag-api/src/main/java/org/apache/tez/dag/api/OutputDescriptor.java
----------------------------------------------------------------------
diff --git a/tez-dag-api/src/main/java/org/apache/tez/dag/api/OutputDescriptor.java b/tez-dag-api/src/main/java/org/apache/tez/dag/api/OutputDescriptor.java
deleted file mode 100644
index 16fb9b1..0000000
--- a/tez-dag-api/src/main/java/org/apache/tez/dag/api/OutputDescriptor.java
+++ /dev/null
@@ -1,32 +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.api;
-
-public class OutputDescriptor extends TezEntityDescriptor {
-
-  public OutputDescriptor(String outputClassName) {
-    super(outputClassName);
-  }
-
-  @Override
-  public OutputDescriptor setUserPayload(byte[] userPayload) {
-    this.userPayload = userPayload;
-    return this;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-dag-api/src/main/java/org/apache/tez/dag/api/ProcessorDescriptor.java
----------------------------------------------------------------------
diff --git a/tez-dag-api/src/main/java/org/apache/tez/dag/api/ProcessorDescriptor.java b/tez-dag-api/src/main/java/org/apache/tez/dag/api/ProcessorDescriptor.java
deleted file mode 100644
index 092147d..0000000
--- a/tez-dag-api/src/main/java/org/apache/tez/dag/api/ProcessorDescriptor.java
+++ /dev/null
@@ -1,31 +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.api;
-
-public class ProcessorDescriptor extends TezEntityDescriptor {
-
-  public ProcessorDescriptor(String processorClassName) {
-    super(processorClassName);
-  }
-
-  public ProcessorDescriptor setUserPayload(byte[] userPayload) {
-    this.userPayload = userPayload;
-    return this;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-dag-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java
----------------------------------------------------------------------
diff --git a/tez-dag-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java b/tez-dag-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java
deleted file mode 100644
index 7447974..0000000
--- a/tez-dag-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java
+++ /dev/null
@@ -1,223 +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.api;
-
-import org.apache.hadoop.conf.Configuration;
-
-public class TezConfiguration extends Configuration {
-
-  public final static String TEZ_SITE_XML = "tez-site.xml";
-
-  static {
-    addDefaultResource(TEZ_SITE_XML);
-  }
-
-  public TezConfiguration() {
-    super();
-  }
-
-  public TezConfiguration(Configuration conf) {
-    super(conf);
-  }
-
-  public static final String TEZ_PREFIX = "tez.";
-  public static final String TEZ_AM_PREFIX = TEZ_PREFIX + "am.";
-  public static final String TEZ_TASK_PREFIX = TEZ_PREFIX + "task.";
-
-  public static final String TEZ_AM_STAGING_DIR = TEZ_PREFIX + "staging-dir";
-  public static final String TEZ_AM_STAGING_DIR_DEFAULT = "/tmp/tez/staging";
-
-  // TODO Should not be required once all tokens are handled via AppSubmissionContext
-  public static final String JOB_SUBMIT_DIR = TEZ_PREFIX + "jobSubmitDir";
-  public static final String APPLICATION_TOKENS_FILE = "appTokens";
-  public static final String TEZ_APPLICATION_MASTER_CLASS =
-      "org.apache.tez.dag.app.DAGAppMaster";
-
-  /** Root Logging level passed to the Tez app master.*/
-  public static final String TEZ_AM_LOG_LEVEL = TEZ_AM_PREFIX+"log.level";
-  public static final String TEZ_AM_LOG_LEVEL_DEFAULT = "INFO";
-
-  public static final String TEZ_AM_JAVA_OPTS = TEZ_AM_PREFIX
-      + "java.opts";
-  public static final String DEFAULT_TEZ_AM_JAVA_OPTS = " -Xmx1024m ";
-
-  public static final String TEZ_AM_CANCEL_DELEGATION_TOKEN = TEZ_AM_PREFIX +
-      "am.complete.cancel.delegation.tokens";
-  public static final boolean TEZ_AM_CANCEL_DELEGATION_TOKEN_DEFAULT = true;
-
-  public static final String TEZ_AM_TASK_LISTENER_THREAD_COUNT =
-      TEZ_AM_PREFIX + "task.listener.thread-count";
-  public static final int TEZ_AM_TASK_LISTENER_THREAD_COUNT_DEFAULT = 30;
-
-  public static final String TEZ_AM_CONTAINER_LISTENER_THREAD_COUNT =
-      TEZ_AM_PREFIX + "container.listener.thread-count";
-  public static final int TEZ_AM_CONTAINER_LISTENER_THREAD_COUNT_DEFAULT = 30;
-
-  // TODO Some of the DAG properties are job specific and not AM specific. Rename accordingly.
-  // TODO Are any of these node blacklisting properties required. (other than for MR compat)
-  public static final String TEZ_AM_MAX_TASK_FAILURES_PER_NODE = TEZ_AM_PREFIX
-      + "maxtaskfailures.per.node";
-  public static final int TEZ_AM_MAX_TASK_FAILURES_PER_NODE_DEFAULT = 3;
-
-  public static final String TEZ_AM_MAX_TASK_ATTEMPTS =
-      TEZ_AM_PREFIX + "max.task.attempts";
-  public static final int TEZ_AM_MAX_TASK_ATTEMPTS_DEFAULT = 4;
-
-  public static final String TEZ_AM_NODE_BLACKLISTING_ENABLED = TEZ_AM_PREFIX
-      + "node-blacklisting.enabled";
-  public static final boolean TEZ_AM_NODE_BLACKLISTING_ENABLED_DEFAULT = true;
-  public static final String TEZ_AM_NODE_BLACKLISTING_IGNORE_THRESHOLD = TEZ_AM_PREFIX
-      + "node-blacklisting.ignore-threshold-node-percent";
-  public static final int TEZ_AM_NODE_BLACKLISTING_IGNORE_THRESHOLD_DEFAULT = 33;
-
-  /** Number of threads to handle job client RPC requests.*/
-  public static final String TEZ_AM_CLIENT_THREAD_COUNT =
-      TEZ_AM_PREFIX + "client.am.thread-count";
-  public static final int TEZ_AM_CLIENT_THREAD_COUNT_DEFAULT = 1;
-  /**
-   * Range of ports that the AM can use when binding. Leave blank
-   * if you want all possible ports.
-   */
-  public static final String TEZ_AM_CLIENT_AM_PORT_RANGE =
-      TEZ_AM_PREFIX + "client.am.port-range";
-
-
-  public static final String TEZ_AM_RESOURCE_MEMORY_MB = TEZ_AM_PREFIX
-      + "resource.memory.mb";
-  public static final int TEZ_AM_RESOURCE_MEMORY_MB_DEFAULT = 1536;
-
-  public static final String TEZ_AM_RESOURCE_CPU_VCORES = TEZ_AM_PREFIX
-      + "resource.cpu.vcores";
-  public static final int TEZ_AM_RESOURCE_CPU_VCORES_DEFAULT = 1;
-
-  public static final String
-          TEZ_AM_SHUFFLE_VERTEX_MANAGER_MIN_SRC_FRACTION = TEZ_AM_PREFIX
-          + "shuffle-vertex-manager.min-src-fraction";
-  public static final float
-          TEZ_AM_SHUFFLE_VERTEX_MANAGER_MIN_SRC_FRACTION_DEFAULT = 0.25f;
-
-  public static final String
-          TEZ_AM_SHUFFLE_VERTEX_MANAGER_MAX_SRC_FRACTION = TEZ_AM_PREFIX
-          + "shuffle-vertex-manager.max-src-fraction";
-  public static final float
-          TEZ_AM_SHUFFLE_VERTEX_MANAGER_MAX_SRC_FRACTION_DEFAULT = 0.75f;
-
-  public static final String
-          TEZ_AM_SHUFFLE_VERTEX_MANAGER_ENABLE_AUTO_PARALLEL = TEZ_AM_PREFIX +
-          "shuffle-vertex-manager.enable.auto-parallel";
-  public static final boolean
-          TEZ_AM_SHUFFLE_VERTEX_MANAGER_ENABLE_AUTO_PARALLEL_DEFAULT = false;
-
-  public static final String
-          TEZ_AM_SHUFFLE_VERTEX_MANAGER_DESIRED_TASK_INPUT_SIZE = TEZ_AM_PREFIX +
-          "shuffle-vertex-manager.desired-task-input-size";
-  public static final long
-          TEZ_AM_SHUFFLE_VERTEX_MANAGER_DESIRED_TASK_INPUT_SIZE_DEFAULT =
-          1024*1024*100L;
-
-  public static final String
-          TEZ_AM_SHUFFLE_VERTEX_MANAGER_MIN_TASK_PARALLELISM = TEZ_AM_PREFIX +
-          "shuffle-vertex-manager.min-task-parallelism";
-  public static final int
-          TEZ_AM_SHUFFLE_VERTEX_MANAGER_MIN_TASK_PARALLELISM_DEFAULT = 1;
-
-  public static final String
-          TEZ_AM_SLOWSTART_DAG_SCHEDULER_MIN_SHUFFLE_RESOURCE_FRACTION = TEZ_AM_PREFIX
-          + "slowstart-dag-scheduler.min-resource-fraction";
-  public static final float
-          TEZ_AM_SLOWSTART_DAG_SCHEDULER_MIN_SHUFFLE_RESOURCE_FRACTION_DEFAULT = 0.5f;
-
-  public static final String TEZ_AM_AGGRESSIVE_SCHEDULING = TEZ_AM_PREFIX +
-      "aggressive.scheduling";
-  public static boolean TEZ_AM_AGGRESSIVE_SCHEDULING_DEFAULT = false;
-
-  /**
-   * The complete path to the serialized dag plan file
-   * <code>TEZ_AM_PLAN_PB_BINARY</code>. Used to make the plan available to
-   * individual tasks if needed. This will typically be a path in the job submit
-   * directory.
-   */
-  public static final String TEZ_AM_PLAN_REMOTE_PATH = TEZ_AM_PREFIX
-      + "dag-am-plan.remote.path";
-
-  public static final String TEZ_AM_RM_HEARTBEAT_INTERVAL_MS_MAX = TEZ_AM_PREFIX
-      + "am-rm.heartbeat.interval-ms.max";
-  public static final int TEZ_AM_RM_HEARTBEAT_INTERVAL_MS_MAX_DEFAULT = 1000;
-
-  public static final String TEZ_TASK_GET_TASK_SLEEP_INTERVAL_MS_MAX = TEZ_TASK_PREFIX
-      + "get-task.sleep.interval-ms.max";
-  public static final int TEZ_TASK_GET_TASK_SLEEP_INTERVAL_MS_MAX_DEFAULT = 500;
-
-  public static final String TEZ_TASK_AM_HEARTBEAT_INTERVAL_MS = TEZ_TASK_PREFIX
-      + "am.heartbeat.interval-ms.max";
-  public static final int TEZ_TASK_AM_HEARTBEAT_INTERVAL_MS_DEFAULT = 100;
-
-  public static final String TEZ_TASK_MAX_EVENTS_PER_HEARTBEAT = TEZ_TASK_PREFIX
-      + "max-events-per-heartbeat.max";
-  public static final int TEZ_TASK_MAX_EVENTS_PER_HEARTBEAT_DEFAULT = 100;
-
-  /**
-   * Configuration to specify whether container should be reused.
-   */
-  public static final String TEZ_AM_CONTAINER_REUSE_ENABLED = TEZ_AM_PREFIX
-      + "container.reuse.enabled";
-  public static final boolean TEZ_AM_CONTAINER_REUSE_ENABLED_DEFAULT = true;
-
-  /**
-   * Whether to reuse containers for rack local tasks. Active only if reuse is
-   * enabled.
-   */
-  public static final String TEZ_AM_CONTAINER_REUSE_RACK_FALLBACK_ENABLED = TEZ_AM_PREFIX
-      + "container.reuse.rack-fallback.enabled";
-  public static final boolean TEZ_AM_CONTAINER_REUSE_RACK_FALLBACK_ENABLED_DEFAULT = true;
-
-  /**
-   * Whether to reuse containers for non-local tasks. Active only if reuse is
-   * enabled.
-   */
-  public static final String TEZ_AM_CONTAINER_REUSE_NON_LOCAL_FALLBACK_ENABLED = TEZ_AM_PREFIX
-      + "container.reuse.non-local-fallback.enabled";
-  public static final boolean TEZ_AM_CONTAINER_REUSE_NON_LOCAL_FALLBACK_ENABLED_DEFAULT = false;
-
-  public static final String TEZ_AM_CONTAINER_REUSE_DELAY_ALLOCATION_MILLIS = TEZ_AM_PREFIX
-      + "container.reuse.delay-allocation-millis";
-  public static final long TEZ_AM_CONTAINER_REUSE_DELAY_ALLOCATION_MILLIS_DEFAULT = 3000l;
-
-  public static final String TEZ_PB_BINARY_CONF_NAME = "tez-conf.pb";
-  public static final String TEZ_PB_PLAN_BINARY_NAME = "tez-dag.pb";
-  public static final String TEZ_PB_PLAN_TEXT_NAME = "tez-dag.pb.txt";
-
-  /*
-   * Logger properties
-   */
-  public static final String TEZ_CONTAINER_LOG4J_PROPERTIES_FILE = "tez-container-log4j.properties";
-  public static final String TEZ_CONTAINER_LOGGER_NAME = "CLA";
-  public static final String TEZ_ROOT_LOGGER_NAME = "tez.root.logger";
-  public static final String TEZ_CONTAINER_LOG_FILE_NAME = "syslog";
-  public static final String TEZ_CONTAINER_ERR_FILE_NAME = "stderr";
-  public static final String TEZ_CONTAINER_OUT_FILE_NAME = "stdout";
-
-
-  public static final String TEZ_LIB_URIS =
-      TEZ_PREFIX + "lib.uris";
-
-  public static final String TEZ_APPLICATION_TYPE = "TEZ-MR*";
-
-  public static final String LOCAL_FRAMEWORK_NAME = "local-tez";
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-dag-api/src/main/java/org/apache/tez/dag/api/TezConstants.java
----------------------------------------------------------------------
diff --git a/tez-dag-api/src/main/java/org/apache/tez/dag/api/TezConstants.java b/tez-dag-api/src/main/java/org/apache/tez/dag/api/TezConstants.java
deleted file mode 100644
index 5463d65..0000000
--- a/tez-dag-api/src/main/java/org/apache/tez/dag/api/TezConstants.java
+++ /dev/null
@@ -1,29 +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.api;
-
-/**
- * Specifies all constant values in Tez
- */
-public class TezConstants {
-
-  // Env variable names
-  public static final String TEZ_AM_IS_SESSION_ENV = "TEZ_AM_IS_SESSION";
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-dag-api/src/main/java/org/apache/tez/dag/api/TezEntityDescriptor.java
----------------------------------------------------------------------
diff --git a/tez-dag-api/src/main/java/org/apache/tez/dag/api/TezEntityDescriptor.java b/tez-dag-api/src/main/java/org/apache/tez/dag/api/TezEntityDescriptor.java
deleted file mode 100644
index 9d4b2c4..0000000
--- a/tez-dag-api/src/main/java/org/apache/tez/dag/api/TezEntityDescriptor.java
+++ /dev/null
@@ -1,42 +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.api;
-
-public abstract class TezEntityDescriptor {
-
-  protected byte[] userPayload;
-  private String className;
-
-  public TezEntityDescriptor(String className) {
-    this.className = className;
-  }
-
-  public byte[] getUserPayload() {
-    return this.userPayload;
-  }
-
-  public TezEntityDescriptor setUserPayload(byte[] userPayload) {
-    this.userPayload = userPayload;
-    return this;
-  }
-
-  public String getClassName() {
-    return this.className;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-dag-api/src/main/java/org/apache/tez/dag/api/TezException.java
----------------------------------------------------------------------
diff --git a/tez-dag-api/src/main/java/org/apache/tez/dag/api/TezException.java b/tez-dag-api/src/main/java/org/apache/tez/dag/api/TezException.java
deleted file mode 100644
index e3b14e7..0000000
--- a/tez-dag-api/src/main/java/org/apache/tez/dag/api/TezException.java
+++ /dev/null
@@ -1,31 +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.api;
-
-/**
- *  Base TezException
- */
-public class TezException extends Exception {
-  private static final long serialVersionUID = 6337442733802964447L;
-  public TezException(Throwable cause) { super(cause); }
-  public TezException(String message) { super(message); }
-  public TezException(String message, Throwable cause) {
-    super(message, cause);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-dag-api/src/main/java/org/apache/tez/dag/api/TezUncheckedException.java
----------------------------------------------------------------------
diff --git a/tez-dag-api/src/main/java/org/apache/tez/dag/api/TezUncheckedException.java b/tez-dag-api/src/main/java/org/apache/tez/dag/api/TezUncheckedException.java
deleted file mode 100644
index f55f6dd..0000000
--- a/tez-dag-api/src/main/java/org/apache/tez/dag/api/TezUncheckedException.java
+++ /dev/null
@@ -1,33 +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.api;
-
-/**
- * Base Tez Unchecked Exception
- */
-public class TezUncheckedException extends RuntimeException {
-
-  private static final long serialVersionUID = -4956339297375386184L;
-  
-  public TezUncheckedException(Throwable cause) { super(cause); }
-  public TezUncheckedException(String message) { super(message); }
-  public TezUncheckedException(String message, Throwable cause) {
-    super(message, cause);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-dag-api/src/main/java/org/apache/tez/dag/api/Vertex.java
----------------------------------------------------------------------
diff --git a/tez-dag-api/src/main/java/org/apache/tez/dag/api/Vertex.java b/tez-dag-api/src/main/java/org/apache/tez/dag/api/Vertex.java
deleted file mode 100644
index 900822b..0000000
--- a/tez-dag-api/src/main/java/org/apache/tez/dag/api/Vertex.java
+++ /dev/null
@@ -1,153 +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.api;
-
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.hadoop.yarn.api.records.LocalResource;
-import org.apache.hadoop.yarn.api.records.Resource;
-import org.apache.tez.dag.api.VertexLocationHint.TaskLocationHint;
-
-public class Vertex { // FIXME rename to Task
-
-  private final String vertexName;
-  private final ProcessorDescriptor processorDescriptor;
-
-  private final int parallelism;
-  private VertexLocationHint taskLocationsHint;
-  private final Resource taskResource;
-  private Map<String, LocalResource> taskLocalResources;
-  private Map<String, String> taskEnvironment;
-
-  private final List<Vertex> inputVertices = new ArrayList<Vertex>();
-  private final List<Vertex> outputVertices = new ArrayList<Vertex>();
-  private final List<String> inputEdgeIds = new ArrayList<String>();
-  private final List<String> outputEdgeIds = new ArrayList<String>();
-  private String javaOpts = "";
-
-
-  public Vertex(String vertexName,
-      ProcessorDescriptor processorDescriptor,
-      int parallelism,
-      Resource taskResource) {
-    this.vertexName = vertexName;
-    this.processorDescriptor = processorDescriptor;
-    this.parallelism = parallelism;
-    this.taskResource = taskResource;
-    if (parallelism == 0) {
-      throw new IllegalArgumentException("Parallelism cannot be 0");
-    }
-    if (taskResource == null) {
-      throw new IllegalArgumentException("Resource cannot be null");
-    }
-  }
-
-  public String getVertexName() { // FIXME rename to getName()
-    return vertexName;
-  }
-
-  public ProcessorDescriptor getProcessorDescriptor() {
-    return this.processorDescriptor;
-  }
-
-  public int getParallelism() {
-    return parallelism;
-  }
-
-  public Resource getTaskResource() {
-    return taskResource;
-  }
-
-  public Vertex setTaskLocationsHint(List<TaskLocationHint> locations) {
-    if (locations == null) {
-      return this;
-    }
-    assert locations.size() == parallelism;
-    taskLocationsHint = new VertexLocationHint(parallelism, locations);
-    return this;
-  }
-
-  // used internally to create parallelism location resource file
-  VertexLocationHint getTaskLocationsHint() {
-    return taskLocationsHint;
-  }
-
-  public Vertex setTaskLocalResources(Map<String, LocalResource> localResources) {
-    this.taskLocalResources = localResources;
-    return this;
-  }
-
-  public Map<String, LocalResource> getTaskLocalResources() {
-    return taskLocalResources;
-  }
-
-  public Vertex setTaskEnvironment(Map<String, String> environment) {
-    this.taskEnvironment = environment;
-    return this;
-  }
-
-  public Map<String, String> getTaskEnvironment() {
-    return taskEnvironment;
-  }
-
-  public Vertex setJavaOpts(String javaOpts){
-     this. javaOpts = javaOpts;
-     return this;
-  }
-
-  public String getJavaOpts(){
-	  return javaOpts;
-  }
-
-  @Override
-  public String toString() {
-    return "[" + vertexName + " : " + processorDescriptor.getClassName() + "]";
-  }
-
-  void addInputVertex(Vertex inputVertex, String edgeId) {
-    inputVertices.add(inputVertex);
-    inputEdgeIds.add(edgeId);
-  }
-
-  void addOutputVertex(Vertex outputVertex, String edgeId) {
-    outputVertices.add(outputVertex);
-    outputEdgeIds.add(edgeId);
-  }
-
-  List<Vertex> getInputVertices() {
-    return inputVertices;
-  }
-
-  List<Vertex> getOutputVertices() {
-    return outputVertices;
-  }
-
-  List<String> getInputEdgeIds() {
-    return inputEdgeIds;
-  }
-
-  List<String> getOutputEdgeIds() {
-    return outputEdgeIds;
-  }
-
-  // FIXME how do we support profiling? Can't profile all tasks.
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-dag-api/src/main/java/org/apache/tez/dag/api/VertexLocationHint.java
----------------------------------------------------------------------
diff --git a/tez-dag-api/src/main/java/org/apache/tez/dag/api/VertexLocationHint.java b/tez-dag-api/src/main/java/org/apache/tez/dag/api/VertexLocationHint.java
deleted file mode 100644
index 4f19314..0000000
--- a/tez-dag-api/src/main/java/org/apache/tez/dag/api/VertexLocationHint.java
+++ /dev/null
@@ -1,154 +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.api;
-
-import java.util.Collections;
-import java.util.List;
-import java.util.Set;
-
-public class VertexLocationHint  {
-
-  private final int numTasks;
-  private final List<TaskLocationHint> taskLocationHints;
-
-  public VertexLocationHint(int numTasks,
-      List<TaskLocationHint> taskLocationHints) {
-    this.numTasks = numTasks;
-    if (taskLocationHints != null) {
-      this.taskLocationHints = Collections.unmodifiableList(taskLocationHints);
-    } else {
-      this.taskLocationHints = null;
-    }
-  }
-
-  public int getNumTasks() {
-    return numTasks;
-  }
-
-  public List<TaskLocationHint> getTaskLocationHints() {
-    return taskLocationHints;
-  }
-
-  @Override
-  public int hashCode() {
-    final int prime = 7883;
-    int result = 1;
-    result = prime * result + numTasks;
-    if (taskLocationHints != null) {
-      result = prime * result + taskLocationHints.hashCode();
-    }
-    return result;
-  }
-
-  @Override
-  public boolean equals(Object obj) {
-    if (this == obj) {
-      return true;
-    }
-    if (obj == null) {
-      return false;
-    }
-    if (getClass() != obj.getClass()) {
-      return false;
-    }
-    VertexLocationHint other = (VertexLocationHint) obj;
-    if (numTasks != other.numTasks) {
-      return false;
-    }
-    if (taskLocationHints != null) {
-      if (!taskLocationHints.equals(other.taskLocationHints)) {
-        return false;
-      }
-    } else if (other.taskLocationHints != null) {
-      return false;
-    }
-    return true;
-  }
-
-  public static class TaskLocationHint {
-
-    // Host names if any to be used
-    private final Set<String> hosts;
-    // Rack names if any to be used
-    private final Set<String> racks;
-
-    public TaskLocationHint(Set<String> hosts, Set<String> racks) {
-      if (hosts != null) {
-        this.hosts = Collections.unmodifiableSet(hosts);
-      } else {
-        this.hosts = null;
-      }
-      if (racks != null) {
-        this.racks = Collections.unmodifiableSet(racks);
-      } else {
-        this.racks = null;
-      }
-    }
-
-    public Set<String> getDataLocalHosts() {
-      return hosts;
-    }
-
-    public Set<String> getRacks() {
-      return racks;
-    }
-
-    @Override
-    public int hashCode() {
-      final int prime = 9397;
-      int result = 1;
-      result = ( hosts != null) ?
-          prime * result + hosts.hashCode() :
-          result + prime;
-      result = ( racks != null) ?
-          prime * result + racks.hashCode() :
-          result + prime;
-      return result;
-    }
-
-    @Override
-    public boolean equals(Object obj) {
-      if (this == obj) {
-        return true;
-      }
-      if (obj == null) {
-        return false;
-      }
-      if (getClass() != obj.getClass()) {
-        return false;
-      }
-      TaskLocationHint other = (TaskLocationHint) obj;
-      if (hosts != null) {
-        if (!hosts.equals(other.hosts)) {
-          return false;
-        }
-      } else if (other.hosts != null) {
-        return false;
-      }
-      if (racks != null) {
-        if (!racks.equals(other.racks)) {
-          return false;
-        }
-      } else if (other.racks != null) {
-        return false;
-      }
-      return true;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-dag-api/src/main/java/org/apache/tez/dag/api/client/DAGClient.java
----------------------------------------------------------------------
diff --git a/tez-dag-api/src/main/java/org/apache/tez/dag/api/client/DAGClient.java b/tez-dag-api/src/main/java/org/apache/tez/dag/api/client/DAGClient.java
deleted file mode 100644
index 9062e8e..0000000
--- a/tez-dag-api/src/main/java/org/apache/tez/dag/api/client/DAGClient.java
+++ /dev/null
@@ -1,67 +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.api.client;
-
-import java.io.Closeable;
-import java.io.IOException;
-
-import org.apache.hadoop.classification.InterfaceAudience.Private;
-import org.apache.hadoop.yarn.api.records.ApplicationId;
-import org.apache.hadoop.yarn.api.records.ApplicationReport;
-import org.apache.tez.dag.api.TezException;
-
-/*
- * Interface class for monitoring the <code>DAG</code> running in a Tez DAG
- * Application Master.
- */
-public interface DAGClient extends Closeable {
-
-  /**
-   * Get the YARN ApplicationId for the app running the DAG
-   * @return <code>ApplicationId</code>
-   */
-  public ApplicationId getApplicationId();
-
-  @Private
-  /**
-   * Get the YARN ApplicationReport for the app running the DAG. For performance
-   * reasons this may be stale copy and should be used to access static info. It
-   * may be null.
-   * @return <code>ApplicationReport</code> or null
-   */
-  public ApplicationReport getApplicationReport();
-
-  /**
-   * Get the status of the specified DAG
-   */
-  public DAGStatus getDAGStatus() throws IOException, TezException;
-
-  /**
-   * Get the status of a Vertex of a DAG
-   */
-  public VertexStatus getVertexStatus(String vertexName)
-      throws IOException, TezException;
-
-  /**
-   * Kill a running DAG
-   *
-   */
-  public void tryKillDAG() throws TezException, IOException;
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-dag-api/src/main/java/org/apache/tez/dag/api/client/DAGStatus.java
----------------------------------------------------------------------
diff --git a/tez-dag-api/src/main/java/org/apache/tez/dag/api/client/DAGStatus.java b/tez-dag-api/src/main/java/org/apache/tez/dag/api/client/DAGStatus.java
deleted file mode 100644
index d61173d..0000000
--- a/tez-dag-api/src/main/java/org/apache/tez/dag/api/client/DAGStatus.java
+++ /dev/null
@@ -1,130 +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.api.client;
-
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.tez.dag.api.records.DAGProtos.DAGStatusProtoOrBuilder;
-import org.apache.tez.dag.api.records.DAGProtos.StringProgressPairProto;
-import org.apache.tez.dag.api.TezUncheckedException;
-
-public class DAGStatus {
-
-  public enum State {
-    SUBMITTED,
-    INITING,
-    RUNNING,
-    SUCCEEDED,
-    KILLED,
-    FAILED,
-    ERROR,
-  };
-
-  DAGStatusProtoOrBuilder proxy = null;
-  Progress progress = null;
-  Map<String, Progress> vertexProgress = null;
-
-  public DAGStatus(DAGStatusProtoOrBuilder proxy) {
-    this.proxy = proxy;
-  }
-
-  public State getState() {
-    switch(proxy.getState()) {
-    case DAG_SUBMITTED:
-      return DAGStatus.State.SUBMITTED;
-    // For simplicity, initing/terminating states are presented as running
-    case DAG_INITING:
-    case DAG_TERMINATING:
-    case DAG_RUNNING:
-      return DAGStatus.State.RUNNING;
-    case DAG_SUCCEEDED:
-      return DAGStatus.State.SUCCEEDED;
-    case DAG_FAILED:
-      return DAGStatus.State.FAILED;
-    case DAG_KILLED:
-      return DAGStatus.State.KILLED;
-    case DAG_ERROR:
-      return DAGStatus.State.ERROR;
-    default:
-      throw new TezUncheckedException("Unsupported value for DAGStatus.State : " +
-                              proxy.getState());
-    }
-  }
-
-  public boolean isCompleted() {
-    State state = getState();
-    return (state == State.SUCCEEDED ||
-             state == State.FAILED ||
-             state == State.KILLED ||
-             state == State.ERROR);
-  }
-
-  public List<String> getDiagnostics() {
-    return proxy.getDiagnosticsList();
-  }
-
-  /**
-   * Gets overall progress value of the DAG.
-   *
-   * @return Progress of the DAG. Maybe null when the DAG is not running. Maybe
-   *         null when the DAG is running and the application master cannot be
-   *         reached - e.g. when the execution platform has restarted the
-   *         application master.
-   * @see Progress
-   */
-  public Progress getDAGProgress() {
-    if(progress == null && proxy.hasDAGProgress()) {
-      progress = new Progress(proxy.getDAGProgress());
-    }
-    return progress;
-  }
-
-  /**
-   * Get the progress of a vertex in the DAG
-   *
-   * @return Progress of the vertex. May be null when the DAG is not running.
-   *         Maybe null when the DAG is running and the application master
-   *         cannot be reached - e.g. when the execution platform has restarted
-   *         the application master.
-   * @see Progress
-   */
-  public Map<String, Progress> getVertexProgress() {
-    if(vertexProgress == null) {
-      if(proxy.getVertexProgressList() != null) {
-        List<StringProgressPairProto> kvList = proxy.getVertexProgressList();
-        vertexProgress = new HashMap<String, Progress>(kvList.size());
-        for(StringProgressPairProto kv : kvList){
-          vertexProgress.put(kv.getKey(), new Progress(kv.getProgress()));
-        }
-      }
-    }
-    return vertexProgress;
-  }
-
-  @Override
-  public String toString() {
-    StringBuilder sb = new StringBuilder();
-    sb.append("status=" + getState()
-        + ", progress=" + getDAGProgress());
-    return sb.toString();
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-dag-api/src/main/java/org/apache/tez/dag/api/client/Progress.java
----------------------------------------------------------------------
diff --git a/tez-dag-api/src/main/java/org/apache/tez/dag/api/client/Progress.java b/tez-dag-api/src/main/java/org/apache/tez/dag/api/client/Progress.java
deleted file mode 100644
index 9577320..0000000
--- a/tez-dag-api/src/main/java/org/apache/tez/dag/api/client/Progress.java
+++ /dev/null
@@ -1,67 +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.api.client;
-
-import org.apache.tez.dag.api.records.DAGProtos.ProgressProtoOrBuilder;
-
-public class Progress {
-  
-  ProgressProtoOrBuilder proxy = null;
-  
-  Progress(ProgressProtoOrBuilder proxy) {
-    this.proxy = proxy;
-  }
-  
-  public int getTotalTaskCount() {
-    return proxy.getTotalTaskCount();
-  }
-
-  public int getSucceededTaskCount() {
-    return proxy.getSucceededTaskCount();
-  }
-
-  public int getRunningTaskCount() {
-    return proxy.getRunningTaskCount();
-  }
-
-  public int getFailedTaskCount() {
-    return proxy.getFailedTaskCount();
-  }
-
-  public int getKilledTaskCount() {
-    return proxy.getKilledTaskCount();
-  }
-  
-  @Override
-  public String toString() {
-    StringBuilder sb = new StringBuilder();
-    sb.append("TotalTasks: ");
-    sb.append(getTotalTaskCount());
-    sb.append(" Succeeded: ");
-    sb.append(getSucceededTaskCount());
-    sb.append(" Running: ");
-    sb.append(getRunningTaskCount());
-    sb.append(" Failed: "); 
-    sb.append(getFailedTaskCount());
-    sb.append(" Killed: "); 
-    sb.append(getKilledTaskCount());
-    return sb.toString();
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-dag-api/src/main/java/org/apache/tez/dag/api/client/VertexStatus.java
----------------------------------------------------------------------
diff --git a/tez-dag-api/src/main/java/org/apache/tez/dag/api/client/VertexStatus.java b/tez-dag-api/src/main/java/org/apache/tez/dag/api/client/VertexStatus.java
deleted file mode 100644
index ce5dbe0..0000000
--- a/tez-dag-api/src/main/java/org/apache/tez/dag/api/client/VertexStatus.java
+++ /dev/null
@@ -1,78 +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.api.client;
-
-import java.util.List;
-
-import org.apache.tez.dag.api.records.DAGProtos.VertexStatusProtoOrBuilder;
-import org.apache.tez.dag.api.TezUncheckedException;
-
-public class VertexStatus {
-  
-  public enum State {
-    INITED,
-    RUNNING,
-    SUCCEEDED,
-    KILLED,
-    FAILED,
-    ERROR,
-    TERMINATING,
-  };
-  
-  VertexStatusProtoOrBuilder proxy = null;
-  Progress progress = null;
-  
-  public VertexStatus(VertexStatusProtoOrBuilder proxy) {
-    this.proxy = proxy;
-  }
-
-  public State getState() {
-    switch(proxy.getState()) {
-    case VERTEX_INITED:
-      return VertexStatus.State.INITED;
-    case VERTEX_RUNNING:
-      return VertexStatus.State.RUNNING;
-    case VERTEX_SUCCEEDED:
-      return VertexStatus.State.SUCCEEDED;
-    case VERTEX_FAILED:
-      return VertexStatus.State.FAILED;
-    case VERTEX_KILLED:
-      return VertexStatus.State.KILLED;
-    case VERTEX_ERROR:
-      return VertexStatus.State.ERROR;
-    case VERTEX_TERMINATING:
-      return VertexStatus.State.TERMINATING;
-    default:
-      throw new TezUncheckedException("Unsupported value for VertexStatus.State : " + 
-                              proxy.getState());
-    }    
-  }
-
-  public List<String> getDiagnostics() {
-    return proxy.getDiagnosticsList();
-  }
-
-  public Progress getProgress() {
-    if(progress == null && proxy.hasProgress()) {
-      progress = new Progress(proxy.getProgress());
-    }
-    return progress;    
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-dag-api/src/main/java/org/apache/tez/dag/api/client/rpc/DAGClientAMProtocolBlockingPB.java
----------------------------------------------------------------------
diff --git a/tez-dag-api/src/main/java/org/apache/tez/dag/api/client/rpc/DAGClientAMProtocolBlockingPB.java b/tez-dag-api/src/main/java/org/apache/tez/dag/api/client/rpc/DAGClientAMProtocolBlockingPB.java
deleted file mode 100644
index a1ee18f..0000000
--- a/tez-dag-api/src/main/java/org/apache/tez/dag/api/client/rpc/DAGClientAMProtocolBlockingPB.java
+++ /dev/null
@@ -1,30 +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.api.client.rpc;
-
-import org.apache.hadoop.ipc.ProtocolInfo;
-import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolRPC.DAGClientAMProtocol;
-
-@ProtocolInfo(
-    protocolName = "org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolPB",
-    protocolVersion = 1)
-public interface DAGClientAMProtocolBlockingPB 
-                              extends DAGClientAMProtocol.BlockingInterface {
-
-}


[27/50] [abbrv] git commit: TEZ-490. Rename SimpleInput / SimpleOutput to be MR specific (part of TEZ-398). (sseth)

Posted by ss...@apache.org.
TEZ-490. Rename SimpleInput / SimpleOutput to be MR specific (part of
TEZ-398). (sseth)


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

Branch: refs/heads/master
Commit: 5d86b9350555819b26110c200c8f3cdda6893020
Parents: c5a8a3c
Author: Siddharth Seth <ss...@apache.org>
Authored: Mon Sep 23 23:37:53 2013 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Mon Sep 23 23:37:53 2013 -0700

----------------------------------------------------------------------
 .../apache/hadoop/mapred/YarnTezDagChild.java   |  20 +-
 .../broadcast/input/BroadcastKVReader.java      |   2 +-
 .../apache/hadoop/mapred/LocalJobRunnerTez.java |   8 +-
 .../org/apache/tez/mapreduce/input/MRInput.java | 438 +++++++++++++++++++
 .../tez/mapreduce/input/MRInputLegacy.java      |  36 ++
 .../apache/tez/mapreduce/input/SimpleInput.java | 438 -------------------
 .../tez/mapreduce/input/SimpleInputLegacy.java  |  36 --
 .../apache/tez/mapreduce/output/MROutput.java   | 326 ++++++++++++++
 .../tez/mapreduce/output/SimpleOutput.java      | 326 --------------
 .../apache/tez/mapreduce/processor/MRTask.java  |  10 +-
 .../tez/mapreduce/processor/MRTaskReporter.java |   2 +-
 .../mapreduce/processor/map/MapProcessor.java   |  50 +--
 .../processor/reduce/ReduceProcessor.java       |   6 +-
 .../processor/map/TestMapProcessor.java         |   6 +-
 .../processor/reduce/TestReduceProcessor.java   |   8 +-
 15 files changed, 856 insertions(+), 856 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/5d86b935/tez-dag/src/main/java/org/apache/hadoop/mapred/YarnTezDagChild.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/hadoop/mapred/YarnTezDagChild.java b/tez-dag/src/main/java/org/apache/hadoop/mapred/YarnTezDagChild.java
index f32fa6b..1967462 100644
--- a/tez-dag/src/main/java/org/apache/hadoop/mapred/YarnTezDagChild.java
+++ b/tez-dag/src/main/java/org/apache/hadoop/mapred/YarnTezDagChild.java
@@ -85,8 +85,8 @@ import org.apache.tez.engine.common.objectregistry.ObjectRegistryModule;
 import org.apache.tez.engine.common.security.JobTokenIdentifier;
 import org.apache.tez.engine.common.security.TokenCache;
 import org.apache.tez.engine.newruntime.LogicalIOProcessorRuntimeTask;
-import org.apache.tez.mapreduce.input.SimpleInputLegacy;
-import org.apache.tez.mapreduce.output.SimpleOutput;
+import org.apache.tez.mapreduce.input.MRInputLegacy;
+import org.apache.tez.mapreduce.output.MROutput;
 
 import com.google.inject.Guice;
 import com.google.inject.Injector;
@@ -497,20 +497,20 @@ public class YarnTezDagChild {
 
     // FIXME need Input/Output vertices else we have this hack
     if (taskSpec.getInputs().isEmpty()) {
-      InputDescriptor simpleInputDesc =
-          new InputDescriptor(SimpleInputLegacy.class.getName());
-      simpleInputDesc.setUserPayload(
+      InputDescriptor mrInputDesc =
+          new InputDescriptor(MRInputLegacy.class.getName());
+      mrInputDesc.setUserPayload(
           taskSpec.getProcessorDescriptor().getUserPayload());
       taskSpec.getInputs().add(
-          new InputSpec("null", simpleInputDesc, 0));
+          new InputSpec("null", mrInputDesc, 0));
     }
     if (taskSpec.getOutputs().isEmpty()) {
-      OutputDescriptor simpleOutputDesc =
-          new OutputDescriptor(SimpleOutput.class.getName());
-      simpleOutputDesc.setUserPayload(
+      OutputDescriptor mrOutputDesc =
+          new OutputDescriptor(MROutput.class.getName());
+      mrOutputDesc.setUserPayload(
           taskSpec.getProcessorDescriptor().getUserPayload());
       taskSpec.getOutputs().add(
-          new OutputSpec("null", simpleOutputDesc, 0));
+          new OutputSpec("null", mrOutputDesc, 0));
     }
     String [] localDirs = StringUtils.getTrimmedStrings(System.getenv(Environment.LOCAL_DIRS.name()));
     conf.setStrings(TezJobConfig.LOCAL_DIRS, localDirs);

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/5d86b935/tez-engine/src/main/java/org/apache/tez/engine/broadcast/input/BroadcastKVReader.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/broadcast/input/BroadcastKVReader.java b/tez-engine/src/main/java/org/apache/tez/engine/broadcast/input/BroadcastKVReader.java
index 0b86a8e..2c53e75 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/broadcast/input/BroadcastKVReader.java
+++ b/tez-engine/src/main/java/org/apache/tez/engine/broadcast/input/BroadcastKVReader.java
@@ -187,7 +187,7 @@ public class BroadcastKVReader<K, V> implements KVReader {
 
   
   
-  // TODO NEWTEZ Move this into a common class. Also used in SImpleInput
+  // TODO NEWTEZ Move this into a common class. Also used in MRInput
   private class SimpleValueIterator implements Iterator<V> {
 
     private V value;

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/5d86b935/tez-mapreduce/src/main/java/org/apache/hadoop/mapred/LocalJobRunnerTez.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/hadoop/mapred/LocalJobRunnerTez.java b/tez-mapreduce/src/main/java/org/apache/hadoop/mapred/LocalJobRunnerTez.java
index f59e836..f2b0a38 100644
--- a/tez-mapreduce/src/main/java/org/apache/hadoop/mapred/LocalJobRunnerTez.java
+++ b/tez-mapreduce/src/main/java/org/apache/hadoop/mapred/LocalJobRunnerTez.java
@@ -83,8 +83,8 @@
 //import org.apache.tez.engine.records.TezTaskDependencyCompletionEventsUpdate;
 //import org.apache.tez.mapreduce.hadoop.IDConverter;
 //import org.apache.tez.mapreduce.hadoop.mapred.MRCounters;
-//import org.apache.tez.mapreduce.input.SimpleInput;
-//import org.apache.tez.mapreduce.output.SimpleOutput;
+//import org.apache.tez.mapreduce.input.MRInput;
+//import org.apache.tez.mapreduce.output.MROutput;
 //import org.apache.tez.mapreduce.processor.map.MapProcessor;
 //import org.apache.tez.mapreduce.processor.reduce.ReduceProcessor;
 //
@@ -252,7 +252,7 @@
 //                  tezMapId, user, localConf.getJobName(), "TODO_vertexName",
 //                  mapProcessorDesc,
 //                  Collections.singletonList(new InputSpec("srcVertex", 0,
-//                      SimpleInput.class.getName())),
+//                      MRInput.class.getName())),
 //                  Collections.singletonList(new OutputSpec("tgtVertex", 0,
 //                      LocalOnFileSorterOutput.class.getName())));
 //
@@ -458,7 +458,7 @@
 //                Collections.singletonList(new InputSpec("TODO_srcVertexName",
 //                    mapIds.size(), LocalMergedInput.class.getName())),
 //                Collections.singletonList(new OutputSpec("TODO_targetVertex",
-//                    0, SimpleOutput.class.getName())));
+//                    0, MROutput.class.getName())));
 //
 //            // move map output to reduce input
 //            for (int i = 0; i < mapIds.size(); i++) {

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/5d86b935/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/MRInput.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/MRInput.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/MRInput.java
new file mode 100644
index 0000000..6066d93
--- /dev/null
+++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/MRInput.java
@@ -0,0 +1,438 @@
+/**
+ * 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.input;
+
+import java.io.IOException;
+import java.util.Iterator;
+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.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FileSystem.Statistics;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.serializer.Deserializer;
+import org.apache.hadoop.io.serializer.SerializationFactory;
+import org.apache.hadoop.mapred.FileSplit;
+import org.apache.hadoop.mapred.InputFormat;
+import org.apache.hadoop.mapred.InputSplit;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.JobContext;
+import org.apache.hadoop.mapred.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.lib.input.FileInputFormatCounter;
+import org.apache.hadoop.mapreduce.split.SplitMetaInfoReaderTez;
+import org.apache.hadoop.mapreduce.split.JobSplit.TaskSplitIndex;
+import org.apache.hadoop.mapreduce.split.JobSplit.TaskSplitMetaInfo;
+import org.apache.hadoop.util.ReflectionUtils;
+import org.apache.tez.common.TezUtils;
+import org.apache.tez.common.counters.TaskCounter;
+import org.apache.tez.common.counters.TezCounter;
+import org.apache.tez.engine.api.Event;
+import org.apache.tez.engine.api.KVReader;
+import org.apache.tez.engine.api.LogicalInput;
+import org.apache.tez.engine.api.TezInputContext;
+import org.apache.tez.mapreduce.common.Utils;
+import org.apache.tez.mapreduce.hadoop.mapred.MRReporter;
+import org.apache.tez.mapreduce.hadoop.mapreduce.TaskAttemptContextImpl;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * {@link MRInput} is an {@link Input} which provides key/values pairs
+ * for the consumer.
+ *
+ * It is compatible with all standard Apache Hadoop MapReduce 
+ * {@link InputFormat} implementations.
+ */
+
+public class MRInput implements LogicalInput {
+
+  private static final Log LOG = LogFactory.getLog(MRInput.class);
+  
+  
+  private TezInputContext inputContext;
+  
+  private JobConf jobConf;
+  private Configuration incrementalConf;
+  private boolean recordReaderCreated = false;
+  
+  boolean useNewApi;
+  
+  org.apache.hadoop.mapreduce.TaskAttemptContext taskAttemptContext;
+
+  @SuppressWarnings("rawtypes")
+  private org.apache.hadoop.mapreduce.InputFormat newInputFormat;
+  @SuppressWarnings("rawtypes")
+  private org.apache.hadoop.mapreduce.RecordReader newRecordReader;
+  protected org.apache.hadoop.mapreduce.InputSplit newInputSplit;
+  
+  @SuppressWarnings("rawtypes")
+  private InputFormat oldInputFormat;
+  @SuppressWarnings("rawtypes")
+  protected RecordReader oldRecordReader;
+
+  protected TaskSplitIndex splitMetaInfo = new TaskSplitIndex();
+  
+  private TezCounter inputRecordCounter;
+  private TezCounter fileInputByteCounter; 
+  private List<Statistics> fsStats;
+
+  @Override
+  public List<Event> initialize(TezInputContext inputContext) throws IOException {
+    this.inputContext = inputContext;
+    Configuration conf = TezUtils.createConfFromUserPayload(inputContext.getUserPayload());
+    this.jobConf = new JobConf(conf);
+
+    // Read split information.
+    TaskSplitMetaInfo[] allMetaInfo = readSplits(conf);
+    TaskSplitMetaInfo thisTaskMetaInfo = allMetaInfo[inputContext.getTaskIndex()];
+    this.splitMetaInfo = new TaskSplitIndex(thisTaskMetaInfo.getSplitLocation(),
+        thisTaskMetaInfo.getStartOffset());
+    
+    // TODO NEWTEZ Rename this to be specific to MRInput. This Input, in
+    // theory, can be used by the MapProcessor, ReduceProcessor or a custom
+    // processor. (The processor could provide the counter though)
+    this.inputRecordCounter = inputContext.getCounters().findCounter(TaskCounter.MAP_INPUT_RECORDS);
+    this.fileInputByteCounter = inputContext.getCounters().findCounter(FileInputFormatCounter.BYTES_READ);
+    
+    useNewApi = this.jobConf.getUseNewMapper();
+
+    if (useNewApi) {
+      TaskAttemptContext taskAttemptContext = createTaskAttemptContext();
+      Class<? extends org.apache.hadoop.mapreduce.InputFormat<?, ?>> inputFormatClazz;
+      try {
+        inputFormatClazz = taskAttemptContext.getInputFormatClass();
+      } catch (ClassNotFoundException e) {
+        throw new IOException("Unable to instantiate InputFormat class", e);
+      }
+
+      newInputFormat = ReflectionUtils.newInstance(inputFormatClazz, this.jobConf);
+
+      newInputSplit = getNewSplitDetails(splitMetaInfo);
+
+      List<Statistics> matchedStats = null;
+      if (newInputSplit instanceof org.apache.hadoop.mapreduce.lib.input.FileSplit) {
+        matchedStats = Utils.getFsStatistics(
+            ((org.apache.hadoop.mapreduce.lib.input.FileSplit)
+                newInputSplit).getPath(), this.jobConf);
+      }
+      fsStats = matchedStats;
+      
+      try {
+        newRecordReader = newInputFormat.createRecordReader(newInputSplit, taskAttemptContext);
+        newRecordReader.initialize(newInputSplit, taskAttemptContext);
+      } catch (InterruptedException e) {
+        throw new IOException("Interrupted while creating record reader", e);
+      }
+    } else { // OLD API
+      oldInputFormat = this.jobConf.getInputFormat();
+      InputSplit oldInputSplit =
+          getOldSplitDetails(splitMetaInfo);
+      
+      
+      List<Statistics> matchedStats = null;
+      if (oldInputSplit instanceof FileSplit) {
+        matchedStats = Utils.getFsStatistics(((FileSplit) oldInputSplit).getPath(), this.jobConf);
+      }
+      fsStats = matchedStats;
+      
+      long bytesInPrev = getInputBytes();
+      oldRecordReader = oldInputFormat.getRecordReader(oldInputSplit,
+          this.jobConf, new MRReporter(inputContext, oldInputSplit));
+      long bytesInCurr = getInputBytes();
+      fileInputByteCounter.increment(bytesInCurr - bytesInPrev);
+      setIncrementalConfigParams(oldInputSplit);
+    }    
+    return null;
+  }
+
+  @Override
+  public KVReader getReader() throws IOException {
+    Preconditions
+        .checkState(recordReaderCreated == false,
+            "Only a single instance of record reader can be created for this input.");
+    recordReaderCreated = true;
+    return new MRInputKVReader();
+  }
+
+
+  @Override
+  public void handleEvents(List<Event> inputEvents) {
+    // Not expecting any events at the moment.
+  }
+
+
+  @Override
+  public void setNumPhysicalInputs(int numInputs) {
+    // Not required at the moment. May be required if splits are sent via events.
+  }
+
+  @Override
+  public List<Event> close() throws IOException {
+    long bytesInPrev = getInputBytes();
+    if (useNewApi) {
+      newRecordReader.close();
+    } else {
+      oldRecordReader.close();
+    }
+    long bytesInCurr = getInputBytes();
+    fileInputByteCounter.increment(bytesInCurr - bytesInPrev);
+    
+    return null;
+  }
+
+  /**
+   * {@link MRInput} sets some additional parameters like split location when using
+   * the new API. This methods returns the list of additional updates, and
+   * should be used by Processors using the old MapReduce API with {@link MRInput}.
+   * 
+   * @return the additional fields set by {@link MRInput}
+   */
+  public Configuration getConfigUpdates() {
+    return new Configuration(incrementalConf);
+  }
+
+  public float getProgress() throws IOException, InterruptedException {
+    if (useNewApi) {
+      return newRecordReader.getProgress();
+    } else {
+      return oldRecordReader.getProgress();
+    }
+  }
+
+  
+  private TaskAttemptContext createTaskAttemptContext() {
+    return new TaskAttemptContextImpl(this.jobConf, inputContext, true);
+  }
+  
+
+  private static class SimpleValueIterator implements Iterator<Object> {
+
+    private Object value;
+
+    public void setValue(Object value) {
+      this.value = value;
+    }
+
+    public boolean hasNext() {
+      return value != null;
+    }
+
+    public Object next() {
+      Object value = this.value;
+      this.value = null;
+      return value;
+    }
+
+    public void remove() {
+      throw new UnsupportedOperationException();
+    }
+  }
+
+  private static class SimpleIterable implements Iterable<Object> {
+    private final Iterator<Object> iterator;
+    public SimpleIterable(Iterator<Object> iterator) {
+      this.iterator = iterator;
+    }
+
+    @Override
+    public Iterator<Object> iterator() {
+      return iterator;
+    }
+  }
+
+
+
+  
+  @SuppressWarnings("unchecked")
+  private InputSplit getOldSplitDetails(TaskSplitIndex splitMetaInfo)
+      throws IOException {
+    Path file = new Path(splitMetaInfo.getSplitLocation());
+    FileSystem fs = FileSystem.getLocal(jobConf);
+    file = fs.makeQualified(file);
+    LOG.info("Reading input split file from : " + file);
+    long offset = splitMetaInfo.getStartOffset();
+    
+    FSDataInputStream inFile = fs.open(file);
+    inFile.seek(offset);
+    String className = Text.readString(inFile);
+    Class<org.apache.hadoop.mapred.InputSplit> cls;
+    try {
+      cls = 
+          (Class<org.apache.hadoop.mapred.InputSplit>) 
+          jobConf.getClassByName(className);
+    } catch (ClassNotFoundException ce) {
+      IOException wrap = new IOException("Split class " + className + 
+          " not found");
+      wrap.initCause(ce);
+      throw wrap;
+    }
+    SerializationFactory factory = new SerializationFactory(jobConf);
+    Deserializer<org.apache.hadoop.mapred.InputSplit> deserializer = 
+        (Deserializer<org.apache.hadoop.mapred.InputSplit>) 
+        factory.getDeserializer(cls);
+    deserializer.open(inFile);
+    org.apache.hadoop.mapred.InputSplit split = deserializer.deserialize(null);
+    long pos = inFile.getPos();
+    inputContext.getCounters().findCounter(TaskCounter.SPLIT_RAW_BYTES)
+        .increment(pos - offset);
+    inFile.close();
+    return split;
+  }
+
+  @SuppressWarnings("unchecked")
+  private org.apache.hadoop.mapreduce.InputSplit getNewSplitDetails(
+      TaskSplitIndex splitMetaInfo) throws IOException {
+    Path file = new Path(splitMetaInfo.getSplitLocation());
+    long offset = splitMetaInfo.getStartOffset();
+    
+    // Split information read from local filesystem.
+    FileSystem fs = FileSystem.getLocal(jobConf);
+    file = fs.makeQualified(file);
+    LOG.info("Reading input split file from : " + file);
+    FSDataInputStream inFile = fs.open(file);
+    inFile.seek(offset);
+    String className = Text.readString(inFile);
+    Class<org.apache.hadoop.mapreduce.InputSplit> cls;
+    try {
+      cls = 
+          (Class<org.apache.hadoop.mapreduce.InputSplit>) 
+          jobConf.getClassByName(className);
+    } catch (ClassNotFoundException ce) {
+      IOException wrap = new IOException("Split class " + className + 
+          " not found");
+      wrap.initCause(ce);
+      throw wrap;
+    }
+    SerializationFactory factory = new SerializationFactory(jobConf);
+    Deserializer<org.apache.hadoop.mapreduce.InputSplit> deserializer = 
+        (Deserializer<org.apache.hadoop.mapreduce.InputSplit>) 
+        factory.getDeserializer(cls);
+    deserializer.open(inFile);
+    org.apache.hadoop.mapreduce.InputSplit split = 
+        deserializer.deserialize(null);
+    long pos = inFile.getPos();
+    inputContext.getCounters().findCounter(TaskCounter.SPLIT_RAW_BYTES)
+        .increment(pos - offset);
+    inFile.close();
+    return split;
+  }
+
+  private void setIncrementalConfigParams(InputSplit inputSplit) {
+    if (inputSplit instanceof FileSplit) {
+      FileSplit fileSplit = (FileSplit) inputSplit;
+      this.incrementalConf = new Configuration(false);
+
+      this.incrementalConf.set(JobContext.MAP_INPUT_FILE, fileSplit.getPath()
+          .toString());
+      this.incrementalConf.setLong(JobContext.MAP_INPUT_START,
+          fileSplit.getStart());
+      this.incrementalConf.setLong(JobContext.MAP_INPUT_PATH,
+          fileSplit.getLength());
+    }
+    LOG.info("Processing split: " + inputSplit);
+  }
+
+  private long getInputBytes() {
+    if (fsStats == null) return 0;
+    long bytesRead = 0;
+    for (Statistics stat: fsStats) {
+      bytesRead = bytesRead + stat.getBytesRead();
+    }
+    return bytesRead;
+  }
+
+  protected TaskSplitMetaInfo[] readSplits(Configuration conf)
+      throws IOException {
+    TaskSplitMetaInfo[] allTaskSplitMetaInfo;
+    allTaskSplitMetaInfo = SplitMetaInfoReaderTez.readSplitMetaInfo(conf,
+        FileSystem.getLocal(conf));
+    return allTaskSplitMetaInfo;
+  }
+  
+  private class MRInputKVReader implements KVReader {
+    
+    Object key;
+    Object value;
+
+    private SimpleValueIterator valueIterator = new SimpleValueIterator();
+    private SimpleIterable valueIterable = new SimpleIterable(valueIterator);
+
+    private final boolean localNewApi;
+    
+    MRInputKVReader() {
+      localNewApi = useNewApi;
+      if (!localNewApi) {
+        key = oldRecordReader.createKey();
+        value =oldRecordReader.createValue();
+      }
+    }
+    
+    // Setup the values iterator once, and set value on the same object each time
+    // to prevent lots of objects being created.
+
+    
+    @SuppressWarnings("unchecked")
+    @Override
+    public boolean next() throws IOException {
+      boolean hasNext = false;
+      long bytesInPrev = getInputBytes();
+      if (localNewApi) {
+        try {
+          hasNext = newRecordReader.nextKeyValue();
+        } catch (InterruptedException e) {
+          Thread.currentThread().interrupt();
+          throw new IOException("Interrupted while checking for next key-value", e);
+        }
+      } else {
+        hasNext = oldRecordReader.next(key, value);
+      }
+      long bytesInCurr = getInputBytes();
+      fileInputByteCounter.increment(bytesInCurr - bytesInPrev);
+      
+      if (hasNext) {
+        inputRecordCounter.increment(1);
+      }
+      
+      return hasNext;
+    }
+
+    @Override
+    public KVRecord getCurrentKV() throws IOException {
+      KVRecord kvRecord = null;
+      if (localNewApi) {
+        try {
+          valueIterator.setValue(newRecordReader.getCurrentValue());
+          kvRecord = new KVRecord(newRecordReader.getCurrentKey(), valueIterable);
+        } catch (InterruptedException e) {
+          Thread.currentThread().interrupt();
+          throw new IOException("Interrupted while fetching next key-value", e);
+        }
+        
+      } else {
+        valueIterator.setValue(value);
+        kvRecord = new KVRecord(key, valueIterable);
+      }
+      return kvRecord;
+    }
+  };
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/5d86b935/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/MRInputLegacy.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/MRInputLegacy.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/MRInputLegacy.java
new file mode 100644
index 0000000..5923746
--- /dev/null
+++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/MRInputLegacy.java
@@ -0,0 +1,36 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.mapreduce.input;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.mapred.RecordReader;
+
+public class MRInputLegacy extends MRInput {
+
+  @Private
+  public org.apache.hadoop.mapreduce.InputSplit getNewInputSplit() {
+    return this.newInputSplit;
+  }  
+  
+  @SuppressWarnings("rawtypes")
+  @Private
+  public RecordReader getOldRecordReader() {
+    return this.oldRecordReader;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/5d86b935/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/SimpleInput.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/SimpleInput.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/SimpleInput.java
deleted file mode 100644
index 598f801..0000000
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/SimpleInput.java
+++ /dev/null
@@ -1,438 +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.input;
-
-import java.io.IOException;
-import java.util.Iterator;
-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.fs.FSDataInputStream;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.FileSystem.Statistics;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.serializer.Deserializer;
-import org.apache.hadoop.io.serializer.SerializationFactory;
-import org.apache.hadoop.mapred.FileSplit;
-import org.apache.hadoop.mapred.InputFormat;
-import org.apache.hadoop.mapred.InputSplit;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.JobContext;
-import org.apache.hadoop.mapred.RecordReader;
-import org.apache.hadoop.mapreduce.TaskAttemptContext;
-import org.apache.hadoop.mapreduce.lib.input.FileInputFormatCounter;
-import org.apache.hadoop.mapreduce.split.SplitMetaInfoReaderTez;
-import org.apache.hadoop.mapreduce.split.JobSplit.TaskSplitIndex;
-import org.apache.hadoop.mapreduce.split.JobSplit.TaskSplitMetaInfo;
-import org.apache.hadoop.util.ReflectionUtils;
-import org.apache.tez.common.TezUtils;
-import org.apache.tez.common.counters.TaskCounter;
-import org.apache.tez.common.counters.TezCounter;
-import org.apache.tez.engine.api.Event;
-import org.apache.tez.engine.api.KVReader;
-import org.apache.tez.engine.api.LogicalInput;
-import org.apache.tez.engine.api.TezInputContext;
-import org.apache.tez.mapreduce.common.Utils;
-import org.apache.tez.mapreduce.hadoop.mapred.MRReporter;
-import org.apache.tez.mapreduce.hadoop.mapreduce.TaskAttemptContextImpl;
-
-import com.google.common.base.Preconditions;
-
-/**
- * {@link SimpleInput} is an {@link Input} which provides key/values pairs
- * for the consumer.
- *
- * It is compatible with all standard Apache Hadoop MapReduce 
- * {@link InputFormat} implementations.
- */
-
-public class SimpleInput implements LogicalInput {
-
-  private static final Log LOG = LogFactory.getLog(SimpleInput.class);
-  
-  
-  private TezInputContext inputContext;
-  
-  private JobConf jobConf;
-  private Configuration incrementalConf;
-  private boolean recordReaderCreated = false;
-  
-  boolean useNewApi;
-  
-  org.apache.hadoop.mapreduce.TaskAttemptContext taskAttemptContext;
-
-  @SuppressWarnings("rawtypes")
-  private org.apache.hadoop.mapreduce.InputFormat newInputFormat;
-  @SuppressWarnings("rawtypes")
-  private org.apache.hadoop.mapreduce.RecordReader newRecordReader;
-  protected org.apache.hadoop.mapreduce.InputSplit newInputSplit;
-  
-  @SuppressWarnings("rawtypes")
-  private InputFormat oldInputFormat;
-  @SuppressWarnings("rawtypes")
-  protected RecordReader oldRecordReader;
-
-  protected TaskSplitIndex splitMetaInfo = new TaskSplitIndex();
-  
-  private TezCounter inputRecordCounter;
-  private TezCounter fileInputByteCounter; 
-  private List<Statistics> fsStats;
-
-  @Override
-  public List<Event> initialize(TezInputContext inputContext) throws IOException {
-    this.inputContext = inputContext;
-    Configuration conf = TezUtils.createConfFromUserPayload(inputContext.getUserPayload());
-    this.jobConf = new JobConf(conf);
-
-    // Read split information.
-    TaskSplitMetaInfo[] allMetaInfo = readSplits(conf);
-    TaskSplitMetaInfo thisTaskMetaInfo = allMetaInfo[inputContext.getTaskIndex()];
-    this.splitMetaInfo = new TaskSplitIndex(thisTaskMetaInfo.getSplitLocation(),
-        thisTaskMetaInfo.getStartOffset());
-    
-    // TODO NEWTEZ Rename this to be specific to SimpleInput. This Input, in
-    // theory, can be used by the MapProcessor, ReduceProcessor or a custom
-    // processor. (The processor could provide the counter though)
-    this.inputRecordCounter = inputContext.getCounters().findCounter(TaskCounter.MAP_INPUT_RECORDS);
-    this.fileInputByteCounter = inputContext.getCounters().findCounter(FileInputFormatCounter.BYTES_READ);
-    
-    useNewApi = this.jobConf.getUseNewMapper();
-
-    if (useNewApi) {
-      TaskAttemptContext taskAttemptContext = createTaskAttemptContext();
-      Class<? extends org.apache.hadoop.mapreduce.InputFormat<?, ?>> inputFormatClazz;
-      try {
-        inputFormatClazz = taskAttemptContext.getInputFormatClass();
-      } catch (ClassNotFoundException e) {
-        throw new IOException("Unable to instantiate InputFormat class", e);
-      }
-
-      newInputFormat = ReflectionUtils.newInstance(inputFormatClazz, this.jobConf);
-
-      newInputSplit = getNewSplitDetails(splitMetaInfo);
-
-      List<Statistics> matchedStats = null;
-      if (newInputSplit instanceof org.apache.hadoop.mapreduce.lib.input.FileSplit) {
-        matchedStats = Utils.getFsStatistics(
-            ((org.apache.hadoop.mapreduce.lib.input.FileSplit)
-                newInputSplit).getPath(), this.jobConf);
-      }
-      fsStats = matchedStats;
-      
-      try {
-        newRecordReader = newInputFormat.createRecordReader(newInputSplit, taskAttemptContext);
-        newRecordReader.initialize(newInputSplit, taskAttemptContext);
-      } catch (InterruptedException e) {
-        throw new IOException("Interrupted while creating record reader", e);
-      }
-    } else { // OLD API
-      oldInputFormat = this.jobConf.getInputFormat();
-      InputSplit oldInputSplit =
-          getOldSplitDetails(splitMetaInfo);
-      
-      
-      List<Statistics> matchedStats = null;
-      if (oldInputSplit instanceof FileSplit) {
-        matchedStats = Utils.getFsStatistics(((FileSplit) oldInputSplit).getPath(), this.jobConf);
-      }
-      fsStats = matchedStats;
-      
-      long bytesInPrev = getInputBytes();
-      oldRecordReader = oldInputFormat.getRecordReader(oldInputSplit,
-          this.jobConf, new MRReporter(inputContext, oldInputSplit));
-      long bytesInCurr = getInputBytes();
-      fileInputByteCounter.increment(bytesInCurr - bytesInPrev);
-      setIncrementalConfigParams(oldInputSplit);
-    }    
-    return null;
-  }
-
-  @Override
-  public KVReader getReader() throws IOException {
-    Preconditions
-        .checkState(recordReaderCreated == false,
-            "Only a single instance of record reader can be created for this input.");
-    recordReaderCreated = true;
-    return new MRInputKVReader();
-  }
-
-
-  @Override
-  public void handleEvents(List<Event> inputEvents) {
-    // Not expecting any events at the moment.
-  }
-
-
-  @Override
-  public void setNumPhysicalInputs(int numInputs) {
-    // Not required at the moment. May be required if splits are sent via events.
-  }
-
-  @Override
-  public List<Event> close() throws IOException {
-    long bytesInPrev = getInputBytes();
-    if (useNewApi) {
-      newRecordReader.close();
-    } else {
-      oldRecordReader.close();
-    }
-    long bytesInCurr = getInputBytes();
-    fileInputByteCounter.increment(bytesInCurr - bytesInPrev);
-    
-    return null;
-  }
-
-  /**
-   * SimpleInputs sets some additional parameters like split location when using
-   * the new API. This methods returns the list of additional updates, and
-   * should be used by Processors using the old MapReduce API with SimpleInput.
-   * 
-   * @return the additional fields set by SimpleInput
-   */
-  public Configuration getConfigUpdates() {
-    return new Configuration(incrementalConf);
-  }
-
-  public float getProgress() throws IOException, InterruptedException {
-    if (useNewApi) {
-      return newRecordReader.getProgress();
-    } else {
-      return oldRecordReader.getProgress();
-    }
-  }
-
-  
-  private TaskAttemptContext createTaskAttemptContext() {
-    return new TaskAttemptContextImpl(this.jobConf, inputContext, true);
-  }
-  
-
-  private static class SimpleValueIterator implements Iterator<Object> {
-
-    private Object value;
-
-    public void setValue(Object value) {
-      this.value = value;
-    }
-
-    public boolean hasNext() {
-      return value != null;
-    }
-
-    public Object next() {
-      Object value = this.value;
-      this.value = null;
-      return value;
-    }
-
-    public void remove() {
-      throw new UnsupportedOperationException();
-    }
-  }
-
-  private static class SimpleIterable implements Iterable<Object> {
-    private final Iterator<Object> iterator;
-    public SimpleIterable(Iterator<Object> iterator) {
-      this.iterator = iterator;
-    }
-
-    @Override
-    public Iterator<Object> iterator() {
-      return iterator;
-    }
-  }
-
-
-
-  
-  @SuppressWarnings("unchecked")
-  private InputSplit getOldSplitDetails(TaskSplitIndex splitMetaInfo)
-      throws IOException {
-    Path file = new Path(splitMetaInfo.getSplitLocation());
-    FileSystem fs = FileSystem.getLocal(jobConf);
-    file = fs.makeQualified(file);
-    LOG.info("Reading input split file from : " + file);
-    long offset = splitMetaInfo.getStartOffset();
-    
-    FSDataInputStream inFile = fs.open(file);
-    inFile.seek(offset);
-    String className = Text.readString(inFile);
-    Class<org.apache.hadoop.mapred.InputSplit> cls;
-    try {
-      cls = 
-          (Class<org.apache.hadoop.mapred.InputSplit>) 
-          jobConf.getClassByName(className);
-    } catch (ClassNotFoundException ce) {
-      IOException wrap = new IOException("Split class " + className + 
-          " not found");
-      wrap.initCause(ce);
-      throw wrap;
-    }
-    SerializationFactory factory = new SerializationFactory(jobConf);
-    Deserializer<org.apache.hadoop.mapred.InputSplit> deserializer = 
-        (Deserializer<org.apache.hadoop.mapred.InputSplit>) 
-        factory.getDeserializer(cls);
-    deserializer.open(inFile);
-    org.apache.hadoop.mapred.InputSplit split = deserializer.deserialize(null);
-    long pos = inFile.getPos();
-    inputContext.getCounters().findCounter(TaskCounter.SPLIT_RAW_BYTES)
-        .increment(pos - offset);
-    inFile.close();
-    return split;
-  }
-
-  @SuppressWarnings("unchecked")
-  private org.apache.hadoop.mapreduce.InputSplit getNewSplitDetails(
-      TaskSplitIndex splitMetaInfo) throws IOException {
-    Path file = new Path(splitMetaInfo.getSplitLocation());
-    long offset = splitMetaInfo.getStartOffset();
-    
-    // Split information read from local filesystem.
-    FileSystem fs = FileSystem.getLocal(jobConf);
-    file = fs.makeQualified(file);
-    LOG.info("Reading input split file from : " + file);
-    FSDataInputStream inFile = fs.open(file);
-    inFile.seek(offset);
-    String className = Text.readString(inFile);
-    Class<org.apache.hadoop.mapreduce.InputSplit> cls;
-    try {
-      cls = 
-          (Class<org.apache.hadoop.mapreduce.InputSplit>) 
-          jobConf.getClassByName(className);
-    } catch (ClassNotFoundException ce) {
-      IOException wrap = new IOException("Split class " + className + 
-          " not found");
-      wrap.initCause(ce);
-      throw wrap;
-    }
-    SerializationFactory factory = new SerializationFactory(jobConf);
-    Deserializer<org.apache.hadoop.mapreduce.InputSplit> deserializer = 
-        (Deserializer<org.apache.hadoop.mapreduce.InputSplit>) 
-        factory.getDeserializer(cls);
-    deserializer.open(inFile);
-    org.apache.hadoop.mapreduce.InputSplit split = 
-        deserializer.deserialize(null);
-    long pos = inFile.getPos();
-    inputContext.getCounters().findCounter(TaskCounter.SPLIT_RAW_BYTES)
-        .increment(pos - offset);
-    inFile.close();
-    return split;
-  }
-
-  private void setIncrementalConfigParams(InputSplit inputSplit) {
-    if (inputSplit instanceof FileSplit) {
-      FileSplit fileSplit = (FileSplit) inputSplit;
-      this.incrementalConf = new Configuration(false);
-
-      this.incrementalConf.set(JobContext.MAP_INPUT_FILE, fileSplit.getPath()
-          .toString());
-      this.incrementalConf.setLong(JobContext.MAP_INPUT_START,
-          fileSplit.getStart());
-      this.incrementalConf.setLong(JobContext.MAP_INPUT_PATH,
-          fileSplit.getLength());
-    }
-    LOG.info("Processing split: " + inputSplit);
-  }
-
-  private long getInputBytes() {
-    if (fsStats == null) return 0;
-    long bytesRead = 0;
-    for (Statistics stat: fsStats) {
-      bytesRead = bytesRead + stat.getBytesRead();
-    }
-    return bytesRead;
-  }
-
-  protected TaskSplitMetaInfo[] readSplits(Configuration conf)
-      throws IOException {
-    TaskSplitMetaInfo[] allTaskSplitMetaInfo;
-    allTaskSplitMetaInfo = SplitMetaInfoReaderTez.readSplitMetaInfo(conf,
-        FileSystem.getLocal(conf));
-    return allTaskSplitMetaInfo;
-  }
-  
-  private class MRInputKVReader implements KVReader {
-    
-    Object key;
-    Object value;
-
-    private SimpleValueIterator valueIterator = new SimpleValueIterator();
-    private SimpleIterable valueIterable = new SimpleIterable(valueIterator);
-
-    private final boolean localNewApi;
-    
-    MRInputKVReader() {
-      localNewApi = useNewApi;
-      if (!localNewApi) {
-        key = oldRecordReader.createKey();
-        value =oldRecordReader.createValue();
-      }
-    }
-    
-    // Setup the values iterator once, and set value on the same object each time
-    // to prevent lots of objects being created.
-
-    
-    @SuppressWarnings("unchecked")
-    @Override
-    public boolean next() throws IOException {
-      boolean hasNext = false;
-      long bytesInPrev = getInputBytes();
-      if (localNewApi) {
-        try {
-          hasNext = newRecordReader.nextKeyValue();
-        } catch (InterruptedException e) {
-          Thread.currentThread().interrupt();
-          throw new IOException("Interrupted while checking for next key-value", e);
-        }
-      } else {
-        hasNext = oldRecordReader.next(key, value);
-      }
-      long bytesInCurr = getInputBytes();
-      fileInputByteCounter.increment(bytesInCurr - bytesInPrev);
-      
-      if (hasNext) {
-        inputRecordCounter.increment(1);
-      }
-      
-      return hasNext;
-    }
-
-    @Override
-    public KVRecord getCurrentKV() throws IOException {
-      KVRecord kvRecord = null;
-      if (localNewApi) {
-        try {
-          valueIterator.setValue(newRecordReader.getCurrentValue());
-          kvRecord = new KVRecord(newRecordReader.getCurrentKey(), valueIterable);
-        } catch (InterruptedException e) {
-          Thread.currentThread().interrupt();
-          throw new IOException("Interrupted while fetching next key-value", e);
-        }
-        
-      } else {
-        valueIterator.setValue(value);
-        kvRecord = new KVRecord(key, valueIterable);
-      }
-      return kvRecord;
-    }
-  };
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/5d86b935/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/SimpleInputLegacy.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/SimpleInputLegacy.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/SimpleInputLegacy.java
deleted file mode 100644
index 4e61aa7..0000000
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/SimpleInputLegacy.java
+++ /dev/null
@@ -1,36 +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.input;
-
-import org.apache.hadoop.classification.InterfaceAudience.Private;
-import org.apache.hadoop.mapred.RecordReader;
-
-public class SimpleInputLegacy extends SimpleInput {
-
-  @Private
-  public org.apache.hadoop.mapreduce.InputSplit getNewInputSplit() {
-    return this.newInputSplit;
-  }  
-  
-  @SuppressWarnings("rawtypes")
-  @Private
-  public RecordReader getOldRecordReader() {
-    return this.oldRecordReader;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/5d86b935/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
new file mode 100644
index 0000000..e6bdbe6
--- /dev/null
+++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/output/MROutput.java
@@ -0,0 +1,326 @@
+package org.apache.tez.mapreduce.output;
+
+import java.io.IOException;
+import java.text.NumberFormat;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+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.Path;
+import org.apache.hadoop.fs.FileSystem.Statistics;
+import org.apache.hadoop.mapred.FileOutputCommitter;
+import org.apache.hadoop.mapred.FileOutputFormat;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.JobContext;
+import org.apache.hadoop.mapred.TaskAttemptID;
+import org.apache.hadoop.mapred.TaskID;
+import org.apache.hadoop.mapreduce.OutputCommitter;
+import org.apache.hadoop.mapreduce.OutputFormat;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.TaskType;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputFormatCounter;
+import org.apache.hadoop.util.ReflectionUtils;
+import org.apache.tez.common.TezUtils;
+import org.apache.tez.common.counters.TaskCounter;
+import org.apache.tez.common.counters.TezCounter;
+import org.apache.tez.engine.api.Event;
+import org.apache.tez.engine.api.KVWriter;
+import org.apache.tez.engine.api.LogicalOutput;
+import org.apache.tez.engine.api.TezOutputContext;
+import org.apache.tez.mapreduce.common.Utils;
+import org.apache.tez.mapreduce.hadoop.MRConfig;
+import org.apache.tez.mapreduce.hadoop.MRJobConfig;
+import org.apache.tez.mapreduce.hadoop.mapred.MRReporter;
+import org.apache.tez.mapreduce.hadoop.mapreduce.TaskAttemptContextImpl;
+import org.apache.tez.mapreduce.processor.MRTaskReporter;
+
+public class MROutput implements LogicalOutput {
+
+  private static final Log LOG = LogFactory.getLog(MROutput.class);
+
+  private static final NumberFormat NUMBER_FORMAT = NumberFormat.getInstance();
+  static {
+    NUMBER_FORMAT.setMinimumIntegerDigits(5);
+    NUMBER_FORMAT.setGroupingUsed(false);
+  }
+
+  private TezOutputContext outputContext;
+  private JobConf jobConf;
+  boolean useNewApi;
+  private AtomicBoolean closed = new AtomicBoolean(false);
+
+  @SuppressWarnings("rawtypes")
+  org.apache.hadoop.mapreduce.OutputFormat newOutputFormat;
+  @SuppressWarnings("rawtypes")
+  org.apache.hadoop.mapreduce.RecordWriter newRecordWriter;
+
+  @SuppressWarnings("rawtypes")
+  org.apache.hadoop.mapred.OutputFormat oldOutputFormat;
+  @SuppressWarnings("rawtypes")
+  org.apache.hadoop.mapred.RecordWriter oldRecordWriter;
+
+  private TezCounter outputRecordCounter;
+  private TezCounter fileOutputByteCounter;
+  private List<Statistics> fsStats;
+
+  private TaskAttemptContext newApiTaskAttemptContext;
+  private org.apache.hadoop.mapred.TaskAttemptContext oldApiTaskAttemptContext;
+
+  private boolean isMapperOutput;
+
+  private OutputCommitter committer;
+
+  @Override
+  public List<Event> initialize(TezOutputContext outputContext)
+      throws IOException, InterruptedException {
+    LOG.info("Initializing Simple Output");
+    this.outputContext = outputContext;
+    Configuration conf = TezUtils.createConfFromUserPayload(
+        outputContext.getUserPayload());
+    this.jobConf = new JobConf(conf);
+    this.useNewApi = this.jobConf.getUseNewMapper();
+    this.isMapperOutput = jobConf.getBoolean(MRConfig.IS_MAP_PROCESSOR,
+        false);
+    jobConf.setInt(MRJobConfig.APPLICATION_ATTEMPT_ID,
+        outputContext.getDAGAttemptNumber());
+
+    outputRecordCounter = outputContext.getCounters().findCounter(
+        TaskCounter.MAP_OUTPUT_RECORDS);
+    fileOutputByteCounter = outputContext.getCounters().findCounter(
+        FileOutputFormatCounter.BYTES_WRITTEN);
+
+    if (useNewApi) {
+      newApiTaskAttemptContext = createTaskAttemptContext();
+      try {
+        newOutputFormat =
+            ReflectionUtils.newInstance(
+                newApiTaskAttemptContext.getOutputFormatClass(), jobConf);
+      } catch (ClassNotFoundException cnfe) {
+        throw new IOException(cnfe);
+      }
+
+      List<Statistics> matchedStats = null;
+      if (newOutputFormat instanceof
+          org.apache.hadoop.mapreduce.lib.output.FileOutputFormat) {
+        matchedStats =
+            Utils.getFsStatistics(
+                org.apache.hadoop.mapreduce.lib.output.FileOutputFormat
+                    .getOutputPath(newApiTaskAttemptContext),
+                jobConf);
+      }
+      fsStats = matchedStats;
+
+      long bytesOutPrev = getOutputBytes();
+      try {
+        newRecordWriter =
+            newOutputFormat.getRecordWriter(newApiTaskAttemptContext);
+      } catch (InterruptedException e) {
+        throw new IOException("Interrupted while creating record writer", e);
+      }
+      long bytesOutCurr = getOutputBytes();
+      fileOutputByteCounter.increment(bytesOutCurr - bytesOutPrev);
+    } else {
+      TaskAttemptID taskAttemptId = new TaskAttemptID(new TaskID(Long.toString(
+          outputContext.getApplicationId().getClusterTimestamp()),
+          outputContext.getApplicationId().getId(),
+          (isMapperOutput ? TaskType.MAP : TaskType.REDUCE),
+          outputContext.getTaskIndex()),
+          outputContext.getTaskAttemptNumber());
+      jobConf.set(JobContext.TASK_ATTEMPT_ID, taskAttemptId.toString());
+      jobConf.set(JobContext.TASK_ID, taskAttemptId.getTaskID().toString());
+      jobConf.setBoolean(JobContext.TASK_ISMAP, isMapperOutput);
+      jobConf.setInt(JobContext.TASK_PARTITION,
+          taskAttemptId.getTaskID().getId());
+      jobConf.set(JobContext.ID, taskAttemptId.getJobID().toString());
+
+      oldApiTaskAttemptContext =
+          new org.apache.tez.mapreduce.hadoop.mapred.TaskAttemptContextImpl(
+              jobConf, taskAttemptId,
+              new MRTaskReporter(outputContext));
+      oldOutputFormat = jobConf.getOutputFormat();
+
+      List<Statistics> matchedStats = null;
+      if (oldOutputFormat
+          instanceof org.apache.hadoop.mapred.FileOutputFormat) {
+        matchedStats =
+            Utils.getFsStatistics(
+                org.apache.hadoop.mapred.FileOutputFormat.getOutputPath(
+                    jobConf),
+                jobConf);
+      }
+      fsStats = matchedStats;
+
+      FileSystem fs = FileSystem.get(jobConf);
+      String finalName = getOutputName();
+
+      long bytesOutPrev = getOutputBytes();
+      oldRecordWriter =
+          oldOutputFormat.getRecordWriter(
+              fs, jobConf, finalName, new MRReporter(outputContext));
+      long bytesOutCurr = getOutputBytes();
+      fileOutputByteCounter.increment(bytesOutCurr - bytesOutPrev);
+    }
+    initCommitter(jobConf, useNewApi);
+
+    LOG.info("Initialized Simple Output"
+        + ", using_new_api: " + useNewApi);
+    return null;
+  }
+
+  public void initCommitter(JobConf job, boolean useNewApi)
+      throws IOException, InterruptedException {
+
+    if (useNewApi) {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("using new api for output committer");
+      }
+
+      OutputFormat<?, ?> outputFormat = null;
+      try {
+        outputFormat = ReflectionUtils.newInstance(
+            newApiTaskAttemptContext.getOutputFormatClass(), job);
+      } catch (ClassNotFoundException cnfe) {
+        throw new IOException("Unknown OutputFormat", cnfe);
+      }
+      this.committer = outputFormat.getOutputCommitter(
+          newApiTaskAttemptContext);
+    } else {
+      this.committer = job.getOutputCommitter();
+    }
+
+    Path outputPath = FileOutputFormat.getOutputPath(job);
+    if (outputPath != null) {
+      if ((this.committer instanceof FileOutputCommitter)) {
+        FileOutputFormat.setWorkOutputPath(job,
+            ((FileOutputCommitter) this.committer).getTaskAttemptPath(
+                oldApiTaskAttemptContext));
+      } else {
+        FileOutputFormat.setWorkOutputPath(job, outputPath);
+      }
+    }
+    if (useNewApi) {
+      this.committer.setupTask(newApiTaskAttemptContext);
+    } else {
+      this.committer.setupTask(oldApiTaskAttemptContext);
+    }
+  }
+
+  public boolean isCommitRequired() throws IOException {
+    if (useNewApi) {
+      return committer.needsTaskCommit(newApiTaskAttemptContext);
+    } else {
+      return committer.needsTaskCommit(oldApiTaskAttemptContext);
+    }
+  }
+
+  private TaskAttemptContext createTaskAttemptContext() {
+    return new TaskAttemptContextImpl(this.jobConf, outputContext,
+        isMapperOutput);
+  }
+
+  private long getOutputBytes() {
+    if (fsStats == null) return 0;
+    long bytesWritten = 0;
+    for (Statistics stat: fsStats) {
+      bytesWritten = bytesWritten + stat.getBytesWritten();
+    }
+    return bytesWritten;
+  }
+
+  private String getOutputName() {
+    return "part-" + NUMBER_FORMAT.format(outputContext.getTaskIndex());
+  }
+
+  @Override
+  public KVWriter getWriter() throws IOException {
+    return new KVWriter() {
+      private final boolean useNewWriter = useNewApi;
+
+      @SuppressWarnings("unchecked")
+      @Override
+      public void write(Object key, Object value) throws IOException {
+        long bytesOutPrev = getOutputBytes();
+        if (useNewWriter) {
+          try {
+            newRecordWriter.write(key, value);
+          } catch (InterruptedException e) {
+            Thread.currentThread().interrupt();
+            throw new IOException("Interrupted while writing next key-value",e);
+          }
+        } else {
+          oldRecordWriter.write(key, value);
+        }
+
+        long bytesOutCurr = getOutputBytes();
+        fileOutputByteCounter.increment(bytesOutCurr - bytesOutPrev);
+        outputRecordCounter.increment(1);
+      }
+    };
+  }
+
+  @Override
+  public void handleEvents(List<Event> outputEvents) {
+    // Not expecting any events at the moment.
+  }
+
+  @Override
+  public synchronized List<Event> close() throws IOException {
+    if (closed.getAndSet(true)) {
+      return null;
+    }
+
+    LOG.info("Closing Simple Output");
+    long bytesOutPrev = getOutputBytes();
+    if (useNewApi) {
+      try {
+        newRecordWriter.close(newApiTaskAttemptContext);
+      } catch (InterruptedException e) {
+        throw new IOException("Interrupted while closing record writer", e);
+      }
+    } else {
+      oldRecordWriter.close(null);
+    }
+    long bytesOutCurr = getOutputBytes();
+    fileOutputByteCounter.increment(bytesOutCurr - bytesOutPrev);
+    LOG.info("Closed Simple Output");
+    return null;
+  }
+
+  @Override
+  public void setNumPhysicalOutputs(int numOutputs) {
+    // Nothing to do for now
+  }
+
+  /**
+   * MROutput expects that a Processor call commit prior to the
+   * Processor's completion
+   * @throws IOException
+   */
+  public void commit() throws IOException {
+    close();
+    if (useNewApi) {
+      committer.commitTask(newApiTaskAttemptContext);
+    } else {
+      committer.commitTask(oldApiTaskAttemptContext);
+    }
+  }
+
+
+  /**
+   * MROutput expects that a Processor call abort in case of any error
+   * ( including an error during commit ) prior to the Processor's completion
+   * @throws IOException
+   */
+  public void abort() throws IOException {
+    close();
+    if (useNewApi) {
+      committer.abortTask(newApiTaskAttemptContext);
+    } else {
+      committer.abortTask(oldApiTaskAttemptContext);
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/5d86b935/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/output/SimpleOutput.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/output/SimpleOutput.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/output/SimpleOutput.java
deleted file mode 100644
index d82c9e2..0000000
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/output/SimpleOutput.java
+++ /dev/null
@@ -1,326 +0,0 @@
-package org.apache.tez.mapreduce.output;
-
-import java.io.IOException;
-import java.text.NumberFormat;
-import java.util.List;
-import java.util.concurrent.atomic.AtomicBoolean;
-
-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.Path;
-import org.apache.hadoop.fs.FileSystem.Statistics;
-import org.apache.hadoop.mapred.FileOutputCommitter;
-import org.apache.hadoop.mapred.FileOutputFormat;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.JobContext;
-import org.apache.hadoop.mapred.TaskAttemptID;
-import org.apache.hadoop.mapred.TaskID;
-import org.apache.hadoop.mapreduce.OutputCommitter;
-import org.apache.hadoop.mapreduce.OutputFormat;
-import org.apache.hadoop.mapreduce.TaskAttemptContext;
-import org.apache.hadoop.mapreduce.TaskType;
-import org.apache.hadoop.mapreduce.lib.output.FileOutputFormatCounter;
-import org.apache.hadoop.util.ReflectionUtils;
-import org.apache.tez.common.TezUtils;
-import org.apache.tez.common.counters.TaskCounter;
-import org.apache.tez.common.counters.TezCounter;
-import org.apache.tez.engine.api.Event;
-import org.apache.tez.engine.api.KVWriter;
-import org.apache.tez.engine.api.LogicalOutput;
-import org.apache.tez.engine.api.TezOutputContext;
-import org.apache.tez.mapreduce.common.Utils;
-import org.apache.tez.mapreduce.hadoop.MRConfig;
-import org.apache.tez.mapreduce.hadoop.MRJobConfig;
-import org.apache.tez.mapreduce.hadoop.mapred.MRReporter;
-import org.apache.tez.mapreduce.hadoop.mapreduce.TaskAttemptContextImpl;
-import org.apache.tez.mapreduce.processor.MRTaskReporter;
-
-public class SimpleOutput implements LogicalOutput {
-
-  private static final Log LOG = LogFactory.getLog(SimpleOutput.class);
-
-  private static final NumberFormat NUMBER_FORMAT = NumberFormat.getInstance();
-  static {
-    NUMBER_FORMAT.setMinimumIntegerDigits(5);
-    NUMBER_FORMAT.setGroupingUsed(false);
-  }
-
-  private TezOutputContext outputContext;
-  private JobConf jobConf;
-  boolean useNewApi;
-  private AtomicBoolean closed = new AtomicBoolean(false);
-
-  @SuppressWarnings("rawtypes")
-  org.apache.hadoop.mapreduce.OutputFormat newOutputFormat;
-  @SuppressWarnings("rawtypes")
-  org.apache.hadoop.mapreduce.RecordWriter newRecordWriter;
-
-  @SuppressWarnings("rawtypes")
-  org.apache.hadoop.mapred.OutputFormat oldOutputFormat;
-  @SuppressWarnings("rawtypes")
-  org.apache.hadoop.mapred.RecordWriter oldRecordWriter;
-
-  private TezCounter outputRecordCounter;
-  private TezCounter fileOutputByteCounter;
-  private List<Statistics> fsStats;
-
-  private TaskAttemptContext newApiTaskAttemptContext;
-  private org.apache.hadoop.mapred.TaskAttemptContext oldApiTaskAttemptContext;
-
-  private boolean isMapperOutput;
-
-  private OutputCommitter committer;
-
-  @Override
-  public List<Event> initialize(TezOutputContext outputContext)
-      throws IOException, InterruptedException {
-    LOG.info("Initializing Simple Output");
-    this.outputContext = outputContext;
-    Configuration conf = TezUtils.createConfFromUserPayload(
-        outputContext.getUserPayload());
-    this.jobConf = new JobConf(conf);
-    this.useNewApi = this.jobConf.getUseNewMapper();
-    this.isMapperOutput = jobConf.getBoolean(MRConfig.IS_MAP_PROCESSOR,
-        false);
-    jobConf.setInt(MRJobConfig.APPLICATION_ATTEMPT_ID,
-        outputContext.getDAGAttemptNumber());
-
-    outputRecordCounter = outputContext.getCounters().findCounter(
-        TaskCounter.MAP_OUTPUT_RECORDS);
-    fileOutputByteCounter = outputContext.getCounters().findCounter(
-        FileOutputFormatCounter.BYTES_WRITTEN);
-
-    if (useNewApi) {
-      newApiTaskAttemptContext = createTaskAttemptContext();
-      try {
-        newOutputFormat =
-            ReflectionUtils.newInstance(
-                newApiTaskAttemptContext.getOutputFormatClass(), jobConf);
-      } catch (ClassNotFoundException cnfe) {
-        throw new IOException(cnfe);
-      }
-
-      List<Statistics> matchedStats = null;
-      if (newOutputFormat instanceof
-          org.apache.hadoop.mapreduce.lib.output.FileOutputFormat) {
-        matchedStats =
-            Utils.getFsStatistics(
-                org.apache.hadoop.mapreduce.lib.output.FileOutputFormat
-                    .getOutputPath(newApiTaskAttemptContext),
-                jobConf);
-      }
-      fsStats = matchedStats;
-
-      long bytesOutPrev = getOutputBytes();
-      try {
-        newRecordWriter =
-            newOutputFormat.getRecordWriter(newApiTaskAttemptContext);
-      } catch (InterruptedException e) {
-        throw new IOException("Interrupted while creating record writer", e);
-      }
-      long bytesOutCurr = getOutputBytes();
-      fileOutputByteCounter.increment(bytesOutCurr - bytesOutPrev);
-    } else {
-      TaskAttemptID taskAttemptId = new TaskAttemptID(new TaskID(Long.toString(
-          outputContext.getApplicationId().getClusterTimestamp()),
-          outputContext.getApplicationId().getId(),
-          (isMapperOutput ? TaskType.MAP : TaskType.REDUCE),
-          outputContext.getTaskIndex()),
-          outputContext.getTaskAttemptNumber());
-      jobConf.set(JobContext.TASK_ATTEMPT_ID, taskAttemptId.toString());
-      jobConf.set(JobContext.TASK_ID, taskAttemptId.getTaskID().toString());
-      jobConf.setBoolean(JobContext.TASK_ISMAP, isMapperOutput);
-      jobConf.setInt(JobContext.TASK_PARTITION,
-          taskAttemptId.getTaskID().getId());
-      jobConf.set(JobContext.ID, taskAttemptId.getJobID().toString());
-
-      oldApiTaskAttemptContext =
-          new org.apache.tez.mapreduce.hadoop.mapred.TaskAttemptContextImpl(
-              jobConf, taskAttemptId,
-              new MRTaskReporter(outputContext));
-      oldOutputFormat = jobConf.getOutputFormat();
-
-      List<Statistics> matchedStats = null;
-      if (oldOutputFormat
-          instanceof org.apache.hadoop.mapred.FileOutputFormat) {
-        matchedStats =
-            Utils.getFsStatistics(
-                org.apache.hadoop.mapred.FileOutputFormat.getOutputPath(
-                    jobConf),
-                jobConf);
-      }
-      fsStats = matchedStats;
-
-      FileSystem fs = FileSystem.get(jobConf);
-      String finalName = getOutputName();
-
-      long bytesOutPrev = getOutputBytes();
-      oldRecordWriter =
-          oldOutputFormat.getRecordWriter(
-              fs, jobConf, finalName, new MRReporter(outputContext));
-      long bytesOutCurr = getOutputBytes();
-      fileOutputByteCounter.increment(bytesOutCurr - bytesOutPrev);
-    }
-    initCommitter(jobConf, useNewApi);
-
-    LOG.info("Initialized Simple Output"
-        + ", using_new_api: " + useNewApi);
-    return null;
-  }
-
-  public void initCommitter(JobConf job, boolean useNewApi)
-      throws IOException, InterruptedException {
-
-    if (useNewApi) {
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("using new api for output committer");
-      }
-
-      OutputFormat<?, ?> outputFormat = null;
-      try {
-        outputFormat = ReflectionUtils.newInstance(
-            newApiTaskAttemptContext.getOutputFormatClass(), job);
-      } catch (ClassNotFoundException cnfe) {
-        throw new IOException("Unknown OutputFormat", cnfe);
-      }
-      this.committer = outputFormat.getOutputCommitter(
-          newApiTaskAttemptContext);
-    } else {
-      this.committer = job.getOutputCommitter();
-    }
-
-    Path outputPath = FileOutputFormat.getOutputPath(job);
-    if (outputPath != null) {
-      if ((this.committer instanceof FileOutputCommitter)) {
-        FileOutputFormat.setWorkOutputPath(job,
-            ((FileOutputCommitter) this.committer).getTaskAttemptPath(
-                oldApiTaskAttemptContext));
-      } else {
-        FileOutputFormat.setWorkOutputPath(job, outputPath);
-      }
-    }
-    if (useNewApi) {
-      this.committer.setupTask(newApiTaskAttemptContext);
-    } else {
-      this.committer.setupTask(oldApiTaskAttemptContext);
-    }
-  }
-
-  public boolean isCommitRequired() throws IOException {
-    if (useNewApi) {
-      return committer.needsTaskCommit(newApiTaskAttemptContext);
-    } else {
-      return committer.needsTaskCommit(oldApiTaskAttemptContext);
-    }
-  }
-
-  private TaskAttemptContext createTaskAttemptContext() {
-    return new TaskAttemptContextImpl(this.jobConf, outputContext,
-        isMapperOutput);
-  }
-
-  private long getOutputBytes() {
-    if (fsStats == null) return 0;
-    long bytesWritten = 0;
-    for (Statistics stat: fsStats) {
-      bytesWritten = bytesWritten + stat.getBytesWritten();
-    }
-    return bytesWritten;
-  }
-
-  private String getOutputName() {
-    return "part-" + NUMBER_FORMAT.format(outputContext.getTaskIndex());
-  }
-
-  @Override
-  public KVWriter getWriter() throws IOException {
-    return new KVWriter() {
-      private final boolean useNewWriter = useNewApi;
-
-      @SuppressWarnings("unchecked")
-      @Override
-      public void write(Object key, Object value) throws IOException {
-        long bytesOutPrev = getOutputBytes();
-        if (useNewWriter) {
-          try {
-            newRecordWriter.write(key, value);
-          } catch (InterruptedException e) {
-            Thread.currentThread().interrupt();
-            throw new IOException("Interrupted while writing next key-value",e);
-          }
-        } else {
-          oldRecordWriter.write(key, value);
-        }
-
-        long bytesOutCurr = getOutputBytes();
-        fileOutputByteCounter.increment(bytesOutCurr - bytesOutPrev);
-        outputRecordCounter.increment(1);
-      }
-    };
-  }
-
-  @Override
-  public void handleEvents(List<Event> outputEvents) {
-    // Not expecting any events at the moment.
-  }
-
-  @Override
-  public synchronized List<Event> close() throws IOException {
-    if (closed.getAndSet(true)) {
-      return null;
-    }
-
-    LOG.info("Closing Simple Output");
-    long bytesOutPrev = getOutputBytes();
-    if (useNewApi) {
-      try {
-        newRecordWriter.close(newApiTaskAttemptContext);
-      } catch (InterruptedException e) {
-        throw new IOException("Interrupted while closing record writer", e);
-      }
-    } else {
-      oldRecordWriter.close(null);
-    }
-    long bytesOutCurr = getOutputBytes();
-    fileOutputByteCounter.increment(bytesOutCurr - bytesOutPrev);
-    LOG.info("Closed Simple Output");
-    return null;
-  }
-
-  @Override
-  public void setNumPhysicalOutputs(int numOutputs) {
-    // Nothing to do for now
-  }
-
-  /**
-   * SimpleOutput expects that a Processor call commit prior to the
-   * Processor's completion
-   * @throws IOException
-   */
-  public void commit() throws IOException {
-    close();
-    if (useNewApi) {
-      committer.commitTask(newApiTaskAttemptContext);
-    } else {
-      committer.commitTask(oldApiTaskAttemptContext);
-    }
-  }
-
-
-  /**
-   * SimpleOutput expects that a Processor call abort in case of any error
-   * ( including an error during commit ) prior to the Processor's completion
-   * @throws IOException
-   */
-  public void abort() throws IOException {
-    close();
-    if (useNewApi) {
-      committer.abortTask(newApiTaskAttemptContext);
-    } else {
-      committer.abortTask(oldApiTaskAttemptContext);
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/5d86b935/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/MRTask.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/MRTask.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/MRTask.java
index f7404d4..fac1454 100644
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/MRTask.java
+++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/MRTask.java
@@ -81,7 +81,7 @@ import org.apache.tez.mapreduce.hadoop.MRConfig;
 import org.apache.tez.mapreduce.hadoop.MRJobConfig;
 import org.apache.tez.mapreduce.hadoop.mapred.TaskAttemptContextImpl;
 import org.apache.tez.mapreduce.hadoop.mapreduce.JobContextImpl;
-import org.apache.tez.mapreduce.output.SimpleOutput;
+import org.apache.tez.mapreduce.output.MROutput;
 
 @SuppressWarnings("deprecation")
 public abstract class MRTask {
@@ -423,8 +423,8 @@ public abstract class MRTask {
         + " And is in the process of committing");
     // TODO change this to use the new context
     // TODO TEZ Interaciton between Commit and OutputReady. Merge ?
-    if (output instanceof SimpleOutput) {
-      SimpleOutput sOut = (SimpleOutput)output;
+    if (output instanceof MROutput) {
+      MROutput sOut = (MROutput)output;
       if (sOut.isCommitRequired()) {
         //wait for commit approval and commit
         // TODO EVENTUALLY - Commit is not required for map tasks.
@@ -458,7 +458,7 @@ public abstract class MRTask {
     statusUpdate();
   }
 
-  private void commit(SimpleOutput output) throws IOException {
+  private void commit(MROutput output) throws IOException {
     int retries = 3;
     while (true) {
       // This will loop till the AM asks for the task to be killed. As
@@ -495,7 +495,7 @@ public abstract class MRTask {
   }
 
   private
-  void discardOutput(SimpleOutput output) {
+  void discardOutput(MROutput output) {
     try {
       output.abort();
     } catch (IOException ioe)  {

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/5d86b935/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/MRTaskReporter.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/MRTaskReporter.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/MRTaskReporter.java
index 22312f7..85139ed 100644
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/MRTaskReporter.java
+++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/MRTaskReporter.java
@@ -65,7 +65,7 @@ public class MRTaskReporter
     if (isProcessorContext) {
       ((TezProcessorContext)context).setProgress(progress);
     } else {
-      // TODO FIXME NEWTEZ - will simpleoutput's reporter use this api?
+      // TODO FIXME NEWTEZ - will MROutput's reporter use this api?
     }
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/5d86b935/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/map/MapProcessor.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/map/MapProcessor.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/map/MapProcessor.java
index 2084146..e4b990a 100644
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/map/MapProcessor.java
+++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/map/MapProcessor.java
@@ -44,9 +44,9 @@ import org.apache.tez.engine.api.LogicalOutput;
 import org.apache.tez.engine.api.TezProcessorContext;
 import org.apache.tez.engine.lib.output.OnFileSortedOutput;
 import org.apache.tez.mapreduce.hadoop.mapreduce.MapContextImpl;
-import org.apache.tez.mapreduce.input.SimpleInput;
-import org.apache.tez.mapreduce.input.SimpleInputLegacy;
-import org.apache.tez.mapreduce.output.SimpleOutput;
+import org.apache.tez.mapreduce.input.MRInput;
+import org.apache.tez.mapreduce.input.MRInputLegacy;
+import org.apache.tez.mapreduce.output.MROutput;
 import org.apache.tez.mapreduce.processor.MRTask;
 import org.apache.tez.mapreduce.processor.MRTaskReporter;
 
@@ -99,15 +99,15 @@ public class MapProcessor extends MRTask implements LogicalIOProcessor {
     LogicalOutput out = outputs.values().iterator().next();
 
     // Sanity check
-    if (!(in instanceof SimpleInputLegacy)) {
+    if (!(in instanceof MRInputLegacy)) {
       throw new IOException(new TezException(
           "Only Simple Input supported. Input: " + in.getClass()));
     }
-    SimpleInputLegacy input = (SimpleInputLegacy)in;
+    MRInputLegacy input = (MRInputLegacy)in;
 
     KVWriter kvWriter = null;
     if (!(out instanceof OnFileSortedOutput)) {
-      kvWriter = ((SimpleOutput)out).getWriter();
+      kvWriter = ((MROutput)out).getWriter();
     } else {
       kvWriter = ((OnFileSortedOutput)out).getWriter();
     }
@@ -124,13 +124,13 @@ public class MapProcessor extends MRTask implements LogicalIOProcessor {
   void runOldMapper(
       final JobConf job,
       final MRTaskReporter reporter,
-      final SimpleInputLegacy input,
+      final MRInputLegacy input,
       final KVWriter output
       ) throws IOException, InterruptedException {
 
     // Initialize input in-line since it sets parameters which may be used by the processor.
-    // Done only for SimpleInput.
-    // TODO use new method in SimpleInput to get required info
+    // Done only for MRInput.
+    // TODO use new method in MRInput to get required info
     //input.initialize(job, master);
 
     RecordReader in = new OldRecordReader(input);
@@ -147,13 +147,13 @@ public class MapProcessor extends MRTask implements LogicalIOProcessor {
 
   private void runNewMapper(final JobConf job,
       MRTaskReporter reporter,
-      final SimpleInputLegacy in,
+      final MRInputLegacy in,
       KVWriter out
       ) throws IOException, InterruptedException {
 
     // Initialize input in-line since it sets parameters which may be used by the processor.
-    // Done only for SimpleInput.
-    // TODO use new method in SimpleInput to get required info
+    // Done only for MRInput.
+    // TODO use new method in MRInput to get required info
     //in.initialize(job, master);
 
     // make a task context so we can get the classes
@@ -197,10 +197,10 @@ public class MapProcessor extends MRTask implements LogicalIOProcessor {
 
   private static class NewRecordReader extends
       org.apache.hadoop.mapreduce.RecordReader {
-    private final SimpleInput in;
+    private final MRInput in;
     private KVReader reader;
 
-    private NewRecordReader(SimpleInput in) throws IOException {
+    private NewRecordReader(MRInput in) throws IOException {
       this.in = in;
       this.reader = in.getReader();
     }
@@ -241,38 +241,38 @@ public class MapProcessor extends MRTask implements LogicalIOProcessor {
   }
 
   private static class OldRecordReader implements RecordReader {
-    private final SimpleInputLegacy simpleInput;
+    private final MRInputLegacy mrInput;
 
-    private OldRecordReader(SimpleInputLegacy simpleInput) {
-      this.simpleInput = simpleInput;
+    private OldRecordReader(MRInputLegacy mrInput) {
+      this.mrInput = mrInput;
     }
 
     @Override
     public boolean next(Object key, Object value) throws IOException {
       // TODO broken
-//      simpleInput.setKey(key);
-//      simpleInput.setValue(value);
+//      mrInput.setKey(key);
+//      mrInput.setValue(value);
 //      try {
-//        return simpleInput.hasNext();
+//        return mrInput.hasNext();
 //      } catch (InterruptedException ie) {
 //        throw new IOException(ie);
 //      }
-      return simpleInput.getOldRecordReader().next(key, value);
+      return mrInput.getOldRecordReader().next(key, value);
     }
 
     @Override
     public Object createKey() {
-      return simpleInput.getOldRecordReader().createKey();
+      return mrInput.getOldRecordReader().createKey();
     }
 
     @Override
     public Object createValue() {
-      return simpleInput.getOldRecordReader().createValue();
+      return mrInput.getOldRecordReader().createValue();
     }
 
     @Override
     public long getPos() throws IOException {
-      return simpleInput.getOldRecordReader().getPos();
+      return mrInput.getOldRecordReader().getPos();
     }
 
     @Override
@@ -282,7 +282,7 @@ public class MapProcessor extends MRTask implements LogicalIOProcessor {
     @Override
     public float getProgress() throws IOException {
       try {
-        return simpleInput.getProgress();
+        return mrInput.getProgress();
       } catch (InterruptedException ie) {
         throw new IOException(ie);
       }

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/5d86b935/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/reduce/ReduceProcessor.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/reduce/ReduceProcessor.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/reduce/ReduceProcessor.java
index 9274765..19acb39 100644
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/reduce/ReduceProcessor.java
+++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/reduce/ReduceProcessor.java
@@ -48,7 +48,7 @@ import org.apache.tez.engine.common.ConfigUtils;
 import org.apache.tez.engine.common.sort.impl.TezRawKeyValueIterator;
 import org.apache.tez.engine.lib.input.ShuffledMergedInputLegacy;
 import org.apache.tez.engine.lib.output.OnFileSortedOutput;
-import org.apache.tez.mapreduce.output.SimpleOutput;
+import org.apache.tez.mapreduce.output.MROutput;
 import org.apache.tez.mapreduce.processor.MRTask;
 import org.apache.tez.mapreduce.processor.MRTaskReporter;
 
@@ -133,8 +133,8 @@ implements LogicalIOProcessor {
     KVReader kvReader = shuffleInput.getReader();
 
     KVWriter kvWriter = null;
-    if((out instanceof SimpleOutput)) {
-      kvWriter = ((SimpleOutput) out).getWriter();
+    if((out instanceof MROutput)) {
+      kvWriter = ((MROutput) out).getWriter();
     } else if ((out instanceof OnFileSortedOutput)) {
       kvWriter = ((OnFileSortedOutput) out).getWriter();
     } else {

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/5d86b935/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/processor/map/TestMapProcessor.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/processor/map/TestMapProcessor.java b/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/processor/map/TestMapProcessor.java
index 06e2f4b..89292ab 100644
--- a/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/processor/map/TestMapProcessor.java
+++ b/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/processor/map/TestMapProcessor.java
@@ -48,7 +48,7 @@ import org.apache.tez.mapreduce.TestUmbilical;
 import org.apache.tez.mapreduce.hadoop.MRJobConfig;
 import org.apache.tez.mapreduce.hadoop.MultiStageMRConfToTezTranslator;
 import org.apache.tez.mapreduce.hadoop.MultiStageMRConfigUtil;
-import org.apache.tez.mapreduce.input.SimpleInputLegacy;
+import org.apache.tez.mapreduce.input.MRInputLegacy;
 import org.apache.tez.mapreduce.partition.MRPartitioner;
 import org.apache.tez.mapreduce.processor.MapUtils;
 import org.junit.After;
@@ -120,7 +120,7 @@ public class TestMapProcessor {
     
     MapUtils.generateInputSplit(localFs, workDir, job, mapInput);
     
-    InputSpec mapInputSpec = new InputSpec("NullSrcVertex", new InputDescriptor(SimpleInputLegacy.class.getName()), 0);
+    InputSpec mapInputSpec = new InputSpec("NullSrcVertex", new InputDescriptor(MRInputLegacy.class.getName()), 0);
     OutputSpec mapOutputSpec = new OutputSpec("NullDestVertex", new OutputDescriptor(LocalOnFileSorterOutput.class.getName()), 1);
 
     LogicalIOProcessorRuntimeTask task = MapUtils.createLogicalTask(localFs, workDir, job, 0,
@@ -191,7 +191,7 @@ public class TestMapProcessor {
 //            localFs, workDir, job, 0, new Path(workDir, "map0"), 
 //            new TestUmbilicalProtocol(true), vertexName, 
 //            Collections.singletonList(new InputSpec("NullVertex", 0,
-//                SimpleInput.class.getName())),
+//                MRInput.class.getName())),
 //            Collections.singletonList(new OutputSpec("FakeVertex", 1,
 //                OldInMemorySortedOutput.class.getName()))
 //            );

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/5d86b935/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/processor/reduce/TestReduceProcessor.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/processor/reduce/TestReduceProcessor.java b/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/processor/reduce/TestReduceProcessor.java
index a3abd76..274c353 100644
--- a/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/processor/reduce/TestReduceProcessor.java
+++ b/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/processor/reduce/TestReduceProcessor.java
@@ -54,8 +54,8 @@ import org.apache.tez.mapreduce.hadoop.IDConverter;
 import org.apache.tez.mapreduce.hadoop.MRJobConfig;
 import org.apache.tez.mapreduce.hadoop.MultiStageMRConfToTezTranslator;
 import org.apache.tez.mapreduce.hadoop.MultiStageMRConfigUtil;
-import org.apache.tez.mapreduce.input.SimpleInputLegacy;
-import org.apache.tez.mapreduce.output.SimpleOutput;
+import org.apache.tez.mapreduce.input.MRInputLegacy;
+import org.apache.tez.mapreduce.output.MROutput;
 import org.apache.tez.mapreduce.partition.MRPartitioner;
 import org.apache.tez.mapreduce.processor.MapUtils;
 import org.junit.After;
@@ -125,7 +125,7 @@ public class TestReduceProcessor {
     Path mapInput = new Path(workDir, "map0");
     MapUtils.generateInputSplit(localFs, workDir, mapConf, mapInput);
     
-    InputSpec mapInputSpec = new InputSpec("NullSrcVertex", new InputDescriptor(SimpleInputLegacy.class.getName()), 0);
+    InputSpec mapInputSpec = new InputSpec("NullSrcVertex", new InputDescriptor(MRInputLegacy.class.getName()), 0);
     OutputSpec mapOutputSpec = new OutputSpec("NullDestVertex", new OutputDescriptor(LocalOnFileSorterOutput.class.getName()), 1);
     // Run a map
     LogicalIOProcessorRuntimeTask mapTask = MapUtils.createLogicalTask(localFs, workDir, mapConf, 0,
@@ -152,7 +152,7 @@ public class TestReduceProcessor {
         ReduceProcessor.class.getName()).setUserPayload(TezUtils.createUserPayloadFromConf(reduceConf));
     
     InputSpec reduceInputSpec = new InputSpec(mapVertexName, new InputDescriptor(LocalMergedInput.class.getName()), 1);
-    OutputSpec reduceOutputSpec = new OutputSpec("NullDestinationVertex", new OutputDescriptor(SimpleOutput.class.getName()), 1);
+    OutputSpec reduceOutputSpec = new OutputSpec("NullDestinationVertex", new OutputDescriptor(MROutput.class.getName()), 1);
     
     // Now run a reduce
     TaskSpec taskSpec = new TaskSpec(


[12/50] [abbrv] Rename *.new* packages back to what they should be, remove dead code from the old packages - mapreduce module - tez-engine module (part of TEZ-398). (sseth)

Posted by ss...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/task/RuntimeTask.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/task/RuntimeTask.java b/tez-engine/src/main/java/org/apache/tez/engine/task/RuntimeTask.java
deleted file mode 100644
index 5631c78..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/task/RuntimeTask.java
+++ /dev/null
@@ -1,92 +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.engine.task;
-
-import java.io.IOException;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.tez.common.TezEngineTaskContext;
-import org.apache.tez.engine.api.Input;
-import org.apache.tez.engine.api.Master;
-import org.apache.tez.engine.api.Output;
-import org.apache.tez.engine.api.Processor;
-import org.apache.tez.engine.api.Task;
-
-public class RuntimeTask implements Task {
-
-  protected final Input[] inputs;
-  protected final Output[] outputs;
-  protected final Processor processor;
-  
-  protected TezEngineTaskContext taskContext;
-  protected byte[] userPayload;
-  protected Configuration conf;
-  protected Master master;
-  
-  public RuntimeTask(TezEngineTaskContext taskContext,
-      Processor processor,
-      Input[] inputs,
-      Output[] outputs) {
-    this.taskContext = taskContext;
-    this.inputs = inputs;
-    this.processor = processor;
-    this.outputs = outputs;
-  }
-
-  @Override
-  public void initialize(Configuration conf, byte[] userPayload,
-      Master master) throws IOException, InterruptedException {
-    this.conf = conf;
-    this.userPayload = userPayload;
-    this.master = master;
-
-    // NOTE: Allow processor to initialize input/output
-    processor.initialize(this.conf, this.master);
-  }
-
-  @Override
-  public Input[] getInputs() {
-    return inputs;
-  }
-
-  @Override
-  public Processor getProcessor() {
-    return processor;
-  }
-
-  @Override
-  public Output[] getOutputs() {
-    return outputs;
-  }
-
-  public void run() throws IOException, InterruptedException {
-    processor.process(inputs, outputs);
-  }
-
-  public void close() throws IOException, InterruptedException {
-    // NOTE: Allow processor to close input/output
-    // This can be changed to close input/output since MRRuntimeTask is used for
-    // MR jobs, which changes the order.
-    processor.close();
-  }
-
-  @Override
-  public Configuration getConfiguration() {
-    return this.conf;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-mapreduce/src/main/java/org/apache/hadoop/mapred/LocalClientProtocolProviderTez.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/hadoop/mapred/LocalClientProtocolProviderTez.java b/tez-mapreduce/src/main/java/org/apache/hadoop/mapred/LocalClientProtocolProviderTez.java
index f51b1da..fc2395f 100644
--- a/tez-mapreduce/src/main/java/org/apache/hadoop/mapred/LocalClientProtocolProviderTez.java
+++ b/tez-mapreduce/src/main/java/org/apache/hadoop/mapred/LocalClientProtocolProviderTez.java
@@ -1,54 +1,54 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.mapred;
-
-import java.io.IOException;
-import java.net.InetSocketAddress;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.mapreduce.MRConfig;
-import org.apache.hadoop.mapreduce.protocol.ClientProtocol;
-import org.apache.hadoop.mapreduce.protocol.ClientProtocolProvider;
-import org.apache.tez.dag.api.TezConfiguration;
-
-@InterfaceAudience.Private
-public class LocalClientProtocolProviderTez extends ClientProtocolProvider {
-
-  @Override
-  public ClientProtocol create(Configuration conf) throws IOException {
-    String framework =
-        conf.get(MRConfig.FRAMEWORK_NAME);
-    if (!TezConfiguration.LOCAL_FRAMEWORK_NAME.equals(framework)) {
-      return null;
-    }
-    return new LocalJobRunnerTez(conf);
-  }
-
-  @Override
-  public ClientProtocol create(InetSocketAddress addr, Configuration conf) {
-    return null; // LocalJobRunner doesn't use a socket
-  }
-
-  @Override
-  public void close(ClientProtocol clientProtocol) {
-    // no clean up required
-  }
-
-}
+///**
+// * Licensed to the Apache Software Foundation (ASF) under one
+// * or more contributor license agreements.  See the NOTICE file
+// * distributed with this work for additional information
+// * regarding copyright ownership.  The ASF licenses this file
+// * to you under the Apache License, Version 2.0 (the
+// * "License"); you may not use this file except in compliance
+// * with the License.  You may obtain a copy of the License at
+// *
+// *     http://www.apache.org/licenses/LICENSE-2.0
+// *
+// * Unless required by applicable law or agreed to in writing, software
+// * distributed under the License is distributed on an "AS IS" BASIS,
+// * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// * See the License for the specific language governing permissions and
+// * limitations under the License.
+// */
+//
+//package org.apache.hadoop.mapred;
+//
+//import java.io.IOException;
+//import java.net.InetSocketAddress;
+//
+//import org.apache.hadoop.classification.InterfaceAudience;
+//import org.apache.hadoop.conf.Configuration;
+//import org.apache.hadoop.mapreduce.MRConfig;
+//import org.apache.hadoop.mapreduce.protocol.ClientProtocol;
+//import org.apache.hadoop.mapreduce.protocol.ClientProtocolProvider;
+//import org.apache.tez.dag.api.TezConfiguration;
+//
+//@InterfaceAudience.Private
+//public class LocalClientProtocolProviderTez extends ClientProtocolProvider {
+//
+//  @Override
+//  public ClientProtocol create(Configuration conf) throws IOException {
+//    String framework =
+//        conf.get(MRConfig.FRAMEWORK_NAME);
+//    if (!TezConfiguration.LOCAL_FRAMEWORK_NAME.equals(framework)) {
+//      return null;
+//    }
+//    return new LocalJobRunnerTez(conf);
+//  }
+//
+//  @Override
+//  public ClientProtocol create(InetSocketAddress addr, Configuration conf) {
+//    return null; // LocalJobRunner doesn't use a socket
+//  }
+//
+//  @Override
+//  public void close(ClientProtocol clientProtocol) {
+//    // no clean up required
+//  }
+//
+//}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-mapreduce/src/main/java/org/apache/hadoop/mapred/LocalJobRunnerMetricsTez.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/hadoop/mapred/LocalJobRunnerMetricsTez.java b/tez-mapreduce/src/main/java/org/apache/hadoop/mapred/LocalJobRunnerMetricsTez.java
index 10c404e..28dcdfb 100644
--- a/tez-mapreduce/src/main/java/org/apache/hadoop/mapred/LocalJobRunnerMetricsTez.java
+++ b/tez-mapreduce/src/main/java/org/apache/hadoop/mapred/LocalJobRunnerMetricsTez.java
@@ -1,98 +1,98 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.mapred;
-
-import org.apache.hadoop.metrics.MetricsContext;
-import org.apache.hadoop.metrics.MetricsRecord;
-import org.apache.hadoop.metrics.MetricsUtil;
-import org.apache.hadoop.metrics.Updater;
-import org.apache.hadoop.metrics.jvm.JvmMetrics;
-
-@SuppressWarnings("deprecation")
-class LocalJobRunnerMetricsTez implements Updater {
-  private final MetricsRecord metricsRecord;
-
-  private int numMapTasksLaunched = 0;
-  private int numMapTasksCompleted = 0;
-  private int numReduceTasksLaunched = 0;
-  private int numReduceTasksCompleted = 0;
-  private int numWaitingMaps = 0;
-  private int numWaitingReduces = 0;
-  
-  public LocalJobRunnerMetricsTez(JobConf conf) {
-    String sessionId = conf.getSessionId();
-    // Initiate JVM Metrics
-    JvmMetrics.init("JobTracker", sessionId);
-    // Create a record for map-reduce metrics
-    MetricsContext context = MetricsUtil.getContext("mapred");
-    // record name is jobtracker for compatibility 
-    metricsRecord = MetricsUtil.createRecord(context, "jobtracker");
-    metricsRecord.setTag("sessionId", sessionId);
-    context.registerUpdater(this);
-  }
-    
-  /**
-   * Since this object is a registered updater, this method will be called
-   * periodically, e.g. every 5 seconds.
-   */
-  public void doUpdates(MetricsContext unused) {
-    synchronized (this) {
-      metricsRecord.incrMetric("maps_launched", numMapTasksLaunched);
-      metricsRecord.incrMetric("maps_completed", numMapTasksCompleted);
-      metricsRecord.incrMetric("reduces_launched", numReduceTasksLaunched);
-      metricsRecord.incrMetric("reduces_completed", numReduceTasksCompleted);
-      metricsRecord.incrMetric("waiting_maps", numWaitingMaps);
-      metricsRecord.incrMetric("waiting_reduces", numWaitingReduces);
-
-      numMapTasksLaunched = 0;
-      numMapTasksCompleted = 0;
-      numReduceTasksLaunched = 0;
-      numReduceTasksCompleted = 0;
-      numWaitingMaps = 0;
-      numWaitingReduces = 0;
-    }
-    metricsRecord.update();
-  }
-
-  public synchronized void launchMap(TaskAttemptID taskAttemptID) {
-    ++numMapTasksLaunched;
-    decWaitingMaps(taskAttemptID.getJobID(), 1);
-  }
-
-  public synchronized void completeMap(TaskAttemptID taskAttemptID) {
-    ++numMapTasksCompleted;
-  }
-
-  public synchronized void launchReduce(TaskAttemptID taskAttemptID) {
-    ++numReduceTasksLaunched;
-    decWaitingReduces(taskAttemptID.getJobID(), 1);
-  }
-
-  public synchronized void completeReduce(TaskAttemptID taskAttemptID) {
-    ++numReduceTasksCompleted;
-  }
-
-  private synchronized void decWaitingMaps(JobID id, int task) {
-    numWaitingMaps -= task;
-  }
-  
-  private synchronized void decWaitingReduces(JobID id, int task){
-    numWaitingReduces -= task;
-  }
-
-}
+///**
+// * Licensed to the Apache Software Foundation (ASF) under one
+// * or more contributor license agreements.  See the NOTICE file
+// * distributed with this work for additional information
+// * regarding copyright ownership.  The ASF licenses this file
+// * to you under the Apache License, Version 2.0 (the
+// * "License"); you may not use this file except in compliance
+// * with the License.  You may obtain a copy of the License at
+// *
+// *     http://www.apache.org/licenses/LICENSE-2.0
+// *
+// * Unless required by applicable law or agreed to in writing, software
+// * distributed under the License is distributed on an "AS IS" BASIS,
+// * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// * See the License for the specific language governing permissions and
+// * limitations under the License.
+// */
+//package org.apache.hadoop.mapred;
+//
+//import org.apache.hadoop.metrics.MetricsContext;
+//import org.apache.hadoop.metrics.MetricsRecord;
+//import org.apache.hadoop.metrics.MetricsUtil;
+//import org.apache.hadoop.metrics.Updater;
+//import org.apache.hadoop.metrics.jvm.JvmMetrics;
+//
+//@SuppressWarnings("deprecation")
+//class LocalJobRunnerMetricsTez implements Updater {
+//  private final MetricsRecord metricsRecord;
+//
+//  private int numMapTasksLaunched = 0;
+//  private int numMapTasksCompleted = 0;
+//  private int numReduceTasksLaunched = 0;
+//  private int numReduceTasksCompleted = 0;
+//  private int numWaitingMaps = 0;
+//  private int numWaitingReduces = 0;
+//  
+//  public LocalJobRunnerMetricsTez(JobConf conf) {
+//    String sessionId = conf.getSessionId();
+//    // Initiate JVM Metrics
+//    JvmMetrics.init("JobTracker", sessionId);
+//    // Create a record for map-reduce metrics
+//    MetricsContext context = MetricsUtil.getContext("mapred");
+//    // record name is jobtracker for compatibility 
+//    metricsRecord = MetricsUtil.createRecord(context, "jobtracker");
+//    metricsRecord.setTag("sessionId", sessionId);
+//    context.registerUpdater(this);
+//  }
+//    
+//  /**
+//   * Since this object is a registered updater, this method will be called
+//   * periodically, e.g. every 5 seconds.
+//   */
+//  public void doUpdates(MetricsContext unused) {
+//    synchronized (this) {
+//      metricsRecord.incrMetric("maps_launched", numMapTasksLaunched);
+//      metricsRecord.incrMetric("maps_completed", numMapTasksCompleted);
+//      metricsRecord.incrMetric("reduces_launched", numReduceTasksLaunched);
+//      metricsRecord.incrMetric("reduces_completed", numReduceTasksCompleted);
+//      metricsRecord.incrMetric("waiting_maps", numWaitingMaps);
+//      metricsRecord.incrMetric("waiting_reduces", numWaitingReduces);
+//
+//      numMapTasksLaunched = 0;
+//      numMapTasksCompleted = 0;
+//      numReduceTasksLaunched = 0;
+//      numReduceTasksCompleted = 0;
+//      numWaitingMaps = 0;
+//      numWaitingReduces = 0;
+//    }
+//    metricsRecord.update();
+//  }
+//
+//  public synchronized void launchMap(TaskAttemptID taskAttemptID) {
+//    ++numMapTasksLaunched;
+//    decWaitingMaps(taskAttemptID.getJobID(), 1);
+//  }
+//
+//  public synchronized void completeMap(TaskAttemptID taskAttemptID) {
+//    ++numMapTasksCompleted;
+//  }
+//
+//  public synchronized void launchReduce(TaskAttemptID taskAttemptID) {
+//    ++numReduceTasksLaunched;
+//    decWaitingReduces(taskAttemptID.getJobID(), 1);
+//  }
+//
+//  public synchronized void completeReduce(TaskAttemptID taskAttemptID) {
+//    ++numReduceTasksCompleted;
+//  }
+//
+//  private synchronized void decWaitingMaps(JobID id, int task) {
+//    numWaitingMaps -= task;
+//  }
+//  
+//  private synchronized void decWaitingReduces(JobID id, int task){
+//    numWaitingReduces -= task;
+//  }
+//
+//}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-mapreduce/src/main/java/org/apache/hadoop/mapred/LocalJobRunnerTez.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/hadoop/mapred/LocalJobRunnerTez.java b/tez-mapreduce/src/main/java/org/apache/hadoop/mapred/LocalJobRunnerTez.java
index ac6d5dd..f59e836 100644
--- a/tez-mapreduce/src/main/java/org/apache/hadoop/mapred/LocalJobRunnerTez.java
+++ b/tez-mapreduce/src/main/java/org/apache/hadoop/mapred/LocalJobRunnerTez.java
@@ -1,878 +1,875 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.mapred;
-
-import java.io.IOException;
-import java.io.OutputStream;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Random;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.ThreadFactory;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicInteger;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.ipc.ProtocolSignature;
-import org.apache.hadoop.mapreduce.Cluster.JobTrackerStatus;
-import org.apache.hadoop.mapreduce.ClusterMetrics;
-import org.apache.hadoop.mapreduce.OutputFormat;
-import org.apache.hadoop.mapreduce.QueueInfo;
-import org.apache.hadoop.mapreduce.TaskCompletionEvent;
-import org.apache.hadoop.mapreduce.TaskTrackerInfo;
-import org.apache.hadoop.mapreduce.TaskType;
-import org.apache.hadoop.mapreduce.protocol.ClientProtocol;
-import org.apache.hadoop.mapreduce.security.token.delegation.DelegationTokenIdentifier;
-import org.apache.hadoop.mapreduce.server.jobtracker.JTConfig;
-import org.apache.hadoop.mapreduce.split.JobSplit.TaskSplitMetaInfo;
-import org.apache.hadoop.mapreduce.split.SplitMetaInfoReader;
-import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl;
-import org.apache.hadoop.mapreduce.v2.LogParams;
-import org.apache.hadoop.security.Credentials;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.security.authorize.AccessControlList;
-import org.apache.hadoop.security.token.Token;
-import org.apache.hadoop.util.ReflectionUtils;
-import org.apache.tez.common.Constants;
-import org.apache.tez.common.ContainerContext;
-import org.apache.tez.common.ContainerTask;
-import org.apache.tez.common.InputSpec;
-import org.apache.tez.common.OutputSpec;
-import org.apache.tez.common.TezEngineTaskContext;
-import org.apache.tez.common.TezJobConfig;
-import org.apache.tez.common.TezTaskUmbilicalProtocol;
-import org.apache.tez.common.counters.TezCounters;
-import org.apache.tez.common.records.ProceedToCompletionResponse;
-import org.apache.tez.dag.api.ProcessorDescriptor;
-import org.apache.tez.dag.records.TezTaskAttemptID;
-import org.apache.tez.engine.api.Task;
-import org.apache.tez.engine.common.task.local.output.TezLocalTaskOutputFiles;
-import org.apache.tez.engine.common.task.local.output.TezTaskOutput;
-import org.apache.tez.engine.lib.oldinput.LocalMergedInput;
-import org.apache.tez.engine.lib.oldoutput.OldLocalOnFileSorterOutput;
-import org.apache.tez.engine.newapi.impl.TezHeartbeatRequest;
-import org.apache.tez.engine.newapi.impl.TezHeartbeatResponse;
-import org.apache.tez.engine.records.OutputContext;
-import org.apache.tez.engine.records.TezTaskDependencyCompletionEventsUpdate;
-import org.apache.tez.engine.runtime.RuntimeUtils;
-import org.apache.tez.mapreduce.hadoop.IDConverter;
-import org.apache.tez.mapreduce.hadoop.mapred.MRCounters;
-import org.apache.tez.mapreduce.input.SimpleInput;
-import org.apache.tez.mapreduce.output.SimpleOutput;
-import org.apache.tez.mapreduce.processor.map.MapProcessor;
-import org.apache.tez.mapreduce.processor.reduce.ReduceProcessor;
-
-import com.google.common.util.concurrent.ThreadFactoryBuilder;
-
-/** Implements MapReduce locally, in-process, for debugging. */
-@InterfaceAudience.Private
-@InterfaceStability.Unstable
-public class LocalJobRunnerTez implements ClientProtocol {
-  public static final Log LOG =
-    LogFactory.getLog(LocalJobRunnerTez.class);
-
-  /** The maximum number of map tasks to run in parallel in LocalJobRunner */
-  public static final String LOCAL_MAX_MAPS =
-    "mapreduce.local.map.tasks.maximum";
-
-  private FileSystem fs;
-  private HashMap<JobID, Job> jobs = new HashMap<JobID, Job>();
-  private JobConf conf;
-  private AtomicInteger map_tasks = new AtomicInteger(0);
-  private int reduce_tasks = 0;
-  final Random rand = new Random();
-
-  private LocalJobRunnerMetricsTez myMetrics = null;
-
-  private static final String jobDir =  "localRunner/";
-
-  private static final TezCounters EMPTY_COUNTERS = new TezCounters();
-
-  public long getProtocolVersion(String protocol, long clientVersion) {
-    return ClientProtocol.versionID;
-  }
-
-  @Override
-  public ProtocolSignature getProtocolSignature(String protocol,
-      long clientVersion, int clientMethodsHash) throws IOException {
-    return ProtocolSignature.getProtocolSignature(
-        this, protocol, clientVersion, clientMethodsHash);
-  }
-
-  private class Job extends Thread implements TezTaskUmbilicalProtocol {
-    // The job directory on the system: JobClient places job configurations here.
-    // This is analogous to JobTracker's system directory.
-    private Path systemJobDir;
-    private Path systemJobFile;
-
-    // The job directory for the task.  Analagous to a task's job directory.
-    private Path localJobDir;
-    private Path localJobFile;
-
-    private JobID id;
-    private JobConf job;
-
-    private int numMapTasks;
-    private float [] partialMapProgress;
-    private TezCounters [] mapCounters;
-    private TezCounters reduceCounters;
-
-    private JobStatus status;
-    private List<TaskAttemptID> mapIds = Collections.synchronizedList(
-        new ArrayList<TaskAttemptID>());
-
-    private JobProfile profile;
-    private FileSystem localFs;
-    boolean killed = false;
-
-    private LocalDistributedCacheManager localDistributedCacheManager;
-
-    public long getProtocolVersion(String protocol, long clientVersion) {
-      return TaskUmbilicalProtocol.versionID;
-    }
-
-    @Override
-    public ProtocolSignature getProtocolSignature(String protocol,
-        long clientVersion, int clientMethodsHash) throws IOException {
-      return ProtocolSignature.getProtocolSignature(
-          this, protocol, clientVersion, clientMethodsHash);
-    }
-
-    public Job(JobID jobid, String jobSubmitDir) throws IOException {
-      this.systemJobDir = new Path(jobSubmitDir);
-      this.systemJobFile = new Path(systemJobDir, "job.xml");
-      this.id = jobid;
-      JobConf conf = new JobConf(systemJobFile);
-      this.localFs = FileSystem.getLocal(conf);
-      this.localJobDir = localFs.makeQualified(conf.getLocalPath(jobDir));
-      this.localJobFile = new Path(this.localJobDir, id + ".xml");
-
-      // Manage the distributed cache.  If there are files to be copied,
-      // this will trigger localFile to be re-written again.
-      localDistributedCacheManager = new LocalDistributedCacheManager();
-      localDistributedCacheManager.setup(conf);
-
-      // Write out configuration file.  Instead of copying it from
-      // systemJobFile, we re-write it, since setup(), above, may have
-      // updated it.
-      OutputStream out = localFs.create(localJobFile);
-      try {
-        conf.writeXml(out);
-      } finally {
-        out.close();
-      }
-      this.job = new JobConf(localJobFile);
-
-      // Job (the current object) is a Thread, so we wrap its class loader.
-      if (localDistributedCacheManager.hasLocalClasspaths()) {
-        setContextClassLoader(localDistributedCacheManager.makeClassLoader(
-                getContextClassLoader()));
-      }
-
-      profile = new JobProfile(job.getUser(), id, systemJobFile.toString(),
-                               "http://localhost:8080/", job.getJobName());
-      status = new JobStatus(id, 0.0f, 0.0f, JobStatus.RUNNING,
-          profile.getUser(), profile.getJobName(), profile.getJobFile(),
-          profile.getURL().toString());
-
-      jobs.put(id, this);
-
-      this.start();
-    }
-
-    /**
-     * A Runnable instance that handles a map task to be run by an executor.
-     */
-    protected class MapTaskRunnable implements Runnable {
-      private final int taskId;
-      private final TaskSplitMetaInfo info;
-      private final JobID jobId;
-      private final JobConf localConf;
-
-      // This is a reference to a shared object passed in by the
-      // external context; this delivers state to the reducers regarding
-      // where to fetch mapper outputs.
-      private final Map<TaskAttemptID, TezTaskOutput> mapOutputFiles;
-
-      public volatile Throwable storedException;
-
-      public MapTaskRunnable(TaskSplitMetaInfo info, int taskId, JobID jobId,
-          Map<TaskAttemptID, TezTaskOutput> mapOutputFiles) {
-        this.info = info;
-        this.taskId = taskId;
-        this.mapOutputFiles = mapOutputFiles;
-        this.jobId = jobId;
-        this.localConf = new JobConf(job);
-      }
-
-      public void run() {
-        try {
-          TaskAttemptID mapId = new TaskAttemptID(new TaskID(
-              jobId, TaskType.MAP, taskId), 0);
-          LOG.info("Starting task: " + mapId);
-          final String user =
-              UserGroupInformation.getCurrentUser().getShortUserName();
-          setupChildMapredLocalDirs(mapId, user, localConf);
-          localConf.setUser(user);
-
-          TezTaskAttemptID tezMapId =
-              IDConverter.fromMRTaskAttemptId(mapId);
-          mapIds.add(mapId);
-          // FIXME invalid task context
-          ProcessorDescriptor mapProcessorDesc = new ProcessorDescriptor(
-                      MapProcessor.class.getName());
-          TezEngineTaskContext taskContext =
-              new TezEngineTaskContext(
-                  tezMapId, user, localConf.getJobName(), "TODO_vertexName",
-                  mapProcessorDesc,
-                  Collections.singletonList(new InputSpec("srcVertex", 0,
-                      SimpleInput.class.getName())),
-                  Collections.singletonList(new OutputSpec("tgtVertex", 0,
-                      OldLocalOnFileSorterOutput.class.getName())));
-
-          TezTaskOutput mapOutput = new TezLocalTaskOutputFiles();
-          mapOutput.setConf(localConf);
-          mapOutputFiles.put(mapId, mapOutput);
-
-          try {
-            map_tasks.getAndIncrement();
-            myMetrics.launchMap(mapId);
-            Task t = RuntimeUtils.createRuntimeTask(taskContext);
-            t.initialize(localConf, null, Job.this);
-            t.run();
-            myMetrics.completeMap(mapId);
-          } finally {
-            map_tasks.getAndDecrement();
-          }
-
-          LOG.info("Finishing task: " + mapId);
-        } catch (Throwable e) {
-          this.storedException = e;
-        }
-      }
-    }
-
-    /**
-     * Create Runnables to encapsulate map tasks for use by the executor
-     * service.
-     * @param taskInfo Info about the map task splits
-     * @param jobId the job id
-     * @param mapOutputFiles a mapping from task attempts to output files
-     * @return a List of Runnables, one per map task.
-     */
-    protected List<MapTaskRunnable> getMapTaskRunnables(
-        TaskSplitMetaInfo [] taskInfo, JobID jobId,
-        Map<TaskAttemptID, TezTaskOutput> mapOutputFiles) {
-
-      int numTasks = 0;
-      ArrayList<MapTaskRunnable> list = new ArrayList<MapTaskRunnable>();
-      for (TaskSplitMetaInfo task : taskInfo) {
-        list.add(new MapTaskRunnable(task, numTasks++, jobId,
-            mapOutputFiles));
-      }
-
-      return list;
-    }
-
-    /**
-     * Initialize the counters that will hold partial-progress from
-     * the various task attempts.
-     * @param numMaps the number of map tasks in this job.
-     */
-    private synchronized void initCounters(int numMaps) {
-      // Initialize state trackers for all map tasks.
-      this.partialMapProgress = new float[numMaps];
-      this.mapCounters = new TezCounters[numMaps];
-      for (int i = 0; i < numMaps; i++) {
-        this.mapCounters[i] = EMPTY_COUNTERS;
-      }
-
-      this.reduceCounters = EMPTY_COUNTERS;
-    }
-
-    /**
-     * Creates the executor service used to run map tasks.
-     *
-     * @param numMapTasks the total number of map tasks to be run
-     * @return an ExecutorService instance that handles map tasks
-     */
-    protected ExecutorService createMapExecutor(int numMapTasks) {
-
-      // Determine the size of the thread pool to use
-      int maxMapThreads = job.getInt(LOCAL_MAX_MAPS, 1);
-      if (maxMapThreads < 1) {
-        throw new IllegalArgumentException(
-            "Configured " + LOCAL_MAX_MAPS + " must be >= 1");
-      }
-      this.numMapTasks = numMapTasks;
-      maxMapThreads = Math.min(maxMapThreads, this.numMapTasks);
-      maxMapThreads = Math.max(maxMapThreads, 1); // In case of no tasks.
-
-      initCounters(this.numMapTasks);
-
-      LOG.debug("Starting thread pool executor.");
-      LOG.debug("Max local threads: " + maxMapThreads);
-      LOG.debug("Map tasks to process: " + this.numMapTasks);
-
-      // Create a new executor service to drain the work queue.
-      ThreadFactory tf = new ThreadFactoryBuilder()
-        .setNameFormat("LocalJobRunner Map Task Executor #%d")
-        .build();
-      ExecutorService executor = Executors.newFixedThreadPool(maxMapThreads, tf);
-
-      return executor;
-    }
-
-    private org.apache.hadoop.mapreduce.OutputCommitter
-    createOutputCommitter(boolean newApiCommitter, JobID jobId, Configuration conf) throws Exception {
-      org.apache.hadoop.mapreduce.OutputCommitter committer = null;
-
-      LOG.info("OutputCommitter set in config "
-          + conf.get("mapred.output.committer.class"));
-
-      if (newApiCommitter) {
-        org.apache.hadoop.mapreduce.TaskID taskId =
-            new org.apache.hadoop.mapreduce.TaskID(jobId, TaskType.MAP, 0);
-        org.apache.hadoop.mapreduce.TaskAttemptID taskAttemptID =
-            new org.apache.hadoop.mapreduce.TaskAttemptID(taskId, 0);
-        org.apache.hadoop.mapreduce.TaskAttemptContext taskContext =
-            new TaskAttemptContextImpl(conf, taskAttemptID);
-        @SuppressWarnings("rawtypes")
-        OutputFormat outputFormat =
-          ReflectionUtils.newInstance(taskContext.getOutputFormatClass(), conf);
-        committer = outputFormat.getOutputCommitter(taskContext);
-      } else {
-        committer = ReflectionUtils.newInstance(conf.getClass(
-            "mapred.output.committer.class", FileOutputCommitter.class,
-            org.apache.hadoop.mapred.OutputCommitter.class), conf);
-      }
-      LOG.info("OutputCommitter is " + committer.getClass().getName());
-      return committer;
-    }
-
-    @Override
-    public void run() {
-      JobID jobId = profile.getJobID();
-      JobContext jContext = new JobContextImpl(job, jobId);
-
-      org.apache.hadoop.mapreduce.OutputCommitter outputCommitter = null;
-      try {
-        outputCommitter = createOutputCommitter(conf.getUseNewMapper(), jobId, conf);
-      } catch (Exception e) {
-        LOG.info("Failed to createOutputCommitter", e);
-        return;
-      }
-
-      try {
-        TaskSplitMetaInfo[] taskSplitMetaInfos =
-          SplitMetaInfoReader.readSplitMetaInfo(jobId, localFs, conf, systemJobDir);
-
-        int numReduceTasks = job.getNumReduceTasks();
-        if (numReduceTasks > 1 || numReduceTasks < 0) {
-          // we only allow 0 or 1 reducer in local mode
-          numReduceTasks = 1;
-          job.setNumReduceTasks(1);
-        }
-        outputCommitter.setupJob(jContext);
-        status.setSetupProgress(1.0f);
-
-        Map<TaskAttemptID, TezTaskOutput> mapOutputFiles =
-            Collections.synchronizedMap(new HashMap<TaskAttemptID, TezTaskOutput>());
-
-        List<MapTaskRunnable> taskRunnables = getMapTaskRunnables(taskSplitMetaInfos,
-            jobId, mapOutputFiles);
-        ExecutorService mapService = createMapExecutor(taskRunnables.size());
-
-        // Start populating the executor with work units.
-        // They may begin running immediately (in other threads).
-        for (Runnable r : taskRunnables) {
-          mapService.submit(r);
-        }
-
-        try {
-          mapService.shutdown(); // Instructs queue to drain.
-
-          // Wait for tasks to finish; do not use a time-based timeout.
-          // (See http://bugs.sun.com/bugdatabase/view_bug.do?bug_id=6179024)
-          LOG.info("Waiting for map tasks");
-          mapService.awaitTermination(Long.MAX_VALUE, TimeUnit.NANOSECONDS);
-        } catch (InterruptedException ie) {
-          // Cancel all threads.
-          mapService.shutdownNow();
-          throw ie;
-        }
-
-        LOG.info("Map task executor complete.");
-
-        // After waiting for the map tasks to complete, if any of these
-        // have thrown an exception, rethrow it now in the main thread context.
-        for (MapTaskRunnable r : taskRunnables) {
-          if (r.storedException != null) {
-            throw new Exception(r.storedException);
-          }
-        }
-
-        TaskAttemptID reduceId = new TaskAttemptID(new TaskID(
-            jobId, TaskType.REDUCE, 0), 0);
-        LOG.info("Starting task: " + reduceId);
-        try {
-          if (numReduceTasks > 0) {
-            String user =
-                UserGroupInformation.getCurrentUser().getShortUserName();
-            JobConf localConf = new JobConf(job);
-            localConf.setUser(user);
-            localConf.set("mapreduce.jobtracker.address", "local");
-            setupChildMapredLocalDirs(reduceId, user, localConf);
-            // FIXME invalid task context
-            ProcessorDescriptor reduceProcessorDesc = new ProcessorDescriptor(
-                ReduceProcessor.class.getName());
-            TezEngineTaskContext taskContext = new TezEngineTaskContext(
-                IDConverter.fromMRTaskAttemptId(reduceId), user,
-                localConf.getJobName(), "TODO_vertexName",
-                reduceProcessorDesc,
-                Collections.singletonList(new InputSpec("TODO_srcVertexName",
-                    mapIds.size(), LocalMergedInput.class.getName())),
-                Collections.singletonList(new OutputSpec("TODO_targetVertex",
-                    0, SimpleOutput.class.getName())));
-
-            // move map output to reduce input
-            for (int i = 0; i < mapIds.size(); i++) {
-              if (!this.isInterrupted()) {
-                TaskAttemptID mapId = mapIds.get(i);
-                if (LOG.isDebugEnabled()) {
-                  LOG.debug("XXX mapId: " + i +
-                      " LOCAL_DIR = " +
-                      mapOutputFiles.get(mapId).getConf().get(
-                          TezJobConfig.LOCAL_DIRS));
-                }
-                Path mapOut = mapOutputFiles.get(mapId).getOutputFile();
-                TezTaskOutput localOutputFile = new TezLocalTaskOutputFiles();
-                localOutputFile.setConf(localConf);
-                Path reduceIn =
-                  localOutputFile.getInputFileForWrite(
-                      IDConverter.fromMRTaskId(mapId.getTaskID()),
-                        localFs.getFileStatus(mapOut).getLen());
-                if (!localFs.mkdirs(reduceIn.getParent())) {
-                  throw new IOException("Mkdirs failed to create "
-                      + reduceIn.getParent().toString());
-                }
-                if (!localFs.rename(mapOut, reduceIn))
-                  throw new IOException("Couldn't rename " + mapOut);
-              } else {
-                throw new InterruptedException();
-              }
-            }
-            if (!this.isInterrupted()) {
-              reduce_tasks += 1;
-              myMetrics.launchReduce(reduceId);
-              Task t = RuntimeUtils.createRuntimeTask(taskContext);
-              t.initialize(localConf, null, Job.this);
-              t.run();
-              myMetrics.completeReduce(reduceId);
-              reduce_tasks -= 1;
-            } else {
-              throw new InterruptedException();
-            }
-          }
-        } finally {
-          for (TezTaskOutput output : mapOutputFiles.values()) {
-            output.removeAll();
-          }
-        }
-        // delete the temporary directory in output directory
-        // FIXME
-        //outputCommitter.commitJob(jContext);
-        status.setCleanupProgress(1.0f);
-
-        if (killed) {
-          this.status.setRunState(JobStatus.KILLED);
-        } else {
-          this.status.setRunState(JobStatus.SUCCEEDED);
-        }
-
-        JobEndNotifier.localRunnerNotification(job, status);
-
-      } catch (Throwable t) {
-        try {
-          outputCommitter.abortJob(jContext,
-            org.apache.hadoop.mapreduce.JobStatus.State.FAILED);
-        } catch (IOException ioe) {
-          LOG.info("Error cleaning up job:" + id);
-        }
-        status.setCleanupProgress(1.0f);
-        if (killed) {
-          this.status.setRunState(JobStatus.KILLED);
-        } else {
-          this.status.setRunState(JobStatus.FAILED);
-        }
-        LOG.warn(id, t);
-
-        JobEndNotifier.localRunnerNotification(job, status);
-
-      } finally {
-        try {
-          fs.delete(systemJobFile.getParent(), true);  // delete submit dir
-          localFs.delete(localJobFile, true);              // delete local copy
-          // Cleanup distributed cache
-          localDistributedCacheManager.close();
-        } catch (IOException e) {
-          LOG.warn("Error cleaning up "+id+": "+e);
-        }
-      }
-    }
-
-    // TaskUmbilicalProtocol methods
-    @Override
-    public ContainerTask getTask(ContainerContext containerContext)
-        throws IOException {
-      return null;
-    }
-
-    /** Return the current values of the counters for this job,
-     * including tasks that are in progress.
-     */
-    public synchronized TezCounters getCurrentCounters() {
-      if (null == mapCounters) {
-        // Counters not yet initialized for job.
-        return EMPTY_COUNTERS;
-      }
-
-      TezCounters current = EMPTY_COUNTERS;
-      for (TezCounters c : mapCounters) {
-        current.incrAllCounters(c);
-      }
-      current.incrAllCounters(reduceCounters);
-      return current;
-    }
-
-    @Override
-    public boolean canCommit(TezTaskAttemptID taskid) throws IOException {
-      return true;
-    }
-
-    @Override
-    public TezTaskDependencyCompletionEventsUpdate
-    getDependentTasksCompletionEvents(
-        int fromEventIdx, int maxEventsToFetch,
-        TezTaskAttemptID reduce) {
-      throw new UnsupportedOperationException(
-          "getDependentTasksCompletionEvents not supported in LocalJobRunner");
-    }
-
-    @Override
-    public void outputReady(TezTaskAttemptID taskAttemptId,
-        OutputContext outputContext) throws IOException {
-      // Ignore for now.
-    }
-
-    @Override
-    public ProceedToCompletionResponse proceedToCompletion(
-        TezTaskAttemptID taskAttemptId) throws IOException {
-      // TODO TEZAM5 Really depends on the module - inmem shuffle or not.
-      return new ProceedToCompletionResponse(true, true);
-    }
-
-    @Override
-    public TezHeartbeatResponse heartbeat(TezHeartbeatRequest request) {
-      // TODO Auto-generated method stub
-      // TODO TODONEWTEZ
-      return null;
-    }
-
-  }
-
-  public LocalJobRunnerTez(Configuration conf) throws IOException {
-    this(new JobConf(conf));
-  }
-
-  @Deprecated
-  public LocalJobRunnerTez(JobConf conf) throws IOException {
-    this.fs = FileSystem.getLocal(conf);
-    this.conf = conf;
-    myMetrics = new LocalJobRunnerMetricsTez(new JobConf(conf));
-  }
-
-  // JobSubmissionProtocol methods
-
-  private static int jobid = 0;
-  public synchronized org.apache.hadoop.mapreduce.JobID getNewJobID() {
-    return new org.apache.hadoop.mapreduce.JobID("local", ++jobid);
-  }
-
-  public org.apache.hadoop.mapreduce.JobStatus submitJob(
-      org.apache.hadoop.mapreduce.JobID jobid, String jobSubmitDir,
-      Credentials credentials) throws IOException {
-    Job job = new Job(JobID.downgrade(jobid), jobSubmitDir);
-    job.job.setCredentials(credentials);
-    return job.status;
-
-  }
-
-  public void killJob(org.apache.hadoop.mapreduce.JobID id) {
-    jobs.get(JobID.downgrade(id)).killed = true;
-    jobs.get(JobID.downgrade(id)).interrupt();
-  }
-
-  public void setJobPriority(org.apache.hadoop.mapreduce.JobID id,
-      String jp) throws IOException {
-    throw new UnsupportedOperationException("Changing job priority " +
-                      "in LocalJobRunner is not supported.");
-  }
-
-  /** Throws {@link UnsupportedOperationException} */
-  public boolean killTask(org.apache.hadoop.mapreduce.TaskAttemptID taskId,
-      boolean shouldFail) throws IOException {
-    throw new UnsupportedOperationException("Killing tasks in " +
-    "LocalJobRunner is not supported");
-  }
-
-  public org.apache.hadoop.mapreduce.TaskReport[] getTaskReports(
-      org.apache.hadoop.mapreduce.JobID id, TaskType type) {
-    return new org.apache.hadoop.mapreduce.TaskReport[0];
-  }
-
-  public org.apache.hadoop.mapreduce.JobStatus getJobStatus(
-      org.apache.hadoop.mapreduce.JobID id) {
-    Job job = jobs.get(JobID.downgrade(id));
-    if(job != null)
-      return job.status;
-    else
-      return null;
-  }
-
-  public org.apache.hadoop.mapreduce.Counters getJobCounters(
-      org.apache.hadoop.mapreduce.JobID id) {
-    Job job = jobs.get(JobID.downgrade(id));
-
-    return new org.apache.hadoop.mapreduce.Counters(
-        new MRCounters(job.getCurrentCounters()));
-  }
-
-  public String getFilesystemName() throws IOException {
-    return fs.getUri().toString();
-  }
-
-  public ClusterMetrics getClusterMetrics() {
-    int numMapTasks = map_tasks.get();
-    return new ClusterMetrics(numMapTasks, reduce_tasks, numMapTasks,
-        reduce_tasks, 0, 0, 1, 1, jobs.size(), 1, 0, 0);
-  }
-
-  public JobTrackerStatus getJobTrackerStatus() {
-    return JobTrackerStatus.RUNNING;
-  }
-
-  public long getTaskTrackerExpiryInterval()
-      throws IOException, InterruptedException {
-    return 0;
-  }
-
-  /**
-   * Get all active trackers in cluster.
-   * @return array of TaskTrackerInfo
-   */
-  public TaskTrackerInfo[] getActiveTrackers()
-      throws IOException, InterruptedException {
-    return null;
-  }
-
-  /**
-   * Get all blacklisted trackers in cluster.
-   * @return array of TaskTrackerInfo
-   */
-  public TaskTrackerInfo[] getBlacklistedTrackers()
-      throws IOException, InterruptedException {
-    return null;
-  }
-
-  public TaskCompletionEvent[] getTaskCompletionEvents(
-      org.apache.hadoop.mapreduce.JobID jobid
-      , int fromEventId, int maxEvents) throws IOException {
-    return TaskCompletionEvent.EMPTY_ARRAY;
-  }
-
-  public org.apache.hadoop.mapreduce.JobStatus[] getAllJobs() {return null;}
-
-
-  /**
-   * Returns the diagnostic information for a particular task in the given job.
-   * To be implemented
-   */
-  public String[] getTaskDiagnostics(
-      org.apache.hadoop.mapreduce.TaskAttemptID taskid) throws IOException{
-	  return new String [0];
-  }
-
-  /**
-   * @see org.apache.hadoop.mapreduce.protocol.ClientProtocol#getSystemDir()
-   */
-  public String getSystemDir() {
-    Path sysDir = new Path(
-      conf.get(JTConfig.JT_SYSTEM_DIR, "/tmp/hadoop/mapred/system"));
-    return fs.makeQualified(sysDir).toString();
-  }
-
-  /**
-   * @see org.apache.hadoop.mapreduce.protocol.ClientProtocol#getQueueAdmins(String)
-   */
-  public AccessControlList getQueueAdmins(String queueName) throws IOException {
-	  return new AccessControlList(" ");// no queue admins for local job runner
-  }
-
-  /**
-   * @see org.apache.hadoop.mapreduce.protocol.ClientProtocol#getStagingAreaDir()
-   */
-  public String getStagingAreaDir() throws IOException {
-    Path stagingRootDir = new Path(conf.get(JTConfig.JT_STAGING_AREA_ROOT,
-        "/tmp/hadoop/mapred/staging"));
-    UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
-    String user;
-    if (ugi != null) {
-      user = ugi.getShortUserName() + rand.nextInt();
-    } else {
-      user = "dummy" + rand.nextInt();
-    }
-    return fs.makeQualified(new Path(stagingRootDir, user+"/.staging")).toString();
-  }
-
-  public String getJobHistoryDir() {
-    return null;
-  }
-
-  @Override
-  public QueueInfo[] getChildQueues(String queueName) throws IOException {
-    return null;
-  }
-
-  @Override
-  public QueueInfo[] getRootQueues() throws IOException {
-    return null;
-  }
-
-  @Override
-  public QueueInfo[] getQueues() throws IOException {
-    return null;
-  }
-
-
-  @Override
-  public QueueInfo getQueue(String queue) throws IOException {
-    return null;
-  }
-
-  @Override
-  public org.apache.hadoop.mapreduce.QueueAclsInfo[]
-      getQueueAclsForCurrentUser() throws IOException{
-    return null;
-  }
-
-  /**
-   * Set the max number of map tasks to run concurrently in the LocalJobRunner.
-   * @param job the job to configure
-   * @param maxMaps the maximum number of map tasks to allow.
-   */
-  public static void setLocalMaxRunningMaps(
-      org.apache.hadoop.mapreduce.JobContext job,
-      int maxMaps) {
-    job.getConfiguration().setInt(LOCAL_MAX_MAPS, maxMaps);
-  }
-
-  /**
-   * @return the max number of map tasks to run concurrently in the
-   * LocalJobRunner.
-   */
-  public static int getLocalMaxRunningMaps(
-      org.apache.hadoop.mapreduce.JobContext job) {
-    return job.getConfiguration().getInt(LOCAL_MAX_MAPS, 1);
-  }
-
-  @Override
-  public void cancelDelegationToken(Token<DelegationTokenIdentifier> token
-                                       ) throws IOException,
-                                                InterruptedException {
-  }
-
-  @Override
-  public Token<DelegationTokenIdentifier>
-     getDelegationToken(Text renewer) throws IOException, InterruptedException {
-    return null;
-  }
-
-  @Override
-  public long renewDelegationToken(Token<DelegationTokenIdentifier> token
-                                      ) throws IOException,InterruptedException{
-    return 0;
-  }
-
-  @Override
-  public LogParams getLogFileParams(org.apache.hadoop.mapreduce.JobID jobID,
-      org.apache.hadoop.mapreduce.TaskAttemptID taskAttemptID)
-      throws IOException, InterruptedException {
-    throw new UnsupportedOperationException("Not supported");
-  }
-
-  static void setupChildMapredLocalDirs(
-      TaskAttemptID taskAttemptID, String user, JobConf conf) {
-    String[] localDirs =
-        conf.getTrimmedStrings(
-            TezJobConfig.LOCAL_DIRS, TezJobConfig.DEFAULT_LOCAL_DIRS);
-    String jobId = taskAttemptID.getJobID().toString();
-    String taskId = taskAttemptID.getTaskID().toString();
-    boolean isCleanup = false;
-    StringBuffer childMapredLocalDir =
-        new StringBuffer(localDirs[0] + Path.SEPARATOR
-            + getLocalTaskDir(user, jobId, taskId, isCleanup));
-    for (int i = 1; i < localDirs.length; i++) {
-      childMapredLocalDir.append("," + localDirs[i] + Path.SEPARATOR
-          + getLocalTaskDir(user, jobId, taskId, isCleanup));
-    }
-    LOG.info(TezJobConfig.LOCAL_DIRS + " for child : " + taskAttemptID +
-        " is " + childMapredLocalDir);
-    conf.set(TezJobConfig.LOCAL_DIRS, childMapredLocalDir.toString());
-    conf.setClass(Constants.TEZ_ENGINE_TASK_OUTPUT_MANAGER,
-        TezLocalTaskOutputFiles.class, TezTaskOutput.class);
-  }
-
-  static final String TASK_CLEANUP_SUFFIX = ".cleanup";
-  static final String SUBDIR = jobDir;
-  static final String JOBCACHE = "jobcache";
-
-  static String getLocalTaskDir(String user, String jobid, String taskid,
-      boolean isCleanupAttempt) {
-    String taskDir = SUBDIR + Path.SEPARATOR + user + Path.SEPARATOR + JOBCACHE
-      + Path.SEPARATOR + jobid + Path.SEPARATOR + taskid;
-    if (isCleanupAttempt) {
-      taskDir = taskDir + TASK_CLEANUP_SUFFIX;
-    }
-    return taskDir;
-  }
-
-
-}
+///**
+// * Licensed to the Apache Software Foundation (ASF) under one
+// * or more contributor license agreements.  See the NOTICE file
+// * distributed with this work for additional information
+// * regarding copyright ownership.  The ASF licenses this file
+// * to you under the Apache License, Version 2.0 (the
+// * "License"); you may not use this file except in compliance
+// * with the License.  You may obtain a copy of the License at
+// *
+// *     http://www.apache.org/licenses/LICENSE-2.0
+// *
+// * Unless required by applicable law or agreed to in writing, software
+// * distributed under the License is distributed on an "AS IS" BASIS,
+// * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// * See the License for the specific language governing permissions and
+// * limitations under the License.
+// */
+//
+//package org.apache.hadoop.mapred;
+//
+//import java.io.IOException;
+//import java.io.OutputStream;
+//import java.util.ArrayList;
+//import java.util.Collections;
+//import java.util.HashMap;
+//import java.util.List;
+//import java.util.Map;
+//import java.util.Random;
+//import java.util.concurrent.ExecutorService;
+//import java.util.concurrent.Executors;
+//import java.util.concurrent.ThreadFactory;
+//import java.util.concurrent.TimeUnit;
+//import java.util.concurrent.atomic.AtomicInteger;
+//
+//import org.apache.commons.logging.Log;
+//import org.apache.commons.logging.LogFactory;
+//import org.apache.hadoop.classification.InterfaceAudience;
+//import org.apache.hadoop.classification.InterfaceStability;
+//import org.apache.hadoop.conf.Configuration;
+//import org.apache.hadoop.fs.FileSystem;
+//import org.apache.hadoop.fs.Path;
+//import org.apache.hadoop.io.Text;
+//import org.apache.hadoop.ipc.ProtocolSignature;
+//import org.apache.hadoop.mapreduce.Cluster.JobTrackerStatus;
+//import org.apache.hadoop.mapreduce.ClusterMetrics;
+//import org.apache.hadoop.mapreduce.OutputFormat;
+//import org.apache.hadoop.mapreduce.QueueInfo;
+//import org.apache.hadoop.mapreduce.TaskCompletionEvent;
+//import org.apache.hadoop.mapreduce.TaskTrackerInfo;
+//import org.apache.hadoop.mapreduce.TaskType;
+//import org.apache.hadoop.mapreduce.protocol.ClientProtocol;
+//import org.apache.hadoop.mapreduce.security.token.delegation.DelegationTokenIdentifier;
+//import org.apache.hadoop.mapreduce.server.jobtracker.JTConfig;
+//import org.apache.hadoop.mapreduce.split.JobSplit.TaskSplitMetaInfo;
+//import org.apache.hadoop.mapreduce.split.SplitMetaInfoReader;
+//import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl;
+//import org.apache.hadoop.mapreduce.v2.LogParams;
+//import org.apache.hadoop.security.Credentials;
+//import org.apache.hadoop.security.UserGroupInformation;
+//import org.apache.hadoop.security.authorize.AccessControlList;
+//import org.apache.hadoop.security.token.Token;
+//import org.apache.hadoop.util.ReflectionUtils;
+//import org.apache.tez.common.Constants;
+//import org.apache.tez.common.ContainerContext;
+//import org.apache.tez.common.ContainerTask;
+//import org.apache.tez.common.InputSpec;
+//import org.apache.tez.common.OutputSpec;
+//import org.apache.tez.common.TezEngineTaskContext;
+//import org.apache.tez.common.TezJobConfig;
+//import org.apache.tez.common.TezTaskUmbilicalProtocol;
+//import org.apache.tez.common.counters.TezCounters;
+//import org.apache.tez.common.records.ProceedToCompletionResponse;
+//import org.apache.tez.dag.api.ProcessorDescriptor;
+//import org.apache.tez.dag.records.TezTaskAttemptID;
+//import org.apache.tez.engine.api.Task;
+//import org.apache.tez.engine.api.impl.TezHeartbeatRequest;
+//import org.apache.tez.engine.api.impl.TezHeartbeatResponse;
+//import org.apache.tez.engine.common.task.local.output.TezLocalTaskOutputFiles;
+//import org.apache.tez.engine.common.task.local.output.TezTaskOutput;
+//import org.apache.tez.engine.lib.input.LocalMergedInput;
+//import org.apache.tez.engine.lib.output.LocalOnFileSorterOutput;
+//import org.apache.tez.engine.records.OutputContext;
+//import org.apache.tez.engine.records.TezTaskDependencyCompletionEventsUpdate;
+//import org.apache.tez.mapreduce.hadoop.IDConverter;
+//import org.apache.tez.mapreduce.hadoop.mapred.MRCounters;
+//import org.apache.tez.mapreduce.input.SimpleInput;
+//import org.apache.tez.mapreduce.output.SimpleOutput;
+//import org.apache.tez.mapreduce.processor.map.MapProcessor;
+//import org.apache.tez.mapreduce.processor.reduce.ReduceProcessor;
+//
+//import com.google.common.util.concurrent.ThreadFactoryBuilder;
+//
+///** Implements MapReduce locally, in-process, for debugging. */
+//@InterfaceAudience.Private
+//@InterfaceStability.Unstable
+//public class LocalJobRunnerTez implements ClientProtocol {
+//  public static final Log LOG =
+//    LogFactory.getLog(LocalJobRunnerTez.class);
+//
+//  /** The maximum number of map tasks to run in parallel in LocalJobRunner */
+//  public static final String LOCAL_MAX_MAPS =
+//    "mapreduce.local.map.tasks.maximum";
+//
+//  private FileSystem fs;
+//  private HashMap<JobID, Job> jobs = new HashMap<JobID, Job>();
+//  private JobConf conf;
+//  private AtomicInteger map_tasks = new AtomicInteger(0);
+//  private int reduce_tasks = 0;
+//  final Random rand = new Random();
+//
+//  private LocalJobRunnerMetricsTez myMetrics = null;
+//
+//  private static final String jobDir =  "localRunner/";
+//
+//  private static final TezCounters EMPTY_COUNTERS = new TezCounters();
+//
+//  public long getProtocolVersion(String protocol, long clientVersion) {
+//    return ClientProtocol.versionID;
+//  }
+//
+//  @Override
+//  public ProtocolSignature getProtocolSignature(String protocol,
+//      long clientVersion, int clientMethodsHash) throws IOException {
+//    return ProtocolSignature.getProtocolSignature(
+//        this, protocol, clientVersion, clientMethodsHash);
+//  }
+//
+//  private class Job extends Thread implements TezTaskUmbilicalProtocol {
+//    // The job directory on the system: JobClient places job configurations here.
+//    // This is analogous to JobTracker's system directory.
+//    private Path systemJobDir;
+//    private Path systemJobFile;
+//
+//    // The job directory for the task.  Analagous to a task's job directory.
+//    private Path localJobDir;
+//    private Path localJobFile;
+//
+//    private JobID id;
+//    private JobConf job;
+//
+//    private int numMapTasks;
+//    private float [] partialMapProgress;
+//    private TezCounters [] mapCounters;
+//    private TezCounters reduceCounters;
+//
+//    private JobStatus status;
+//    private List<TaskAttemptID> mapIds = Collections.synchronizedList(
+//        new ArrayList<TaskAttemptID>());
+//
+//    private JobProfile profile;
+//    private FileSystem localFs;
+//    boolean killed = false;
+//
+//    private LocalDistributedCacheManager localDistributedCacheManager;
+//
+//    public long getProtocolVersion(String protocol, long clientVersion) {
+//      return TaskUmbilicalProtocol.versionID;
+//    }
+//
+//    @Override
+//    public ProtocolSignature getProtocolSignature(String protocol,
+//        long clientVersion, int clientMethodsHash) throws IOException {
+//      return ProtocolSignature.getProtocolSignature(
+//          this, protocol, clientVersion, clientMethodsHash);
+//    }
+//
+//    public Job(JobID jobid, String jobSubmitDir) throws IOException {
+//      this.systemJobDir = new Path(jobSubmitDir);
+//      this.systemJobFile = new Path(systemJobDir, "job.xml");
+//      this.id = jobid;
+//      JobConf conf = new JobConf(systemJobFile);
+//      this.localFs = FileSystem.getLocal(conf);
+//      this.localJobDir = localFs.makeQualified(conf.getLocalPath(jobDir));
+//      this.localJobFile = new Path(this.localJobDir, id + ".xml");
+//
+//      // Manage the distributed cache.  If there are files to be copied,
+//      // this will trigger localFile to be re-written again.
+//      localDistributedCacheManager = new LocalDistributedCacheManager();
+//      localDistributedCacheManager.setup(conf);
+//
+//      // Write out configuration file.  Instead of copying it from
+//      // systemJobFile, we re-write it, since setup(), above, may have
+//      // updated it.
+//      OutputStream out = localFs.create(localJobFile);
+//      try {
+//        conf.writeXml(out);
+//      } finally {
+//        out.close();
+//      }
+//      this.job = new JobConf(localJobFile);
+//
+//      // Job (the current object) is a Thread, so we wrap its class loader.
+//      if (localDistributedCacheManager.hasLocalClasspaths()) {
+//        setContextClassLoader(localDistributedCacheManager.makeClassLoader(
+//                getContextClassLoader()));
+//      }
+//
+//      profile = new JobProfile(job.getUser(), id, systemJobFile.toString(),
+//                               "http://localhost:8080/", job.getJobName());
+//      status = new JobStatus(id, 0.0f, 0.0f, JobStatus.RUNNING,
+//          profile.getUser(), profile.getJobName(), profile.getJobFile(),
+//          profile.getURL().toString());
+//
+//      jobs.put(id, this);
+//
+//      this.start();
+//    }
+//
+//    /**
+//     * A Runnable instance that handles a map task to be run by an executor.
+//     */
+//    protected class MapTaskRunnable implements Runnable {
+//      private final int taskId;
+//      private final TaskSplitMetaInfo info;
+//      private final JobID jobId;
+//      private final JobConf localConf;
+//
+//      // This is a reference to a shared object passed in by the
+//      // external context; this delivers state to the reducers regarding
+//      // where to fetch mapper outputs.
+//      private final Map<TaskAttemptID, TezTaskOutput> mapOutputFiles;
+//
+//      public volatile Throwable storedException;
+//
+//      public MapTaskRunnable(TaskSplitMetaInfo info, int taskId, JobID jobId,
+//          Map<TaskAttemptID, TezTaskOutput> mapOutputFiles) {
+//        this.info = info;
+//        this.taskId = taskId;
+//        this.mapOutputFiles = mapOutputFiles;
+//        this.jobId = jobId;
+//        this.localConf = new JobConf(job);
+//      }
+//
+//      public void run() {
+//        try {
+//          TaskAttemptID mapId = new TaskAttemptID(new TaskID(
+//              jobId, TaskType.MAP, taskId), 0);
+//          LOG.info("Starting task: " + mapId);
+//          final String user =
+//              UserGroupInformation.getCurrentUser().getShortUserName();
+//          setupChildMapredLocalDirs(mapId, user, localConf);
+//          localConf.setUser(user);
+//
+//          TezTaskAttemptID tezMapId =
+//              IDConverter.fromMRTaskAttemptId(mapId);
+//          mapIds.add(mapId);
+//          // FIXME invalid task context
+//          ProcessorDescriptor mapProcessorDesc = new ProcessorDescriptor(
+//                      MapProcessor.class.getName());
+//          TezEngineTaskContext taskContext =
+//              new TezEngineTaskContext(
+//                  tezMapId, user, localConf.getJobName(), "TODO_vertexName",
+//                  mapProcessorDesc,
+//                  Collections.singletonList(new InputSpec("srcVertex", 0,
+//                      SimpleInput.class.getName())),
+//                  Collections.singletonList(new OutputSpec("tgtVertex", 0,
+//                      LocalOnFileSorterOutput.class.getName())));
+//
+//          TezTaskOutput mapOutput = new TezLocalTaskOutputFiles(localConf, "TODO_uniqueId");
+//          mapOutputFiles.put(mapId, mapOutput);
+//
+//          try {
+//            map_tasks.getAndIncrement();
+//            myMetrics.launchMap(mapId);
+//            Task t = RuntimeUtils.createRuntimeTask(taskContext);
+//            t.initialize(localConf, null, Job.this);
+//            t.run();
+//            myMetrics.completeMap(mapId);
+//          } finally {
+//            map_tasks.getAndDecrement();
+//          }
+//
+//          LOG.info("Finishing task: " + mapId);
+//        } catch (Throwable e) {
+//          this.storedException = e;
+//        }
+//      }
+//    }
+//
+//    /**
+//     * Create Runnables to encapsulate map tasks for use by the executor
+//     * service.
+//     * @param taskInfo Info about the map task splits
+//     * @param jobId the job id
+//     * @param mapOutputFiles a mapping from task attempts to output files
+//     * @return a List of Runnables, one per map task.
+//     */
+//    protected List<MapTaskRunnable> getMapTaskRunnables(
+//        TaskSplitMetaInfo [] taskInfo, JobID jobId,
+//        Map<TaskAttemptID, TezTaskOutput> mapOutputFiles) {
+//
+//      int numTasks = 0;
+//      ArrayList<MapTaskRunnable> list = new ArrayList<MapTaskRunnable>();
+//      for (TaskSplitMetaInfo task : taskInfo) {
+//        list.add(new MapTaskRunnable(task, numTasks++, jobId,
+//            mapOutputFiles));
+//      }
+//
+//      return list;
+//    }
+//
+//    /**
+//     * Initialize the counters that will hold partial-progress from
+//     * the various task attempts.
+//     * @param numMaps the number of map tasks in this job.
+//     */
+//    private synchronized void initCounters(int numMaps) {
+//      // Initialize state trackers for all map tasks.
+//      this.partialMapProgress = new float[numMaps];
+//      this.mapCounters = new TezCounters[numMaps];
+//      for (int i = 0; i < numMaps; i++) {
+//        this.mapCounters[i] = EMPTY_COUNTERS;
+//      }
+//
+//      this.reduceCounters = EMPTY_COUNTERS;
+//    }
+//
+//    /**
+//     * Creates the executor service used to run map tasks.
+//     *
+//     * @param numMapTasks the total number of map tasks to be run
+//     * @return an ExecutorService instance that handles map tasks
+//     */
+//    protected ExecutorService createMapExecutor(int numMapTasks) {
+//
+//      // Determine the size of the thread pool to use
+//      int maxMapThreads = job.getInt(LOCAL_MAX_MAPS, 1);
+//      if (maxMapThreads < 1) {
+//        throw new IllegalArgumentException(
+//            "Configured " + LOCAL_MAX_MAPS + " must be >= 1");
+//      }
+//      this.numMapTasks = numMapTasks;
+//      maxMapThreads = Math.min(maxMapThreads, this.numMapTasks);
+//      maxMapThreads = Math.max(maxMapThreads, 1); // In case of no tasks.
+//
+//      initCounters(this.numMapTasks);
+//
+//      LOG.debug("Starting thread pool executor.");
+//      LOG.debug("Max local threads: " + maxMapThreads);
+//      LOG.debug("Map tasks to process: " + this.numMapTasks);
+//
+//      // Create a new executor service to drain the work queue.
+//      ThreadFactory tf = new ThreadFactoryBuilder()
+//        .setNameFormat("LocalJobRunner Map Task Executor #%d")
+//        .build();
+//      ExecutorService executor = Executors.newFixedThreadPool(maxMapThreads, tf);
+//
+//      return executor;
+//    }
+//
+//    private org.apache.hadoop.mapreduce.OutputCommitter
+//    createOutputCommitter(boolean newApiCommitter, JobID jobId, Configuration conf) throws Exception {
+//      org.apache.hadoop.mapreduce.OutputCommitter committer = null;
+//
+//      LOG.info("OutputCommitter set in config "
+//          + conf.get("mapred.output.committer.class"));
+//
+//      if (newApiCommitter) {
+//        org.apache.hadoop.mapreduce.TaskID taskId =
+//            new org.apache.hadoop.mapreduce.TaskID(jobId, TaskType.MAP, 0);
+//        org.apache.hadoop.mapreduce.TaskAttemptID taskAttemptID =
+//            new org.apache.hadoop.mapreduce.TaskAttemptID(taskId, 0);
+//        org.apache.hadoop.mapreduce.TaskAttemptContext taskContext =
+//            new TaskAttemptContextImpl(conf, taskAttemptID);
+//        @SuppressWarnings("rawtypes")
+//        OutputFormat outputFormat =
+//          ReflectionUtils.newInstance(taskContext.getOutputFormatClass(), conf);
+//        committer = outputFormat.getOutputCommitter(taskContext);
+//      } else {
+//        committer = ReflectionUtils.newInstance(conf.getClass(
+//            "mapred.output.committer.class", FileOutputCommitter.class,
+//            org.apache.hadoop.mapred.OutputCommitter.class), conf);
+//      }
+//      LOG.info("OutputCommitter is " + committer.getClass().getName());
+//      return committer;
+//    }
+//
+//    @Override
+//    public void run() {
+//      JobID jobId = profile.getJobID();
+//      JobContext jContext = new JobContextImpl(job, jobId);
+//
+//      org.apache.hadoop.mapreduce.OutputCommitter outputCommitter = null;
+//      try {
+//        outputCommitter = createOutputCommitter(conf.getUseNewMapper(), jobId, conf);
+//      } catch (Exception e) {
+//        LOG.info("Failed to createOutputCommitter", e);
+//        return;
+//      }
+//
+//      try {
+//        TaskSplitMetaInfo[] taskSplitMetaInfos =
+//          SplitMetaInfoReader.readSplitMetaInfo(jobId, localFs, conf, systemJobDir);
+//
+//        int numReduceTasks = job.getNumReduceTasks();
+//        if (numReduceTasks > 1 || numReduceTasks < 0) {
+//          // we only allow 0 or 1 reducer in local mode
+//          numReduceTasks = 1;
+//          job.setNumReduceTasks(1);
+//        }
+//        outputCommitter.setupJob(jContext);
+//        status.setSetupProgress(1.0f);
+//
+//        Map<TaskAttemptID, TezTaskOutput> mapOutputFiles =
+//            Collections.synchronizedMap(new HashMap<TaskAttemptID, TezTaskOutput>());
+//
+//        List<MapTaskRunnable> taskRunnables = getMapTaskRunnables(taskSplitMetaInfos,
+//            jobId, mapOutputFiles);
+//        ExecutorService mapService = createMapExecutor(taskRunnables.size());
+//
+//        // Start populating the executor with work units.
+//        // They may begin running immediately (in other threads).
+//        for (Runnable r : taskRunnables) {
+//          mapService.submit(r);
+//        }
+//
+//        try {
+//          mapService.shutdown(); // Instructs queue to drain.
+//
+//          // Wait for tasks to finish; do not use a time-based timeout.
+//          // (See http://bugs.sun.com/bugdatabase/view_bug.do?bug_id=6179024)
+//          LOG.info("Waiting for map tasks");
+//          mapService.awaitTermination(Long.MAX_VALUE, TimeUnit.NANOSECONDS);
+//        } catch (InterruptedException ie) {
+//          // Cancel all threads.
+//          mapService.shutdownNow();
+//          throw ie;
+//        }
+//
+//        LOG.info("Map task executor complete.");
+//
+//        // After waiting for the map tasks to complete, if any of these
+//        // have thrown an exception, rethrow it now in the main thread context.
+//        for (MapTaskRunnable r : taskRunnables) {
+//          if (r.storedException != null) {
+//            throw new Exception(r.storedException);
+//          }
+//        }
+//
+//        TaskAttemptID reduceId = new TaskAttemptID(new TaskID(
+//            jobId, TaskType.REDUCE, 0), 0);
+//        LOG.info("Starting task: " + reduceId);
+//        try {
+//          if (numReduceTasks > 0) {
+//            String user =
+//                UserGroupInformation.getCurrentUser().getShortUserName();
+//            JobConf localConf = new JobConf(job);
+//            localConf.setUser(user);
+//            localConf.set("mapreduce.jobtracker.address", "local");
+//            setupChildMapredLocalDirs(reduceId, user, localConf);
+//            // FIXME invalid task context
+//            ProcessorDescriptor reduceProcessorDesc = new ProcessorDescriptor(
+//                ReduceProcessor.class.getName());
+//            TezEngineTaskContext taskContext = new TezEngineTaskContext(
+//                IDConverter.fromMRTaskAttemptId(reduceId), user,
+//                localConf.getJobName(), "TODO_vertexName",
+//                reduceProcessorDesc,
+//                Collections.singletonList(new InputSpec("TODO_srcVertexName",
+//                    mapIds.size(), LocalMergedInput.class.getName())),
+//                Collections.singletonList(new OutputSpec("TODO_targetVertex",
+//                    0, SimpleOutput.class.getName())));
+//
+//            // move map output to reduce input
+//            for (int i = 0; i < mapIds.size(); i++) {
+//              if (!this.isInterrupted()) {
+//                TaskAttemptID mapId = mapIds.get(i);
+//                if (LOG.isDebugEnabled()) {
+//                  // TODO NEWTEZ Fix this logging.
+////                  LOG.debug("XXX mapId: " + i +
+////                      " LOCAL_DIR = " +
+////                      mapOutputFiles.get(mapId).getConf().get(
+////                          TezJobConfig.LOCAL_DIRS));
+//                }
+//                Path mapOut = mapOutputFiles.get(mapId).getOutputFile();
+//                TezTaskOutput localOutputFile = new TezLocalTaskOutputFiles(localConf, "TODO_uniqueId");
+//                Path reduceIn =
+//                  localOutputFile.getInputFileForWrite(
+//                      mapId.getTaskID().getId(), localFs.getFileStatus(mapOut).getLen());
+//                if (!localFs.mkdirs(reduceIn.getParent())) {
+//                  throw new IOException("Mkdirs failed to create "
+//                      + reduceIn.getParent().toString());
+//                }
+//                if (!localFs.rename(mapOut, reduceIn))
+//                  throw new IOException("Couldn't rename " + mapOut);
+//              } else {
+//                throw new InterruptedException();
+//              }
+//            }
+//            if (!this.isInterrupted()) {
+//              reduce_tasks += 1;
+//              myMetrics.launchReduce(reduceId);
+//              Task t = RuntimeUtils.createRuntimeTask(taskContext);
+//              t.initialize(localConf, null, Job.this);
+//              t.run();
+//              myMetrics.completeReduce(reduceId);
+//              reduce_tasks -= 1;
+//            } else {
+//              throw new InterruptedException();
+//            }
+//          }
+//        } finally {
+//          for (TezTaskOutput output : mapOutputFiles.values()) {
+//            output.removeAll();
+//          }
+//        }
+//        // delete the temporary directory in output directory
+//        // FIXME
+//        //outputCommitter.commitJob(jContext);
+//        status.setCleanupProgress(1.0f);
+//
+//        if (killed) {
+//          this.status.setRunState(JobStatus.KILLED);
+//        } else {
+//          this.status.setRunState(JobStatus.SUCCEEDED);
+//        }
+//
+//        JobEndNotifier.localRunnerNotification(job, status);
+//
+//      } catch (Throwable t) {
+//        try {
+//          outputCommitter.abortJob(jContext,
+//            org.apache.hadoop.mapreduce.JobStatus.State.FAILED);
+//        } catch (IOException ioe) {
+//          LOG.info("Error cleaning up job:" + id);
+//        }
+//        status.setCleanupProgress(1.0f);
+//        if (killed) {
+//          this.status.setRunState(JobStatus.KILLED);
+//        } else {
+//          this.status.setRunState(JobStatus.FAILED);
+//        }
+//        LOG.warn(id, t);
+//
+//        JobEndNotifier.localRunnerNotification(job, status);
+//
+//      } finally {
+//        try {
+//          fs.delete(systemJobFile.getParent(), true);  // delete submit dir
+//          localFs.delete(localJobFile, true);              // delete local copy
+//          // Cleanup distributed cache
+//          localDistributedCacheManager.close();
+//        } catch (IOException e) {
+//          LOG.warn("Error cleaning up "+id+": "+e);
+//        }
+//      }
+//    }
+//
+//    // TaskUmbilicalProtocol methods
+//    @Override
+//    public ContainerTask getTask(ContainerContext containerContext)
+//        throws IOException {
+//      return null;
+//    }
+//
+//    /** Return the current values of the counters for this job,
+//     * including tasks that are in progress.
+//     */
+//    public synchronized TezCounters getCurrentCounters() {
+//      if (null == mapCounters) {
+//        // Counters not yet initialized for job.
+//        return EMPTY_COUNTERS;
+//      }
+//
+//      TezCounters current = EMPTY_COUNTERS;
+//      for (TezCounters c : mapCounters) {
+//        current.incrAllCounters(c);
+//      }
+//      current.incrAllCounters(reduceCounters);
+//      return current;
+//    }
+//
+//    @Override
+//    public boolean canCommit(TezTaskAttemptID taskid) throws IOException {
+//      return true;
+//    }
+//
+//    @Override
+//    public TezTaskDependencyCompletionEventsUpdate
+//    getDependentTasksCompletionEvents(
+//        int fromEventIdx, int maxEventsToFetch,
+//        TezTaskAttemptID reduce) {
+//      throw new UnsupportedOperationException(
+//          "getDependentTasksCompletionEvents not supported in LocalJobRunner");
+//    }
+//
+//    @Override
+//    public void outputReady(TezTaskAttemptID taskAttemptId,
+//        OutputContext outputContext) throws IOException {
+//      // Ignore for now.
+//    }
+//
+//    @Override
+//    public ProceedToCompletionResponse proceedToCompletion(
+//        TezTaskAttemptID taskAttemptId) throws IOException {
+//      // TODO TEZAM5 Really depends on the module - inmem shuffle or not.
+//      return new ProceedToCompletionResponse(true, true);
+//    }
+//
+//    @Override
+//    public TezHeartbeatResponse heartbeat(TezHeartbeatRequest request) {
+//      // TODO Auto-generated method stub
+//      // TODO TODONEWTEZ
+//      return null;
+//    }
+//
+//  }
+//
+//  public LocalJobRunnerTez(Configuration conf) throws IOException {
+//    this(new JobConf(conf));
+//  }
+//
+//  @Deprecated
+//  public LocalJobRunnerTez(JobConf conf) throws IOException {
+//    this.fs = FileSystem.getLocal(conf);
+//    this.conf = conf;
+//    myMetrics = new LocalJobRunnerMetricsTez(new JobConf(conf));
+//  }
+//
+//  // JobSubmissionProtocol methods
+//
+//  private static int jobid = 0;
+//  public synchronized org.apache.hadoop.mapreduce.JobID getNewJobID() {
+//    return new org.apache.hadoop.mapreduce.JobID("local", ++jobid);
+//  }
+//
+//  public org.apache.hadoop.mapreduce.JobStatus submitJob(
+//      org.apache.hadoop.mapreduce.JobID jobid, String jobSubmitDir,
+//      Credentials credentials) throws IOException {
+//    Job job = new Job(JobID.downgrade(jobid), jobSubmitDir);
+//    job.job.setCredentials(credentials);
+//    return job.status;
+//
+//  }
+//
+//  public void killJob(org.apache.hadoop.mapreduce.JobID id) {
+//    jobs.get(JobID.downgrade(id)).killed = true;
+//    jobs.get(JobID.downgrade(id)).interrupt();
+//  }
+//
+//  public void setJobPriority(org.apache.hadoop.mapreduce.JobID id,
+//      String jp) throws IOException {
+//    throw new UnsupportedOperationException("Changing job priority " +
+//                      "in LocalJobRunner is not supported.");
+//  }
+//
+//  /** Throws {@link UnsupportedOperationException} */
+//  public boolean killTask(org.apache.hadoop.mapreduce.TaskAttemptID taskId,
+//      boolean shouldFail) throws IOException {
+//    throw new UnsupportedOperationException("Killing tasks in " +
+//    "LocalJobRunner is not supported");
+//  }
+//
+//  public org.apache.hadoop.mapreduce.TaskReport[] getTaskReports(
+//      org.apache.hadoop.mapreduce.JobID id, TaskType type) {
+//    return new org.apache.hadoop.mapreduce.TaskReport[0];
+//  }
+//
+//  public org.apache.hadoop.mapreduce.JobStatus getJobStatus(
+//      org.apache.hadoop.mapreduce.JobID id) {
+//    Job job = jobs.get(JobID.downgrade(id));
+//    if(job != null)
+//      return job.status;
+//    else
+//      return null;
+//  }
+//
+//  public org.apache.hadoop.mapreduce.Counters getJobCounters(
+//      org.apache.hadoop.mapreduce.JobID id) {
+//    Job job = jobs.get(JobID.downgrade(id));
+//
+//    return new org.apache.hadoop.mapreduce.Counters(
+//        new MRCounters(job.getCurrentCounters()));
+//  }
+//
+//  public String getFilesystemName() throws IOException {
+//    return fs.getUri().toString();
+//  }
+//
+//  public ClusterMetrics getClusterMetrics() {
+//    int numMapTasks = map_tasks.get();
+//    return new ClusterMetrics(numMapTasks, reduce_tasks, numMapTasks,
+//        reduce_tasks, 0, 0, 1, 1, jobs.size(), 1, 0, 0);
+//  }
+//
+//  public JobTrackerStatus getJobTrackerStatus() {
+//    return JobTrackerStatus.RUNNING;
+//  }
+//
+//  public long getTaskTrackerExpiryInterval()
+//      throws IOException, InterruptedException {
+//    return 0;
+//  }
+//
+//  /**
+//   * Get all active trackers in cluster.
+//   * @return array of TaskTrackerInfo
+//   */
+//  public TaskTrackerInfo[] getActiveTrackers()
+//      throws IOException, InterruptedException {
+//    return null;
+//  }
+//
+//  /**
+//   * Get all blacklisted trackers in cluster.
+//   * @return array of TaskTrackerInfo
+//   */
+//  public TaskTrackerInfo[] getBlacklistedTrackers()
+//      throws IOException, InterruptedException {
+//    return null;
+//  }
+//
+//  public TaskCompletionEvent[] getTaskCompletionEvents(
+//      org.apache.hadoop.mapreduce.JobID jobid
+//      , int fromEventId, int maxEvents) throws IOException {
+//    return TaskCompletionEvent.EMPTY_ARRAY;
+//  }
+//
+//  public org.apache.hadoop.mapreduce.JobStatus[] getAllJobs() {return null;}
+//
+//
+//  /**
+//   * Returns the diagnostic information for a particular task in the given job.
+//   * To be implemented
+//   */
+//  public String[] getTaskDiagnostics(
+//      org.apache.hadoop.mapreduce.TaskAttemptID taskid) throws IOException{
+//	  return new String [0];
+//  }
+//
+//  /**
+//   * @see org.apache.hadoop.mapreduce.protocol.ClientProtocol#getSystemDir()
+//   */
+//  public String getSystemDir() {
+//    Path sysDir = new Path(
+//      conf.get(JTConfig.JT_SYSTEM_DIR, "/tmp/hadoop/mapred/system"));
+//    return fs.makeQualified(sysDir).toString();
+//  }
+//
+//  /**
+//   * @see org.apache.hadoop.mapreduce.protocol.ClientProtocol#getQueueAdmins(String)
+//   */
+//  public AccessControlList getQueueAdmins(String queueName) throws IOException {
+//	  return new AccessControlList(" ");// no queue admins for local job runner
+//  }
+//
+//  /**
+//   * @see org.apache.hadoop.mapreduce.protocol.ClientProtocol#getStagingAreaDir()
+//   */
+//  public String getStagingAreaDir() throws IOException {
+//    Path stagingRootDir = new Path(conf.get(JTConfig.JT_STAGING_AREA_ROOT,
+//        "/tmp/hadoop/mapred/staging"));
+//    UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
+//    String user;
+//    if (ugi != null) {
+//      user = ugi.getShortUserName() + rand.nextInt();
+//    } else {
+//      user = "dummy" + rand.nextInt();
+//    }
+//    return fs.makeQualified(new Path(stagingRootDir, user+"/.staging")).toString();
+//  }
+//
+//  public String getJobHistoryDir() {
+//    return null;
+//  }
+//
+//  @Override
+//  public QueueInfo[] getChildQueues(String queueName) throws IOException {
+//    return null;
+//  }
+//
+//  @Override
+//  public QueueInfo[] getRootQueues() throws IOException {
+//    return null;
+//  }
+//
+//  @Override
+//  public QueueInfo[] getQueues() throws IOException {
+//    return null;
+//  }
+//
+//
+//  @Override
+//  public QueueInfo getQueue(String queue) throws IOException {
+//    return null;
+//  }
+//
+//  @Override
+//  public org.apache.hadoop.mapreduce.QueueAclsInfo[]
+//      getQueueAclsForCurrentUser() throws IOException{
+//    return null;
+//  }
+//
+//  /**
+//   * Set the max number of map tasks to run concurrently in the LocalJobRunner.
+//   * @param job the job to configure
+//   * @param maxMaps the maximum number of map tasks to allow.
+//   */
+//  public static void setLocalMaxRunningMaps(
+//      org.apache.hadoop.mapreduce.JobContext job,
+//      int maxMaps) {
+//    job.getConfiguration().setInt(LOCAL_MAX_MAPS, maxMaps);
+//  }
+//
+//  /**
+//   * @return the max number of map tasks to run concurrently in the
+//   * LocalJobRunner.
+//   */
+//  public static int getLocalMaxRunningMaps(
+//      org.apache.hadoop.mapreduce.JobContext job) {
+//    return job.getConfiguration().getInt(LOCAL_MAX_MAPS, 1);
+//  }
+//
+//  @Override
+//  public void cancelDelegationToken(Token<DelegationTokenIdentifier> token
+//                                       ) throws IOException,
+//                                                InterruptedException {
+//  }
+//
+//  @Override
+//  public Token<DelegationTokenIdentifier>
+//     getDelegationToken(Text renewer) throws IOException, InterruptedException {
+//    return null;
+//  }
+//
+//  @Override
+//  public long renewDelegationToken(Token<DelegationTokenIdentifier> token
+//                                      ) throws IOException,InterruptedException{
+//    return 0;
+//  }
+//
+//  @Override
+//  public LogParams getLogFileParams(org.apache.hadoop.mapreduce.JobID jobID,
+//      org.apache.hadoop.mapreduce.TaskAttemptID taskAttemptID)
+//      throws IOException, InterruptedException {
+//    throw new UnsupportedOperationException("Not supported");
+//  }
+//
+//  static void setupChildMapredLocalDirs(
+//      TaskAttemptID taskAttemptID, String user, JobConf conf) {
+//    String[] localDirs =
+//        conf.getTrimmedStrings(
+//            TezJobConfig.LOCAL_DIRS, TezJobConfig.DEFAULT_LOCAL_DIRS);
+//    String jobId = taskAttemptID.getJobID().toString();
+//    String taskId = taskAttemptID.getTaskID().toString();
+//    boolean isCleanup = false;
+//    StringBuffer childMapredLocalDir =
+//        new StringBuffer(localDirs[0] + Path.SEPARATOR
+//            + getLocalTaskDir(user, jobId, taskId, isCleanup));
+//    for (int i = 1; i < localDirs.length; i++) {
+//      childMapredLocalDir.append("," + localDirs[i] + Path.SEPARATOR
+//          + getLocalTaskDir(user, jobId, taskId, isCleanup));
+//    }
+//    LOG.info(TezJobConfig.LOCAL_DIRS + " for child : " + taskAttemptID +
+//        " is " + childMapredLocalDir);
+//    conf.set(TezJobConfig.LOCAL_DIRS, childMapredLocalDir.toString());
+//    conf.setClass(Constants.TEZ_ENGINE_TASK_OUTPUT_MANAGER,
+//        TezLocalTaskOutputFiles.class, TezTaskOutput.class);
+//  }
+//
+//  static final String TASK_CLEANUP_SUFFIX = ".cleanup";
+//  static final String SUBDIR = jobDir;
+//  static final String JOBCACHE = "jobcache";
+//
+//  static String getLocalTaskDir(String user, String jobid, String taskid,
+//      boolean isCleanupAttempt) {
+//    String taskDir = SUBDIR + Path.SEPARATOR + user + Path.SEPARATOR + JOBCACHE
+//      + Path.SEPARATOR + jobid + Path.SEPARATOR + taskid;
+//    if (isCleanupAttempt) {
+//      taskDir = taskDir + TASK_CLEANUP_SUFFIX;
+//    }
+//    return taskDir;
+//  }
+//
+//
+//}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/SimpleInput.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/SimpleInput.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/SimpleInput.java
index 1e0b146..f03fdc7 100644
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/SimpleInput.java
+++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/SimpleInput.java
@@ -46,8 +46,8 @@ import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.tez.common.TezUtils;
 import org.apache.tez.common.counters.TaskCounter;
 import org.apache.tez.common.counters.TezCounter;
+import org.apache.tez.engine.api.KVReader;
 import org.apache.tez.engine.newapi.Event;
-import org.apache.tez.engine.newapi.KVReader;
 import org.apache.tez.engine.newapi.LogicalInput;
 import org.apache.tez.engine.newapi.TezInputContext;
 import org.apache.tez.mapreduce.common.Utils;

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/output/SimpleOutput.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/output/SimpleOutput.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/output/SimpleOutput.java
index 91fb8cc..dd28ab8 100644
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/output/SimpleOutput.java
+++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/output/SimpleOutput.java
@@ -26,8 +26,8 @@ import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.tez.common.TezUtils;
 import org.apache.tez.common.counters.TaskCounter;
 import org.apache.tez.common.counters.TezCounter;
+import org.apache.tez.engine.api.KVWriter;
 import org.apache.tez.engine.newapi.Event;
-import org.apache.tez.engine.newapi.KVWriter;
 import org.apache.tez.engine.newapi.LogicalOutput;
 import org.apache.tez.engine.newapi.TezOutputContext;
 import org.apache.tez.mapreduce.common.Utils;

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/map/MapProcessor.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/map/MapProcessor.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/map/MapProcessor.java
index 80a2337..e1ed168 100644
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/map/MapProcessor.java
+++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/map/MapProcessor.java
@@ -35,10 +35,10 @@ import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.tez.common.counters.TaskCounter;
 import org.apache.tez.common.counters.TezCounter;
 import org.apache.tez.dag.api.TezException;
+import org.apache.tez.engine.api.KVReader;
+import org.apache.tez.engine.api.KVWriter;
 import org.apache.tez.engine.lib.output.OnFileSortedOutput;
 import org.apache.tez.engine.newapi.Event;
-import org.apache.tez.engine.newapi.KVReader;
-import org.apache.tez.engine.newapi.KVWriter;
 import org.apache.tez.engine.newapi.LogicalIOProcessor;
 import org.apache.tez.engine.newapi.LogicalInput;
 import org.apache.tez.engine.newapi.LogicalOutput;

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/reduce/ReduceProcessor.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/reduce/ReduceProcessor.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/reduce/ReduceProcessor.java
index 88ab754..b7288e8 100644
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/reduce/ReduceProcessor.java
+++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/reduce/ReduceProcessor.java
@@ -37,12 +37,12 @@ import org.apache.hadoop.util.Progressable;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.tez.common.counters.TaskCounter;
 import org.apache.tez.common.counters.TezCounter;
+import org.apache.tez.engine.api.KVReader;
+import org.apache.tez.engine.api.KVWriter;
 import org.apache.tez.engine.common.ConfigUtils;
 import org.apache.tez.engine.common.sort.impl.TezRawKeyValueIterator;
 import org.apache.tez.engine.lib.output.OnFileSortedOutput;
 import org.apache.tez.engine.newapi.Event;
-import org.apache.tez.engine.newapi.KVReader;
-import org.apache.tez.engine.newapi.KVWriter;
 import org.apache.tez.engine.newapi.LogicalIOProcessor;
 import org.apache.tez.engine.newapi.LogicalInput;
 import org.apache.tez.engine.newapi.LogicalOutput;


[26/50] [abbrv] git commit: TEZ-474. Fix TestMapProcessor and TestReduceProcessor unit tests (part of TEZ-398). (sseth)

Posted by ss...@apache.org.
TEZ-474. Fix TestMapProcessor and TestReduceProcessor unit tests (part
of TEZ-398). (sseth)


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

Branch: refs/heads/master
Commit: c5a8a3c6ee72dcf5ec34e28fd87d2685e8b9bb1d
Parents: d316f72
Author: Siddharth Seth <ss...@apache.org>
Authored: Mon Sep 23 23:23:39 2013 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Mon Sep 23 23:23:39 2013 -0700

----------------------------------------------------------------------
 .../org/apache/tez/common/TezJobConfig.java     |   1 +
 .../apache/hadoop/mapred/YarnOutputFiles.java   | 236 ------------------
 .../tez/engine/lib/input/LocalMergedInput.java  |   8 +-
 .../engine/lib/input/ShuffledMergedInput.java   |   2 +-
 .../lib/input/ShuffledMergedInputLegacy.java    |  30 +++
 .../lib/output/LocalOnFileSorterOutput.java     |   7 +-
 .../engine/lib/output/OnFileSortedOutput.java   |   4 +
 .../LogicalIOProcessorRuntimeTask.java          |  46 ++++
 .../tez/mapreduce/examples/MRRSleepJob.java     |   2 +-
 .../mapreduce/examples/OrderedWordCount.java    |   2 +-
 .../apache/tez/mapreduce/TestMRRJobsDAGApi.java |   2 +-
 .../input/ShuffledMergedInputLegacy.java        |  29 ---
 .../apache/tez/mapreduce/processor/MRTask.java  |   5 -
 .../processor/reduce/ReduceProcessor.java       |   2 +-
 .../mapreduce/task/impl/YarnOutputFiles.java    | 239 -------------------
 .../org/apache/tez/mapreduce/TestUmbilical.java |  62 +++++
 .../tez/mapreduce/TestUmbilicalProtocol.java    |  91 -------
 .../tez/mapreduce/processor/MapUtils.java       |  28 ++-
 .../processor/map/TestMapProcessor.java         |  31 ++-
 .../processor/reduce/TestReduceProcessor.java   |  59 +++--
 .../org/apache/tez/mapreduce/YARNRunner.java    |   2 +-
 21 files changed, 233 insertions(+), 655 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/c5a8a3c6/tez-api/src/main/java/org/apache/tez/common/TezJobConfig.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/common/TezJobConfig.java b/tez-api/src/main/java/org/apache/tez/common/TezJobConfig.java
index 7c4540c..2c4b911 100644
--- a/tez-api/src/main/java/org/apache/tez/common/TezJobConfig.java
+++ b/tez-api/src/main/java/org/apache/tez/common/TezJobConfig.java
@@ -64,6 +64,7 @@ public class TezJobConfig {
   /**
    * List of directories avialble to the engine. 
    */
+  @Private
   public static final String LOCAL_DIRS = "tez.engine.local.dirs";
   public static final String DEFAULT_LOCAL_DIRS = "/tmp";
 

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/c5a8a3c6/tez-dag/src/main/java/org/apache/hadoop/mapred/YarnOutputFiles.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/hadoop/mapred/YarnOutputFiles.java b/tez-dag/src/main/java/org/apache/hadoop/mapred/YarnOutputFiles.java
deleted file mode 100644
index e43cf47..0000000
--- a/tez-dag/src/main/java/org/apache/hadoop/mapred/YarnOutputFiles.java
+++ /dev/null
@@ -1,236 +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.hadoop.mapred;
-
-import java.io.IOException;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.LocalDirAllocator;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.mapreduce.JobContext;
-import org.apache.hadoop.mapreduce.MRConfig;
-
-/**
- * Manipulate the working area for the transient store for maps and reduces.
- *
- * This class is used by map and reduce tasks to identify the directories that
- * they need to write to/read from for intermediate files. The callers of
- * these methods are from child space.
- */
-@InterfaceAudience.Private
-@InterfaceStability.Unstable
-public class YarnOutputFiles extends MapOutputFile {
-
-  private JobConf conf;
-
-  private static final String JOB_OUTPUT_DIR = "output";
-  private static final String SPILL_FILE_PATTERN = "%s_spill_%d.out";
-  private static final String SPILL_INDEX_FILE_PATTERN = SPILL_FILE_PATTERN
-      + ".index";
-
-  public YarnOutputFiles() {
-  }
-
-  // assume configured to $localdir/usercache/$user/appcache/$appId
-  private LocalDirAllocator lDirAlloc = 
-    new LocalDirAllocator(MRConfig.LOCAL_DIR);
-
-  private Path getAttemptOutputDir() {
-    return new Path(JOB_OUTPUT_DIR, conf.get(JobContext.TASK_ATTEMPT_ID));
-  }
-  
-  /**
-   * Return the path to local map output file created earlier
-   * 
-   * @return path
-   * @throws IOException
-   */
-  public Path getOutputFile() throws IOException {
-    Path attemptOutput =
-      new Path(getAttemptOutputDir(), MAP_OUTPUT_FILENAME_STRING);
-    return lDirAlloc.getLocalPathToRead(attemptOutput.toString(), conf);
-  }
-
-  /**
-   * Create a local map output file name.
-   * 
-   * @param size the size of the file
-   * @return path
-   * @throws IOException
-   */
-  public Path getOutputFileForWrite(long size) throws IOException {
-    Path attemptOutput = 
-      new Path(getAttemptOutputDir(), MAP_OUTPUT_FILENAME_STRING);
-    return lDirAlloc.getLocalPathForWrite(attemptOutput.toString(), size, conf);
-  }
-
-  /**
-   * Create a local map output file name on the same volume.
-   */
-  public Path getOutputFileForWriteInVolume(Path existing) {
-    Path outputDir = new Path(existing.getParent(), JOB_OUTPUT_DIR);
-    Path attemptOutputDir = new Path(outputDir,
-        conf.get(JobContext.TASK_ATTEMPT_ID));
-    return new Path(attemptOutputDir, MAP_OUTPUT_FILENAME_STRING);
-  }
-
-  /**
-   * Return the path to a local map output index file created earlier
-   * 
-   * @return path
-   * @throws IOException
-   */
-  public Path getOutputIndexFile() throws IOException {
-    Path attemptIndexOutput =
-      new Path(getAttemptOutputDir(), MAP_OUTPUT_FILENAME_STRING +
-                                      MAP_OUTPUT_INDEX_SUFFIX_STRING);
-    return lDirAlloc.getLocalPathToRead(attemptIndexOutput.toString(), conf);
-  }
-
-  /**
-   * Create a local map output index file name.
-   * 
-   * @param size the size of the file
-   * @return path
-   * @throws IOException
-   */
-  public Path getOutputIndexFileForWrite(long size) throws IOException {
-    Path attemptIndexOutput =
-      new Path(getAttemptOutputDir(), MAP_OUTPUT_FILENAME_STRING +
-                                      MAP_OUTPUT_INDEX_SUFFIX_STRING);
-    return lDirAlloc.getLocalPathForWrite(attemptIndexOutput.toString(),
-        size, conf);
-  }
-
-  /**
-   * Create a local map output index file name on the same volume.
-   */
-  public Path getOutputIndexFileForWriteInVolume(Path existing) {
-    Path outputDir = new Path(existing.getParent(), JOB_OUTPUT_DIR);
-    Path attemptOutputDir = new Path(outputDir,
-        conf.get(JobContext.TASK_ATTEMPT_ID));
-    return new Path(attemptOutputDir, MAP_OUTPUT_FILENAME_STRING +
-                                      MAP_OUTPUT_INDEX_SUFFIX_STRING);
-  }
-
-  /**
-   * Return a local map spill file created earlier.
-   * 
-   * @param spillNumber the number
-   * @return path
-   * @throws IOException
-   */
-  public Path getSpillFile(int spillNumber) throws IOException {
-    return lDirAlloc.getLocalPathToRead(
-        String.format(SPILL_FILE_PATTERN,
-            conf.get(JobContext.TASK_ATTEMPT_ID), spillNumber), conf);
-  }
-
-  /**
-   * Create a local map spill file name.
-   * 
-   * @param spillNumber the number
-   * @param size the size of the file
-   * @return path
-   * @throws IOException
-   */
-  public Path getSpillFileForWrite(int spillNumber, long size)
-      throws IOException {
-    return lDirAlloc.getLocalPathForWrite(
-        String.format(String.format(SPILL_FILE_PATTERN,
-            conf.get(JobContext.TASK_ATTEMPT_ID), spillNumber)), size, conf);
-  }
-
-  /**
-   * Return a local map spill index file created earlier
-   * 
-   * @param spillNumber the number
-   * @return path
-   * @throws IOException
-   */
-  public Path getSpillIndexFile(int spillNumber) throws IOException {
-    return lDirAlloc.getLocalPathToRead(
-        String.format(SPILL_INDEX_FILE_PATTERN,
-            conf.get(JobContext.TASK_ATTEMPT_ID), spillNumber), conf);
-  }
-
-  /**
-   * Create a local map spill index file name.
-   * 
-   * @param spillNumber the number
-   * @param size the size of the file
-   * @return path
-   * @throws IOException
-   */
-  public Path getSpillIndexFileForWrite(int spillNumber, long size)
-      throws IOException {
-    return lDirAlloc.getLocalPathForWrite(
-        String.format(SPILL_INDEX_FILE_PATTERN,
-            conf.get(JobContext.TASK_ATTEMPT_ID), spillNumber), size, conf);
-  }
-
-  /**
-   * Return a local reduce input file created earlier
-   * 
-   * @param mapId a map task id
-   * @return path
-   * @throws IOException 
-   */
-  public Path getInputFile(int mapId) throws IOException {
-    throw new UnsupportedOperationException("Incompatible with LocalRunner");
-  }
-
-  /**
-   * Create a local reduce input file name.
-   * 
-   * @param mapId a map task id
-   * @param size the size of the file
-   * @return path
-   * @throws IOException
-   */
-  public Path getInputFileForWrite(org.apache.hadoop.mapreduce.TaskID mapId,
-      long size) throws IOException {
-    return lDirAlloc.getLocalPathForWrite(String.format(
-        REDUCE_INPUT_FILE_FORMAT_STRING,
-        getAttemptOutputDir().toString(), mapId.getId()),
-        size, conf);
-  }
-
-  /** Removes all of the files related to a task. */
-  public void removeAll() throws IOException {
-    throw new UnsupportedOperationException("Incompatible with LocalRunner");
-  }
-
-  @Override
-  public void setConf(Configuration conf) {
-    if (conf instanceof JobConf) {
-      this.conf = (JobConf) conf;
-    } else {
-      this.conf = new JobConf(conf);
-    }
-  }
-
-  @Override
-  public Configuration getConf() {
-    return conf;
-  }
-  
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/c5a8a3c6/tez-engine/src/main/java/org/apache/tez/engine/lib/input/LocalMergedInput.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/lib/input/LocalMergedInput.java b/tez-engine/src/main/java/org/apache/tez/engine/lib/input/LocalMergedInput.java
index ed57c61..6371787 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/lib/input/LocalMergedInput.java
+++ b/tez-engine/src/main/java/org/apache/tez/engine/lib/input/LocalMergedInput.java
@@ -31,11 +31,7 @@ import org.apache.tez.engine.common.localshuffle.LocalShuffle;
  * <code>LocalMergedInput</code> in an {@link LogicalInput} which shuffles intermediate
  * sorted data, merges them and provides key/<values> to the consumer. 
  */
-public class LocalMergedInput extends ShuffledMergedInput {
-
-
-  // TODO NEWTEZ Fix CombineProcessor
-  //private CombineInput raw;
+public class LocalMergedInput extends ShuffledMergedInputLegacy {
 
   @Override
   public List<Event> initialize(TezInputContext inputContext) throws IOException {
@@ -43,8 +39,8 @@ public class LocalMergedInput extends ShuffledMergedInput {
     this.conf = TezUtils.createConfFromUserPayload(inputContext.getUserPayload());
 
     LocalShuffle localShuffle = new LocalShuffle(inputContext, conf, numInputs);
-    // TODO NEWTEZ async run and checkIfComplete methods
     rawIter = localShuffle.run();
+    createValuesIterator();
     return Collections.emptyList();
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/c5a8a3c6/tez-engine/src/main/java/org/apache/tez/engine/lib/input/ShuffledMergedInput.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/lib/input/ShuffledMergedInput.java b/tez-engine/src/main/java/org/apache/tez/engine/lib/input/ShuffledMergedInput.java
index 3db0632..a984b0f 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/lib/input/ShuffledMergedInput.java
+++ b/tez-engine/src/main/java/org/apache/tez/engine/lib/input/ShuffledMergedInput.java
@@ -156,7 +156,7 @@ public class ShuffledMergedInput implements LogicalInput {
   }
 
   @SuppressWarnings({ "rawtypes", "unchecked" })
-  private void createValuesIterator()
+  protected void createValuesIterator()
       throws IOException {
     vIter = new ValuesIterator(rawIter,
         (RawComparator) ConfigUtils.getIntermediateInputKeyComparator(conf),

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/c5a8a3c6/tez-engine/src/main/java/org/apache/tez/engine/lib/input/ShuffledMergedInputLegacy.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/lib/input/ShuffledMergedInputLegacy.java b/tez-engine/src/main/java/org/apache/tez/engine/lib/input/ShuffledMergedInputLegacy.java
new file mode 100644
index 0000000..f2da031
--- /dev/null
+++ b/tez-engine/src/main/java/org/apache/tez/engine/lib/input/ShuffledMergedInputLegacy.java
@@ -0,0 +1,30 @@
+/**
+ * <code>ShuffleMergedInput</code> in a {@link LogicalInput} which shuffles
+ * intermediate sorted data, merges them and provides key/<values> to the
+ * consumer.
+ * 
+ * The Copy and Merge will be triggered by the initialization - which is handled
+ * by the Tez framework. Input is not consumable until the Copy and Merge are
+ * complete. Methods are provided to check for this, as well as to wait for
+ * completion. Attempting to get a reader on a non-complete input will block.
+ * 
+ */
+
+package org.apache.tez.engine.lib.input;
+
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience.LimitedPrivate;
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.tez.engine.common.sort.impl.TezRawKeyValueIterator;
+
+@LimitedPrivate("mapreduce")
+public class ShuffledMergedInputLegacy extends ShuffledMergedInput {
+
+  @Private
+  public TezRawKeyValueIterator getIterator() throws IOException, InterruptedException {
+    // wait for input so that iterator is available
+    waitForInputReady();
+    return rawIter;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/c5a8a3c6/tez-engine/src/main/java/org/apache/tez/engine/lib/output/LocalOnFileSorterOutput.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/lib/output/LocalOnFileSorterOutput.java b/tez-engine/src/main/java/org/apache/tez/engine/lib/output/LocalOnFileSorterOutput.java
index b24e10d..7fd26d7 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/lib/output/LocalOnFileSorterOutput.java
+++ b/tez-engine/src/main/java/org/apache/tez/engine/lib/output/LocalOnFileSorterOutput.java
@@ -48,11 +48,16 @@ public class LocalOnFileSorterOutput extends OnFileSortedOutput {
             outputContext.getTaskIndex(),
             localFs.getFileStatus(src).getLen());
 
+    LOG.info("Renaming src = " + src + ", dst = " + dst);
     if (LOG.isDebugEnabled()) {
       LOG.debug("Renaming src = " + src + ", dst = " + dst);
     }
     localFs.rename(src, dst);
-    // TODO NEWTEZ Event generation.
+    return null;
+  }
+  
+  @Override
+  protected List<Event> generateDataMovementEventsOnClose() throws IOException {
     return null;
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/c5a8a3c6/tez-engine/src/main/java/org/apache/tez/engine/lib/output/OnFileSortedOutput.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/lib/output/OnFileSortedOutput.java b/tez-engine/src/main/java/org/apache/tez/engine/lib/output/OnFileSortedOutput.java
index 685722e..9c9eba0 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/lib/output/OnFileSortedOutput.java
+++ b/tez-engine/src/main/java/org/apache/tez/engine/lib/output/OnFileSortedOutput.java
@@ -93,6 +93,10 @@ public class OnFileSortedOutput implements LogicalOutput {
     sorter.close();
     this.endTime = System.nanoTime();
 
+   return generateDataMovementEventsOnClose();
+  }
+  
+  protected List<Event> generateDataMovementEventsOnClose() throws IOException {
     String host = System.getenv(ApplicationConstants.Environment.NM_HOST
         .toString());
     ByteBuffer shuffleMetadata = outputContext

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/c5a8a3c6/tez-engine/src/main/java/org/apache/tez/engine/newruntime/LogicalIOProcessorRuntimeTask.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/newruntime/LogicalIOProcessorRuntimeTask.java b/tez-engine/src/main/java/org/apache/tez/engine/newruntime/LogicalIOProcessorRuntimeTask.java
index bfd898b..29063f9 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/newruntime/LogicalIOProcessorRuntimeTask.java
+++ b/tez-engine/src/main/java/org/apache/tez/engine/newruntime/LogicalIOProcessorRuntimeTask.java
@@ -61,6 +61,7 @@ import org.apache.tez.engine.api.impl.EventMetaData.EventProducerConsumerType;
 import org.apache.tez.engine.common.security.JobTokenIdentifier;
 import org.apache.tez.engine.shuffle.common.ShuffleUtils;
 
+import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 
 @Private
@@ -75,6 +76,10 @@ public class LogicalIOProcessorRuntimeTask extends RuntimeTask {
   private final List<OutputSpec> outputSpecs;
   private final List<LogicalOutput> outputs;
 
+  private List<TezInputContext> inputContexts;
+  private List<TezOutputContext> outputContexts;
+  private TezProcessorContext processorContext;
+  
   private final ProcessorDescriptor processorDescriptor;
   private final LogicalIOProcessor processor;
 
@@ -95,6 +100,8 @@ public class LogicalIOProcessorRuntimeTask extends RuntimeTask {
     super(taskSpec, tezConf, tezUmbilical);
     LOG.info("Initializing LogicalIOProcessorRuntimeTask with TaskSpec: "
         + taskSpec);
+    this.inputContexts = new ArrayList<TezInputContext>(taskSpec.getInputs().size());
+    this.outputContexts = new ArrayList<TezOutputContext>(taskSpec.getOutputs().size());
     this.inputSpecs = taskSpec.getInputs();
     this.inputs = createInputs(inputSpecs);
     this.outputSpecs = taskSpec.getOutputs();
@@ -185,6 +192,7 @@ public class LogicalIOProcessorRuntimeTask extends RuntimeTask {
   private void initializeInput(Input input, InputSpec inputSpec)
       throws Exception {
     TezInputContext tezInputContext = createInputContext(inputSpec);
+    inputContexts.add(tezInputContext);
     if (input instanceof LogicalInput) {
       ((LogicalInput) input).setNumPhysicalInputs(inputSpec
           .getPhysicalEdgeCount());
@@ -199,6 +207,7 @@ public class LogicalIOProcessorRuntimeTask extends RuntimeTask {
   private void initializeOutput(Output output, OutputSpec outputSpec)
       throws Exception {
     TezOutputContext tezOutputContext = createOutputContext(outputSpec);
+    outputContexts.add(tezOutputContext);
     if (output instanceof LogicalOutput) {
       ((LogicalOutput) output).setNumPhysicalOutputs(outputSpec
           .getPhysicalEdgeCount());
@@ -215,6 +224,7 @@ public class LogicalIOProcessorRuntimeTask extends RuntimeTask {
     LOG.info("Initializing processor"
         + ", processorClassName=" + processorDescriptor.getClassName());
     TezProcessorContext processorContext = createProcessorContext();
+    this.processorContext = processorContext;
     processor.initialize(processorContext);
   }
 
@@ -425,5 +435,41 @@ public class LogicalIOProcessorRuntimeTask extends RuntimeTask {
       eventRouterThread.interrupt();
     }
   }
+  
+  @Private
+  @VisibleForTesting
+  public List<TezInputContext> getInputContexts() {
+    return this.inputContexts;
+  }
+  
+  @Private
+  @VisibleForTesting
+  public List<TezOutputContext> getOutputContexts() {
+    return this.outputContexts;
+  }
 
+  @Private
+  @VisibleForTesting
+  public TezProcessorContext getProcessorContext() {
+    return this.processorContext;
+  }
+  
+  @Private
+  @VisibleForTesting
+  public Map<String, LogicalInput> getInputs() {
+    return this.inputMap;
+  }
+  
+  @Private
+  @VisibleForTesting
+  public Map<String, LogicalOutput> getOutputs() {
+    return this.outputMap;
+  }
+  
+  @Private
+  @VisibleForTesting
+  public LogicalIOProcessor getProcessor() {
+    return this.processor;
+  }
+  
 }

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/c5a8a3c6/tez-mapreduce-examples/src/main/java/org/apache/tez/mapreduce/examples/MRRSleepJob.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce-examples/src/main/java/org/apache/tez/mapreduce/examples/MRRSleepJob.java b/tez-mapreduce-examples/src/main/java/org/apache/tez/mapreduce/examples/MRRSleepJob.java
index 429d458..05675b5 100644
--- a/tez-mapreduce-examples/src/main/java/org/apache/tez/mapreduce/examples/MRRSleepJob.java
+++ b/tez-mapreduce-examples/src/main/java/org/apache/tez/mapreduce/examples/MRRSleepJob.java
@@ -80,12 +80,12 @@ import org.apache.tez.dag.api.client.DAGStatus;
 import org.apache.tez.engine.common.objectregistry.ObjectLifeCycle;
 import org.apache.tez.engine.common.objectregistry.ObjectRegistry;
 import org.apache.tez.engine.common.objectregistry.ObjectRegistryFactory;
+import org.apache.tez.engine.lib.input.ShuffledMergedInputLegacy;
 import org.apache.tez.engine.lib.output.OnFileSortedOutput;
 import org.apache.tez.mapreduce.hadoop.InputSplitInfo;
 import org.apache.tez.mapreduce.hadoop.MRHelpers;
 import org.apache.tez.mapreduce.hadoop.MultiStageMRConfToTezTranslator;
 import org.apache.tez.mapreduce.hadoop.MultiStageMRConfigUtil;
-import org.apache.tez.mapreduce.input.ShuffledMergedInputLegacy;
 import org.apache.tez.mapreduce.processor.map.MapProcessor;
 import org.apache.tez.mapreduce.processor.reduce.ReduceProcessor;
 

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/c5a8a3c6/tez-mapreduce-examples/src/main/java/org/apache/tez/mapreduce/examples/OrderedWordCount.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce-examples/src/main/java/org/apache/tez/mapreduce/examples/OrderedWordCount.java b/tez-mapreduce-examples/src/main/java/org/apache/tez/mapreduce/examples/OrderedWordCount.java
index 07fe58a..ec419c1 100644
--- a/tez-mapreduce-examples/src/main/java/org/apache/tez/mapreduce/examples/OrderedWordCount.java
+++ b/tez-mapreduce-examples/src/main/java/org/apache/tez/mapreduce/examples/OrderedWordCount.java
@@ -70,12 +70,12 @@ import org.apache.tez.dag.api.EdgeProperty.DataSourceType;
 import org.apache.tez.dag.api.EdgeProperty.SchedulingType;
 import org.apache.tez.dag.api.client.DAGClient;
 import org.apache.tez.dag.api.client.DAGStatus;
+import org.apache.tez.engine.lib.input.ShuffledMergedInputLegacy;
 import org.apache.tez.engine.lib.output.OnFileSortedOutput;
 import org.apache.tez.mapreduce.hadoop.InputSplitInfo;
 import org.apache.tez.mapreduce.hadoop.MRHelpers;
 import org.apache.tez.mapreduce.hadoop.MRJobConfig;
 import org.apache.tez.mapreduce.hadoop.MultiStageMRConfToTezTranslator;
-import org.apache.tez.mapreduce.input.ShuffledMergedInputLegacy;
 import org.apache.tez.mapreduce.processor.map.MapProcessor;
 import org.apache.tez.mapreduce.processor.reduce.ReduceProcessor;
 

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/c5a8a3c6/tez-mapreduce-tests/src/test/java/org/apache/tez/mapreduce/TestMRRJobsDAGApi.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce-tests/src/test/java/org/apache/tez/mapreduce/TestMRRJobsDAGApi.java b/tez-mapreduce-tests/src/test/java/org/apache/tez/mapreduce/TestMRRJobsDAGApi.java
index aca5b8e..7e662cb 100644
--- a/tez-mapreduce-tests/src/test/java/org/apache/tez/mapreduce/TestMRRJobsDAGApi.java
+++ b/tez-mapreduce-tests/src/test/java/org/apache/tez/mapreduce/TestMRRJobsDAGApi.java
@@ -68,6 +68,7 @@ import org.apache.tez.dag.api.EdgeProperty.SchedulingType;
 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.engine.lib.input.ShuffledMergedInputLegacy;
 import org.apache.tez.engine.lib.output.OnFileSortedOutput;
 import org.apache.tez.mapreduce.examples.MRRSleepJob;
 import org.apache.tez.mapreduce.examples.MRRSleepJob.ISleepReducer;
@@ -79,7 +80,6 @@ import org.apache.tez.mapreduce.hadoop.InputSplitInfo;
 import org.apache.tez.mapreduce.hadoop.MRHelpers;
 import org.apache.tez.mapreduce.hadoop.MRJobConfig;
 import org.apache.tez.mapreduce.hadoop.MultiStageMRConfToTezTranslator;
-import org.apache.tez.mapreduce.input.ShuffledMergedInputLegacy;
 import org.apache.tez.mapreduce.processor.map.MapProcessor;
 import org.apache.tez.mapreduce.processor.reduce.ReduceProcessor;
 import org.junit.AfterClass;

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/c5a8a3c6/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/ShuffledMergedInputLegacy.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/ShuffledMergedInputLegacy.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/ShuffledMergedInputLegacy.java
deleted file mode 100644
index 2d230d6..0000000
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/ShuffledMergedInputLegacy.java
+++ /dev/null
@@ -1,29 +0,0 @@
-/**
- * <code>ShuffleMergedInput</code> in a {@link LogicalInput} which shuffles
- * intermediate sorted data, merges them and provides key/<values> to the
- * consumer.
- * 
- * The Copy and Merge will be triggered by the initialization - which is handled
- * by the Tez framework. Input is not consumable until the Copy and Merge are
- * complete. Methods are provided to check for this, as well as to wait for
- * completion. Attempting to get a reader on a non-complete input will block.
- * 
- */
-
-package org.apache.tez.mapreduce.input;
-
-import java.io.IOException;
-
-import org.apache.hadoop.classification.InterfaceAudience.Private;
-import org.apache.tez.engine.common.sort.impl.TezRawKeyValueIterator;
-import org.apache.tez.engine.lib.input.ShuffledMergedInput;
-
-public class ShuffledMergedInputLegacy extends ShuffledMergedInput {
-
-  @Private
-  public TezRawKeyValueIterator getIterator() throws IOException, InterruptedException {
-    // wait for input so that iterator is available
-    waitForInputReady();
-    return rawIter;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/c5a8a3c6/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/MRTask.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/MRTask.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/MRTask.java
index 1a01466..f7404d4 100644
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/MRTask.java
+++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/MRTask.java
@@ -44,7 +44,6 @@ import org.apache.hadoop.io.RawComparator;
 import org.apache.hadoop.mapred.FileAlreadyExistsException;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.JobContext;
-import org.apache.hadoop.mapred.MapOutputFile;
 import org.apache.hadoop.mapred.RawKeyValueIterator;
 import org.apache.hadoop.mapred.TaskAttemptContext;
 import org.apache.hadoop.mapred.TaskAttemptID;
@@ -83,7 +82,6 @@ import org.apache.tez.mapreduce.hadoop.MRJobConfig;
 import org.apache.tez.mapreduce.hadoop.mapred.TaskAttemptContextImpl;
 import org.apache.tez.mapreduce.hadoop.mapreduce.JobContextImpl;
 import org.apache.tez.mapreduce.output.SimpleOutput;
-import org.apache.tez.mapreduce.task.impl.YarnOutputFiles;
 
 @SuppressWarnings("deprecation")
 public abstract class MRTask {
@@ -204,9 +202,6 @@ public abstract class MRTask {
     // Containers.
     // Set it in conf, so as to be able to be used the the OutputCommitter.
 
-    jobConf.setClass(MRConfig.TASK_LOCAL_OUTPUT_CLASS, YarnOutputFiles.class,
-        MapOutputFile.class); // MR
-
     // Not needed. This is probably being set via the source/consumer meta
     Token<JobTokenIdentifier> jobToken = TokenCache.getJobToken(credentials);
     if (jobToken != null) {

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/c5a8a3c6/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/reduce/ReduceProcessor.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/reduce/ReduceProcessor.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/reduce/ReduceProcessor.java
index 9210187..9274765 100644
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/reduce/ReduceProcessor.java
+++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/reduce/ReduceProcessor.java
@@ -46,8 +46,8 @@ import org.apache.tez.engine.api.LogicalOutput;
 import org.apache.tez.engine.api.TezProcessorContext;
 import org.apache.tez.engine.common.ConfigUtils;
 import org.apache.tez.engine.common.sort.impl.TezRawKeyValueIterator;
+import org.apache.tez.engine.lib.input.ShuffledMergedInputLegacy;
 import org.apache.tez.engine.lib.output.OnFileSortedOutput;
-import org.apache.tez.mapreduce.input.ShuffledMergedInputLegacy;
 import org.apache.tez.mapreduce.output.SimpleOutput;
 import org.apache.tez.mapreduce.processor.MRTask;
 import org.apache.tez.mapreduce.processor.MRTaskReporter;

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/c5a8a3c6/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/task/impl/YarnOutputFiles.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/task/impl/YarnOutputFiles.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/task/impl/YarnOutputFiles.java
deleted file mode 100644
index e28e474..0000000
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/task/impl/YarnOutputFiles.java
+++ /dev/null
@@ -1,239 +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.task.impl;
-
-import java.io.IOException;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.LocalDirAllocator;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.MapOutputFile;
-import org.apache.hadoop.mapreduce.JobContext;
-import org.apache.hadoop.mapreduce.MRConfig;
-import org.apache.tez.common.Constants;
-
-/**
- * Manipulate the working area for the transient store for maps and reduces.
- *
- * This class is used by map and reduce tasks to identify the directories that
- * they need to write to/read from for intermediate files. The callers of
- * these methods are from child space.
- */
-@InterfaceAudience.Private
-@InterfaceStability.Unstable
-public class YarnOutputFiles extends MapOutputFile {
-
-  private JobConf conf;
-
-  private static final String JOB_OUTPUT_DIR = "output";
-  private static final String SPILL_FILE_PATTERN = "%s_spill_%d.out";
-  private static final String SPILL_INDEX_FILE_PATTERN = SPILL_FILE_PATTERN
-      + ".index";
-
-  public YarnOutputFiles() {
-  }
-
-  // assume configured to $localdir/usercache/$user/appcache/$appId
-  private LocalDirAllocator lDirAlloc = 
-    new LocalDirAllocator(MRConfig.LOCAL_DIR);
-
-  private Path getAttemptOutputDir() {
-    return new Path(JOB_OUTPUT_DIR, conf.get(JobContext.TASK_ATTEMPT_ID));
-  }
-  
-  /**
-   * Return the path to local map output file created earlier
-   * 
-   * @return path
-   * @throws IOException
-   */
-  public Path getOutputFile() throws IOException {
-    Path attemptOutput =
-      new Path(getAttemptOutputDir(), Constants.MAP_OUTPUT_FILENAME_STRING);
-    return lDirAlloc.getLocalPathToRead(attemptOutput.toString(), conf);
-  }
-
-  /**
-   * Create a local map output file name.
-   * 
-   * @param size the size of the file
-   * @return path
-   * @throws IOException
-   */
-  public Path getOutputFileForWrite(long size) throws IOException {
-    Path attemptOutput = 
-      new Path(getAttemptOutputDir(), Constants.MAP_OUTPUT_FILENAME_STRING);
-    return lDirAlloc.getLocalPathForWrite(attemptOutput.toString(), size, conf);
-  }
-
-  /**
-   * Create a local map output file name on the same volume.
-   */
-  public Path getOutputFileForWriteInVolume(Path existing) {
-    Path outputDir = new Path(existing.getParent(), JOB_OUTPUT_DIR);
-    Path attemptOutputDir = new Path(outputDir,
-        conf.get(JobContext.TASK_ATTEMPT_ID));
-    return new Path(attemptOutputDir, Constants.MAP_OUTPUT_FILENAME_STRING);
-  }
-
-  /**
-   * Return the path to a local map output index file created earlier
-   * 
-   * @return path
-   * @throws IOException
-   */
-  public Path getOutputIndexFile() throws IOException {
-    Path attemptIndexOutput =
-      new Path(getAttemptOutputDir(), Constants.MAP_OUTPUT_FILENAME_STRING +
-          Constants.MAP_OUTPUT_INDEX_SUFFIX_STRING);
-    return lDirAlloc.getLocalPathToRead(attemptIndexOutput.toString(), conf);
-  }
-
-  /**
-   * Create a local map output index file name.
-   * 
-   * @param size the size of the file
-   * @return path
-   * @throws IOException
-   */
-  public Path getOutputIndexFileForWrite(long size) throws IOException {
-    Path attemptIndexOutput =
-      new Path(getAttemptOutputDir(), Constants.MAP_OUTPUT_FILENAME_STRING +
-          Constants.MAP_OUTPUT_INDEX_SUFFIX_STRING);
-    return lDirAlloc.getLocalPathForWrite(attemptIndexOutput.toString(),
-        size, conf);
-  }
-
-  /**
-   * Create a local map output index file name on the same volume.
-   */
-  public Path getOutputIndexFileForWriteInVolume(Path existing) {
-    Path outputDir = new Path(existing.getParent(), JOB_OUTPUT_DIR);
-    Path attemptOutputDir = new Path(outputDir,
-        conf.get(JobContext.TASK_ATTEMPT_ID));
-    return new Path(attemptOutputDir, Constants.MAP_OUTPUT_FILENAME_STRING +
-        Constants.MAP_OUTPUT_INDEX_SUFFIX_STRING);
-  }
-
-  /**
-   * Return a local map spill file created earlier.
-   * 
-   * @param spillNumber the number
-   * @return path
-   * @throws IOException
-   */
-  public Path getSpillFile(int spillNumber) throws IOException {
-    return lDirAlloc.getLocalPathToRead(
-        String.format(SPILL_FILE_PATTERN,
-            conf.get(JobContext.TASK_ATTEMPT_ID), spillNumber), conf);
-  }
-
-  /**
-   * Create a local map spill file name.
-   * 
-   * @param spillNumber the number
-   * @param size the size of the file
-   * @return path
-   * @throws IOException
-   */
-  public Path getSpillFileForWrite(int spillNumber, long size)
-      throws IOException {
-    return lDirAlloc.getLocalPathForWrite(
-        String.format(String.format(SPILL_FILE_PATTERN,
-            conf.get(JobContext.TASK_ATTEMPT_ID), spillNumber)), size, conf);
-  }
-
-  /**
-   * Return a local map spill index file created earlier
-   * 
-   * @param spillNumber the number
-   * @return path
-   * @throws IOException
-   */
-  public Path getSpillIndexFile(int spillNumber) throws IOException {
-    return lDirAlloc.getLocalPathToRead(
-        String.format(SPILL_INDEX_FILE_PATTERN,
-            conf.get(JobContext.TASK_ATTEMPT_ID), spillNumber), conf);
-  }
-
-  /**
-   * Create a local map spill index file name.
-   * 
-   * @param spillNumber the number
-   * @param size the size of the file
-   * @return path
-   * @throws IOException
-   */
-  public Path getSpillIndexFileForWrite(int spillNumber, long size)
-      throws IOException {
-    return lDirAlloc.getLocalPathForWrite(
-        String.format(SPILL_INDEX_FILE_PATTERN,
-            conf.get(JobContext.TASK_ATTEMPT_ID), spillNumber), size, conf);
-  }
-
-  /**
-   * Return a local reduce input file created earlier
-   * 
-   * @param mapId a map task id
-   * @return path
-   * @throws IOException 
-   */
-  public Path getInputFile(int mapId) throws IOException {
-    throw new UnsupportedOperationException("Incompatible with LocalRunner");
-  }
-
-  /**
-   * Create a local reduce input file name.
-   * 
-   * @param mapId a map task id
-   * @param size the size of the file
-   * @return path
-   * @throws IOException
-   */
-  public Path getInputFileForWrite(org.apache.hadoop.mapreduce.TaskID mapId,
-      long size) throws IOException {
-    return lDirAlloc.getLocalPathForWrite(String.format(
-        Constants.REDUCE_INPUT_FILE_FORMAT_STRING,
-        getAttemptOutputDir().toString(), mapId.getId()),
-        size, conf);
-  }
-
-  /** Removes all of the files related to a task. */
-  public void removeAll() throws IOException {
-    throw new UnsupportedOperationException("Incompatible with LocalRunner");
-  }
-
-  @Override
-  public void setConf(Configuration conf) {
-    if (conf instanceof JobConf) {
-      this.conf = (JobConf) conf;
-    } else {
-      this.conf = new JobConf(conf);
-    }
-  }
-
-  @Override
-  public Configuration getConf() {
-    return conf;
-  }
-  
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/c5a8a3c6/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/TestUmbilical.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/TestUmbilical.java b/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/TestUmbilical.java
new file mode 100644
index 0000000..9de2ed1
--- /dev/null
+++ b/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/TestUmbilical.java
@@ -0,0 +1,62 @@
+/**
+ * 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;
+
+import java.io.IOException;
+import java.util.Collection;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.tez.dag.records.TezTaskAttemptID;
+import org.apache.tez.engine.api.impl.EventMetaData;
+import org.apache.tez.engine.api.impl.TezEvent;
+import org.apache.tez.engine.api.impl.TezUmbilical;
+
+public class TestUmbilical implements TezUmbilical {
+
+  private static final Log LOG = LogFactory.getLog(TestUmbilical.class);
+
+  public TestUmbilical() {
+  }
+
+  @Override
+  public void addEvents(Collection<TezEvent> events) {
+    if (events != null && events.size() > 0) {
+      LOG.info("#Events Received: " + events.size());
+      for (TezEvent event : events) {
+        LOG.info("Event: " + event);
+      }
+    }
+  }
+
+  @Override
+  public void signalFatalError(TezTaskAttemptID taskAttemptID,
+      String diagnostics, EventMetaData sourceInfo) {
+    LOG.info("Received fatal error from task: " + taskAttemptID
+        + ", Diagnostics: " + diagnostics);
+
+  }
+
+  @Override
+  public boolean canCommit(TezTaskAttemptID taskAttemptID) throws IOException {
+    LOG.info("Got canCommit from task: " + taskAttemptID);
+    return true;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/c5a8a3c6/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/TestUmbilicalProtocol.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/TestUmbilicalProtocol.java b/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/TestUmbilicalProtocol.java
deleted file mode 100644
index d5823f7..0000000
--- a/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/TestUmbilicalProtocol.java
+++ /dev/null
@@ -1,91 +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;
-
-import java.io.IOException;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.ipc.ProtocolSignature;
-import org.apache.tez.common.ContainerContext;
-import org.apache.tez.common.ContainerTask;
-import org.apache.tez.common.TezTaskUmbilicalProtocol;
-import org.apache.tez.common.records.ProceedToCompletionResponse;
-import org.apache.tez.dag.records.TezTaskAttemptID;
-import org.apache.tez.engine.api.impl.TezHeartbeatRequest;
-import org.apache.tez.engine.api.impl.TezHeartbeatResponse;
-
-public class TestUmbilicalProtocol implements TezTaskUmbilicalProtocol {
-
-  private static final Log LOG = LogFactory.getLog(TestUmbilicalProtocol.class);
-  private ProceedToCompletionResponse proceedToCompletionResponse;
-
-
-  public TestUmbilicalProtocol() {
-    proceedToCompletionResponse = new ProceedToCompletionResponse(false, true);
-  }
-
-  public TestUmbilicalProtocol(boolean shouldLinger) {
-    if (shouldLinger) {
-      proceedToCompletionResponse = new ProceedToCompletionResponse(false, false);
-    } else {
-      proceedToCompletionResponse = new ProceedToCompletionResponse(false, true);
-    }
-  }
-
-  @Override
-  public ProtocolSignature getProtocolSignature(String arg0, long arg1, int arg2)
-      throws IOException {
-    // TODO Auto-generated method stub
-    return null;
-  }
-
-  @Override
-  public long getProtocolVersion(String arg0, long arg1) throws IOException {
-    // TODO Auto-generated method stub
-    return 0;
-  }
-
-  @Override
-  public ContainerTask getTask(ContainerContext containerContext)
-      throws IOException {
-    // TODO Auto-generated method stub
-    return null;
-  }
-
-  @Override
-  public boolean canCommit(TezTaskAttemptID taskid) throws IOException {
-    LOG.info("Got 'can-commit' from " + taskid);
-    return true;
-  }
-
-  @Override
-  public ProceedToCompletionResponse proceedToCompletion(
-      TezTaskAttemptID taskAttemptId) throws IOException {
-    return proceedToCompletionResponse;
-  }
-
-  @Override
-  public TezHeartbeatResponse heartbeat(TezHeartbeatRequest request) {
-    // TODO Auto-generated method stub
-    // TODO TODONEWTEZ
-    return null;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/c5a8a3c6/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/processor/MapUtils.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/processor/MapUtils.java b/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/processor/MapUtils.java
index 85e6653..4b2c0e8 100644
--- a/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/processor/MapUtils.java
+++ b/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/processor/MapUtils.java
@@ -43,13 +43,16 @@ import org.apache.hadoop.mapred.SequenceFileInputFormat;
 import org.apache.hadoop.mapreduce.split.JobSplit;
 import org.apache.hadoop.mapreduce.split.JobSplit.SplitMetaInfo;
 import org.apache.hadoop.mapreduce.split.SplitMetaInfoReaderTez;
+import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.util.DiskChecker.DiskErrorException;
 import org.apache.tez.common.TezJobConfig;
+import org.apache.tez.common.TezUtils;
 import org.apache.tez.dag.api.ProcessorDescriptor;
 import org.apache.tez.engine.api.impl.InputSpec;
 import org.apache.tez.engine.api.impl.OutputSpec;
 import org.apache.tez.engine.api.impl.TaskSpec;
 import org.apache.tez.engine.api.impl.TezUmbilical;
+import org.apache.tez.engine.common.security.JobTokenIdentifier;
 import org.apache.tez.engine.newruntime.LogicalIOProcessorRuntimeTask;
 import org.apache.tez.mapreduce.TezTestUtils;
 import org.apache.tez.mapreduce.hadoop.MRJobConfig;
@@ -108,6 +111,7 @@ public class MapUtils {
       throws IOException {
     FileInputFormat.setInputPaths(job, workDir);
 
+    LOG.info("Generating data at path: " + file);
     // create a file with length entries
     @SuppressWarnings("deprecation")
     SequenceFile.Writer writer = 
@@ -147,6 +151,7 @@ public class MapUtils {
       InputSplit split) throws IOException {
     Path jobSplitFile = new Path(conf.get(TezJobConfig.TASK_LOCAL_RESOURCE_DIR,
         TezJobConfig.DEFAULT_TASK_LOCAL_RESOURCE_DIR), MRJobConfig.JOB_SPLIT);
+    LOG.info("Writing split to: " + jobSplitFile);
     FSDataOutputStream out = FileSystem.create(fs, jobSplitFile,
         new FsPermission(JOB_FILE_PERMISSION));
 
@@ -173,17 +178,23 @@ public class MapUtils {
     outMeta.close();
   }
 
-  public static LogicalIOProcessorRuntimeTask runMapProcessor(FileSystem fs, Path workDir,
+  public static void generateInputSplit(FileSystem fs, Path workDir, JobConf jobConf, Path mapInput) throws IOException {
+    jobConf.setInputFormat(SequenceFileInputFormat.class);
+    InputSplit split = createInputSplit(fs, workDir, jobConf, mapInput);
+    writeSplitFiles(fs, jobConf, split);
+  }
+  
+  public static LogicalIOProcessorRuntimeTask createLogicalTask(FileSystem fs, Path workDir,
       JobConf jobConf, int mapId, Path mapInput,
       TezUmbilical umbilical,
       String vertexName, List<InputSpec> inputSpecs,
       List<OutputSpec> outputSpecs) throws Exception {
     jobConf.setInputFormat(SequenceFileInputFormat.class);
-    InputSplit split = createInputSplit(fs, workDir, jobConf, mapInput);
 
     ProcessorDescriptor mapProcessorDesc = new ProcessorDescriptor(
-        MapProcessor.class.getName());
-    writeSplitFiles(fs, jobConf, split);
+        MapProcessor.class.getName()).setUserPayload(TezUtils.createUserPayloadFromConf(jobConf));
+    
+    Token<JobTokenIdentifier> shuffleToken = new Token<JobTokenIdentifier>();
 
     TaskSpec taskSpec = new TaskSpec(
         TezTestUtils.getMockTaskAttemptId(0, 0, mapId, 0),
@@ -192,16 +203,13 @@ public class MapUtils {
         mapProcessorDesc,
         inputSpecs,
         outputSpecs);
-    
-    // TODO NEWTEZ Fix umbilical access
+
     LogicalIOProcessorRuntimeTask task = new LogicalIOProcessorRuntimeTask(
         taskSpec,
-        1,
+        0,
         jobConf,
         umbilical,
-        null);
-    task.initialize();
-    task.run();
+        shuffleToken);
     return task;
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/c5a8a3c6/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/processor/map/TestMapProcessor.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/processor/map/TestMapProcessor.java b/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/processor/map/TestMapProcessor.java
index 2ecce8b..06e2f4b 100644
--- a/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/processor/map/TestMapProcessor.java
+++ b/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/processor/map/TestMapProcessor.java
@@ -30,23 +30,26 @@ import org.apache.hadoop.io.DataInputBuffer;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapreduce.MRConfig;
 import org.apache.tez.common.Constants;
 import org.apache.tez.common.TezJobConfig;
 import org.apache.tez.dag.api.InputDescriptor;
 import org.apache.tez.dag.api.OutputDescriptor;
+import org.apache.tez.engine.api.TezInputContext;
 import org.apache.tez.engine.api.impl.InputSpec;
 import org.apache.tez.engine.api.impl.OutputSpec;
-import org.apache.tez.engine.api.impl.TezUmbilical;
 import org.apache.tez.engine.common.InputAttemptIdentifier;
 import org.apache.tez.engine.common.sort.impl.IFile;
 import org.apache.tez.engine.common.task.local.output.TezLocalTaskOutputFiles;
 import org.apache.tez.engine.common.task.local.output.TezTaskOutput;
 import org.apache.tez.engine.lib.output.LocalOnFileSorterOutput;
 import org.apache.tez.engine.newruntime.LogicalIOProcessorRuntimeTask;
+import org.apache.tez.mapreduce.TestUmbilical;
 import org.apache.tez.mapreduce.hadoop.MRJobConfig;
 import org.apache.tez.mapreduce.hadoop.MultiStageMRConfToTezTranslator;
 import org.apache.tez.mapreduce.hadoop.MultiStageMRConfigUtil;
 import org.apache.tez.mapreduce.input.SimpleInputLegacy;
+import org.apache.tez.mapreduce.partition.MRPartitioner;
 import org.apache.tez.mapreduce.processor.MapUtils;
 import org.junit.After;
 import org.junit.Before;
@@ -67,6 +70,7 @@ public class TestMapProcessor {
       workDir =
           new Path(new Path(System.getProperty("test.build.data", "/tmp")),
                    "TestMapProcessor").makeQualified(localFs);
+      LOG.info("Using workDir: " + workDir);
       MapUtils.configureLocalDirs(defaultConf, workDir.toString());
     } catch (IOException e) {
       throw new RuntimeException("init failure", e);
@@ -79,10 +83,12 @@ public class TestMapProcessor {
 
   public void setUpJobConf(JobConf job) {
     job.set(TezJobConfig.LOCAL_DIRS, workDir.toString());
+    job.set(MRConfig.LOCAL_DIR, workDir.toString());
     job.setClass(
         Constants.TEZ_ENGINE_TASK_OUTPUT_MANAGER,
         TezLocalTaskOutputFiles.class, 
         TezTaskOutput.class);
+    job.set(TezJobConfig.TEZ_ENGINE_PARTITIONER_CLASS, MRPartitioner.class.getName());
     job.setNumReduceTasks(1);
   }
 
@@ -97,7 +103,6 @@ public class TestMapProcessor {
     String vertexName = MultiStageMRConfigUtil.getInitialMapVertexName();
     JobConf jobConf = new JobConf(defaultConf);
     setUpJobConf(jobConf);
-    TezTaskOutput mapOutputs = new TezLocalTaskOutputFiles(jobConf, "TODONEWTEZ_uniqueId");
 
     Configuration conf = MultiStageMRConfToTezTranslator.convertMRToLinearTez(jobConf);
     conf.setInt(MRJobConfig.APPLICATION_ATTEMPT_ID, 0);
@@ -110,15 +115,27 @@ public class TestMapProcessor {
     job.set(TezJobConfig.TASK_LOCAL_RESOURCE_DIR, new Path(workDir,
         "localized-resources").toUri().toString());
     
-    InputSpec mapInputSpec = new InputSpec("NullSrcVertex", new InputDescriptor(SimpleInputLegacy.class.getName()), 1);
+    Path mapInput = new Path(workDir, "map0");
+    
+    
+    MapUtils.generateInputSplit(localFs, workDir, job, mapInput);
+    
+    InputSpec mapInputSpec = new InputSpec("NullSrcVertex", new InputDescriptor(SimpleInputLegacy.class.getName()), 0);
     OutputSpec mapOutputSpec = new OutputSpec("NullDestVertex", new OutputDescriptor(LocalOnFileSorterOutput.class.getName()), 1);
 
-    // TODO NEWTEZ FIXME TezUmbilical handling
-    LogicalIOProcessorRuntimeTask t = MapUtils.runMapProcessor(localFs, workDir, job, 0,
-        new Path(workDir, "map0"), (TezUmbilical) null, vertexName,
+    LogicalIOProcessorRuntimeTask task = MapUtils.createLogicalTask(localFs, workDir, job, 0,
+        new Path(workDir, "map0"), new TestUmbilical(), vertexName,
         Collections.singletonList(mapInputSpec),
         Collections.singletonList(mapOutputSpec));
-
+    
+    task.initialize();
+    task.run();
+    task.close();
+    
+    TezInputContext inputContext = task.getInputContexts().get(0);
+    TezTaskOutput mapOutputs = new TezLocalTaskOutputFiles(jobConf, inputContext.getUniqueIdentifier());
+    
+    
     // TODO NEWTEZ FIXME OutputCommitter verification
 //    MRTask mrTask = (MRTask)t.getProcessor();
 //    Assert.assertEquals(TezNullOutputCommitter.class.getName(), mrTask

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/c5a8a3c6/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/processor/reduce/TestReduceProcessor.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/processor/reduce/TestReduceProcessor.java b/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/processor/reduce/TestReduceProcessor.java
index 1d35f9b..a3abd76 100644
--- a/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/processor/reduce/TestReduceProcessor.java
+++ b/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/processor/reduce/TestReduceProcessor.java
@@ -31,20 +31,24 @@ import org.apache.hadoop.io.Text;
 import org.apache.hadoop.mapred.FileOutputFormat;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.SequenceFileOutputFormat;
+import org.apache.hadoop.mapreduce.MRConfig;
+import org.apache.hadoop.security.token.Token;
 import org.apache.tez.common.Constants;
 import org.apache.tez.common.TezJobConfig;
+import org.apache.tez.common.TezUtils;
 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.engine.api.impl.InputSpec;
 import org.apache.tez.engine.api.impl.OutputSpec;
 import org.apache.tez.engine.api.impl.TaskSpec;
-import org.apache.tez.engine.api.impl.TezUmbilical;
+import org.apache.tez.engine.common.security.JobTokenIdentifier;
 import org.apache.tez.engine.common.task.local.output.TezLocalTaskOutputFiles;
 import org.apache.tez.engine.common.task.local.output.TezTaskOutput;
 import org.apache.tez.engine.lib.input.LocalMergedInput;
 import org.apache.tez.engine.lib.output.LocalOnFileSorterOutput;
 import org.apache.tez.engine.newruntime.LogicalIOProcessorRuntimeTask;
+import org.apache.tez.mapreduce.TestUmbilical;
 import org.apache.tez.mapreduce.TezTestUtils;
 import org.apache.tez.mapreduce.hadoop.IDConverter;
 import org.apache.tez.mapreduce.hadoop.MRJobConfig;
@@ -52,8 +56,8 @@ import org.apache.tez.mapreduce.hadoop.MultiStageMRConfToTezTranslator;
 import org.apache.tez.mapreduce.hadoop.MultiStageMRConfigUtil;
 import org.apache.tez.mapreduce.input.SimpleInputLegacy;
 import org.apache.tez.mapreduce.output.SimpleOutput;
+import org.apache.tez.mapreduce.partition.MRPartitioner;
 import org.apache.tez.mapreduce.processor.MapUtils;
-import org.apache.tez.mapreduce.processor.reduce.ReduceProcessor;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
@@ -75,7 +79,7 @@ public class TestReduceProcessor {
       workDir =
           new Path(new Path(System.getProperty("test.build.data", "/tmp")),
                    "TestReduceProcessor").makeQualified(localFs);
-      
+      LOG.info("Using workDir: " + workDir);
       MapUtils.configureLocalDirs(defaultConf, workDir.toString());
     } catch (IOException e) {
       throw new RuntimeException("init failure", e);
@@ -84,10 +88,12 @@ public class TestReduceProcessor {
 
   public void setUpJobConf(JobConf job) {
     job.set(TezJobConfig.LOCAL_DIRS, workDir.toString());
+    job.set(MRConfig.LOCAL_DIR, workDir.toString());
     job.setClass(
         Constants.TEZ_ENGINE_TASK_OUTPUT_MANAGER,
         TezLocalTaskOutputFiles.class, 
         TezTaskOutput.class);
+    job.set(TezJobConfig.TEZ_ENGINE_PARTITIONER_CLASS, MRPartitioner.class.getName());
     job.setNumReduceTasks(1);
   }
 
@@ -104,10 +110,10 @@ public class TestReduceProcessor {
     String reduceVertexName = MultiStageMRConfigUtil.getFinalReduceVertexName();
     JobConf jobConf = new JobConf(defaultConf);
     setUpJobConf(jobConf);
-    TezTaskOutput mapOutputs = new TezLocalTaskOutputFiles(jobConf, "TODONEWTEZ_uniqueId");
     
     Configuration conf = MultiStageMRConfToTezTranslator.convertMRToLinearTez(jobConf);
     conf.setInt(MRJobConfig.APPLICATION_ATTEMPT_ID, 0);
+    
     Configuration mapStageConf = MultiStageMRConfigUtil.getConfForVertex(conf,
         mapVertexName);
     
@@ -116,19 +122,24 @@ public class TestReduceProcessor {
     mapConf.set(TezJobConfig.TASK_LOCAL_RESOURCE_DIR, new Path(workDir,
         "localized-resources").toUri().toString());
     
-    InputSpec mapInputSpec = new InputSpec("NullSrcVertex", new InputDescriptor(
-        SimpleInputLegacy.class.getName()), 0);
-    OutputSpec mapOutputSpec = new OutputSpec("NullDestVertex", new OutputDescriptor(
-        LocalOnFileSorterOutput.class.getName()), 1);
+    Path mapInput = new Path(workDir, "map0");
+    MapUtils.generateInputSplit(localFs, workDir, mapConf, mapInput);
+    
+    InputSpec mapInputSpec = new InputSpec("NullSrcVertex", new InputDescriptor(SimpleInputLegacy.class.getName()), 0);
+    OutputSpec mapOutputSpec = new OutputSpec("NullDestVertex", new OutputDescriptor(LocalOnFileSorterOutput.class.getName()), 1);
     // Run a map
-    // TODO NEWTEZ FIX Umbilical creation
-    MapUtils.runMapProcessor(localFs, workDir, mapConf, 0,
-        new Path(workDir, "map0"), (TezUmbilical) null, mapVertexName,
+    LogicalIOProcessorRuntimeTask mapTask = MapUtils.createLogicalTask(localFs, workDir, mapConf, 0,
+        mapInput, new TestUmbilical(), mapVertexName,
         Collections.singletonList(mapInputSpec),
         Collections.singletonList(mapOutputSpec));
 
+    mapTask.initialize();
+    mapTask.run();
+    mapTask.close();
+    
     LOG.info("Starting reduce...");
     
+    Token<JobTokenIdentifier> shuffleToken = new Token<JobTokenIdentifier>();
     
     Configuration reduceStageConf = MultiStageMRConfigUtil.getConfForVertex(conf,
         reduceVertexName);
@@ -138,7 +149,7 @@ public class TestReduceProcessor {
         "localized-resources").toUri().toString());
     FileOutputFormat.setOutputPath(reduceConf, new Path(workDir, "output"));
     ProcessorDescriptor reduceProcessorDesc = new ProcessorDescriptor(
-        ReduceProcessor.class.getName());
+        ReduceProcessor.class.getName()).setUserPayload(TezUtils.createUserPayloadFromConf(reduceConf));
     
     InputSpec reduceInputSpec = new InputSpec(mapVertexName, new InputDescriptor(LocalMergedInput.class.getName()), 1);
     OutputSpec reduceOutputSpec = new OutputSpec("NullDestinationVertex", new OutputDescriptor(SimpleOutput.class.getName()), 1);
@@ -151,28 +162,26 @@ public class TestReduceProcessor {
         reduceProcessorDesc,
         Collections.singletonList(reduceInputSpec),
         Collections.singletonList(reduceOutputSpec));
-    
-    // TODO NEWTEZ FIXME Umbilical and jobToken
+
     LogicalIOProcessorRuntimeTask task = new LogicalIOProcessorRuntimeTask(
         taskSpec,
-        1,
+        0,
         reduceConf,
-        (TezUmbilical) null,
-        null);
+        new TestUmbilical(),
+        shuffleToken);
     
     task.initialize();
     task.run();
-    
-//    MRTask mrTask = (MRTask)t.getProcessor();
-//    TODO NEWTEZ Verify the partitioner has been created
-//    Assert.assertNull(mrTask.getPartitioner());
     task.close();
     
-    // Can this be done via some utility class ? MapOutputFile derivative, or
-    // instantiating the OutputCommitter
-    
+    // MRTask mrTask = (MRTask)t.getProcessor();
+    // TODO NEWTEZ Verify the partitioner has not been created
+    // Likely not applicable anymore.
+    // Assert.assertNull(mrTask.getPartitioner());
+
+
 
-    // TODO NEWTEZ FIXME uniqueId generation and event generation (mockTaskId will not work here)
+    // Only a task commit happens, hence the data is still in the temporary directory.
     Path reduceOutputDir = new Path(new Path(workDir, "output"),
         "_temporary/0/" + IDConverter
             .toMRTaskId(TezTestUtils.getMockTaskId(0, 1, 0)));

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/c5a8a3c6/tez-yarn-client/src/main/java/org/apache/tez/mapreduce/YARNRunner.java
----------------------------------------------------------------------
diff --git a/tez-yarn-client/src/main/java/org/apache/tez/mapreduce/YARNRunner.java b/tez-yarn-client/src/main/java/org/apache/tez/mapreduce/YARNRunner.java
index 56f9035..6496b55 100644
--- a/tez-yarn-client/src/main/java/org/apache/tez/mapreduce/YARNRunner.java
+++ b/tez-yarn-client/src/main/java/org/apache/tez/mapreduce/YARNRunner.java
@@ -95,13 +95,13 @@ import org.apache.tez.dag.api.Vertex;
 import org.apache.tez.dag.api.VertexLocationHint.TaskLocationHint;
 import org.apache.tez.dag.api.client.DAGClient;
 import org.apache.tez.dag.api.client.DAGStatus;
+import org.apache.tez.engine.lib.input.ShuffledMergedInputLegacy;
 import org.apache.tez.engine.lib.output.OnFileSortedOutput;
 import org.apache.tez.mapreduce.hadoop.DeprecatedKeys;
 import org.apache.tez.mapreduce.hadoop.MRHelpers;
 import org.apache.tez.mapreduce.hadoop.MRJobConfig;
 import org.apache.tez.mapreduce.hadoop.MultiStageMRConfToTezTranslator;
 import org.apache.tez.mapreduce.hadoop.MultiStageMRConfigUtil;
-import org.apache.tez.mapreduce.input.ShuffledMergedInputLegacy;
 import org.apache.tez.mapreduce.processor.map.MapProcessor;
 import org.apache.tez.mapreduce.processor.reduce.ReduceProcessor;
 


[04/50] [abbrv] git commit: TEZ-472. Various fixes including NPE in shuffle when run MR jobs using mapred apis. (hitesh)

Posted by ss...@apache.org.
TEZ-472. Various fixes including NPE in shuffle when run MR jobs using mapred apis. (hitesh)


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

Branch: refs/heads/master
Commit: 9a7e7bccb3cdf1989945092c95c3b8e2a216946d
Parents: eb0f6ff
Author: Hitesh Shah <hi...@apache.org>
Authored: Fri Sep 20 16:12:53 2013 -0700
Committer: Hitesh Shah <hi...@apache.org>
Committed: Fri Sep 20 16:12:53 2013 -0700

----------------------------------------------------------------------
 .../engine/lib/input/ShuffledMergedInput.java   |   2 +-
 .../tez/mapreduce/examples/ExampleDriver.java   |   3 +
 .../tez/mapreduce/examples/MapredWordCount.java | 163 +++++++++++++++++++
 .../tez/mapreduce/examples/WordCount.java       |  14 +-
 .../tez/mapreduce/newoutput/SimpleOutput.java   |  23 ++-
 .../org/apache/tez/mapreduce/DAGJobStatus.java  |  12 +-
 6 files changed, 199 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/9a7e7bcc/tez-engine/src/main/java/org/apache/tez/engine/lib/input/ShuffledMergedInput.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/lib/input/ShuffledMergedInput.java b/tez-engine/src/main/java/org/apache/tez/engine/lib/input/ShuffledMergedInput.java
index 91bb6d5..eccd119 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/lib/input/ShuffledMergedInput.java
+++ b/tez-engine/src/main/java/org/apache/tez/engine/lib/input/ShuffledMergedInput.java
@@ -122,7 +122,7 @@ public class ShuffledMergedInput implements LogicalInput {
    */
   @Override
   public KVReader getReader() throws IOException {
-    if (rawIter != null) {
+    if (rawIter == null) {
       try {
         waitForInputReady();
       } catch (InterruptedException e) {

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/9a7e7bcc/tez-mapreduce-examples/src/main/java/org/apache/tez/mapreduce/examples/ExampleDriver.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce-examples/src/main/java/org/apache/tez/mapreduce/examples/ExampleDriver.java b/tez-mapreduce-examples/src/main/java/org/apache/tez/mapreduce/examples/ExampleDriver.java
index c9691ff..6c062a4 100644
--- a/tez-mapreduce-examples/src/main/java/org/apache/tez/mapreduce/examples/ExampleDriver.java
+++ b/tez-mapreduce-examples/src/main/java/org/apache/tez/mapreduce/examples/ExampleDriver.java
@@ -41,6 +41,9 @@ public class ExampleDriver {
     try {
       pgd.addClass("wordcount", WordCount.class,
           "A map/reduce program that counts the words in the input files.");
+      pgd.addClass("mapredwordcount", MapredWordCount.class,
+          "A map/reduce program that counts the words in the input files"
+         + " using the mapred apis.");
       pgd.addClass("wordcountmrrtest", WordCountMRRTest.class,
           "A map/reduce program that counts the words in the input files."
           + " Map splits on spaces. First reduce splits on \".\"");

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/9a7e7bcc/tez-mapreduce-examples/src/main/java/org/apache/tez/mapreduce/examples/MapredWordCount.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce-examples/src/main/java/org/apache/tez/mapreduce/examples/MapredWordCount.java b/tez-mapreduce-examples/src/main/java/org/apache/tez/mapreduce/examples/MapredWordCount.java
new file mode 100644
index 0000000..33aad89
--- /dev/null
+++ b/tez-mapreduce-examples/src/main/java/org/apache/tez/mapreduce/examples/MapredWordCount.java
@@ -0,0 +1,163 @@
+/**
+ * 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;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.StringTokenizer;
+
+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.IntWritable;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapred.FileInputFormat;
+import org.apache.hadoop.mapred.FileOutputFormat;
+import org.apache.hadoop.mapred.JobClient;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.MapReduceBase;
+import org.apache.hadoop.mapred.Mapper;
+import org.apache.hadoop.mapred.OutputCollector;
+import org.apache.hadoop.mapred.Reducer;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
+
+/**
+ * This is an example Hadoop Map/Reduce application using the mapred apis.
+ * It reads the text input files, breaks each line into words
+ * and counts them. The output is a locally sorted list of words and the
+ * count of how often they occurred.
+ *
+ * To run: bin/hadoop jar examples.jar wordcount
+ *            [-m <i>maps</i>] [-r <i>reduces</i>] <i>in-dir</i> <i>out-dir</i>
+ */
+public class MapredWordCount extends Configured implements Tool {
+
+  private static final Log LOG = LogFactory.getLog(MapredWordCount.class);
+
+  /**
+   * Counts the words in each line.
+   * For each line of input, break the line into words and emit them as
+   * (<b>word</b>, <b>1</b>).
+   */
+  public static class MapClass extends MapReduceBase
+    implements Mapper<LongWritable, Text, Text, IntWritable> {
+
+    private final static IntWritable one = new IntWritable(1);
+    private Text word = new Text();
+
+    public void map(LongWritable key, Text value,
+                    OutputCollector<Text, IntWritable> output,
+                    Reporter reporter) throws IOException {
+      String line = value.toString();
+      StringTokenizer itr = new StringTokenizer(line);
+      while (itr.hasMoreTokens()) {
+        word.set(itr.nextToken());
+        output.collect(word, one);
+      }
+    }
+  }
+
+  /**
+   * A reducer class that just emits the sum of the input values.
+   */
+  public static class Reduce extends MapReduceBase
+    implements Reducer<Text, IntWritable, Text, IntWritable> {
+
+    public void reduce(Text key, Iterator<IntWritable> values,
+                       OutputCollector<Text, IntWritable> output,
+                       Reporter reporter) throws IOException {
+      int sum = 0;
+      while (values.hasNext()) {
+        sum += values.next().get();
+      }
+      output.collect(key, new IntWritable(sum));
+    }
+  }
+
+  static int printUsage() {
+    System.out.println("wordcount [-m <maps>] [-r <reduces>] <input> <output>");
+    ToolRunner.printGenericCommandUsage(System.out);
+    return -1;
+  }
+
+  /**
+   * The main driver for word count map/reduce program.
+   * Invoke this method to submit the map/reduce job.
+   * @throws IOException When there is communication problems with the
+   *                     job tracker.
+   */
+  public int run(String[] args) throws Exception {
+    JobConf conf = new JobConf(getConf(), WordCount.class);
+    conf.setJobName("wordcount");
+    LOG.info("Running WordCount job using mapred apis");
+
+    // the keys are words (strings)
+    conf.setOutputKeyClass(Text.class);
+    // the values are counts (ints)
+    conf.setOutputValueClass(IntWritable.class);
+
+    conf.setMapperClass(MapClass.class);
+    conf.setCombinerClass(Reduce.class);
+    conf.setReducerClass(Reduce.class);
+
+    List<String> other_args = new ArrayList<String>();
+    for(int i=0; i < args.length; ++i) {
+      try {
+        if ("-m".equals(args[i])) {
+          conf.setNumMapTasks(Integer.parseInt(args[++i]));
+        } else if ("-r".equals(args[i])) {
+          conf.setNumReduceTasks(Integer.parseInt(args[++i]));
+        } else {
+          other_args.add(args[i]);
+        }
+      } catch (NumberFormatException except) {
+        LOG.error("Integer expected instead of " + args[i]);
+        return printUsage();
+      } catch (ArrayIndexOutOfBoundsException except) {
+        LOG.error("Required parameter missing from " + args[i-1]);
+        return printUsage();
+      }
+    }
+    // Make sure there are exactly 2 parameters left.
+    if (other_args.size() != 2) {
+      LOG.error("Wrong number of parameters: " +
+          other_args.size() + " instead of 2.");
+      return printUsage();
+    }
+    FileInputFormat.setInputPaths(conf, other_args.get(0));
+    FileOutputFormat.setOutputPath(conf, new Path(other_args.get(1)));
+
+    JobClient.runJob(conf);
+    return 0;
+  }
+
+  public static void main(String[] args) throws Exception {
+    int res = ToolRunner.run(new Configuration(),
+        new MapredWordCount(), args);
+    System.exit(res);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/9a7e7bcc/tez-mapreduce-examples/src/main/java/org/apache/tez/mapreduce/examples/WordCount.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce-examples/src/main/java/org/apache/tez/mapreduce/examples/WordCount.java b/tez-mapreduce-examples/src/main/java/org/apache/tez/mapreduce/examples/WordCount.java
index 5d02201..fc1103e 100644
--- a/tez-mapreduce-examples/src/main/java/org/apache/tez/mapreduce/examples/WordCount.java
+++ b/tez-mapreduce-examples/src/main/java/org/apache/tez/mapreduce/examples/WordCount.java
@@ -15,6 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 package org.apache.tez.mapreduce.examples;
 
 import java.io.IOException;
@@ -32,14 +33,13 @@ import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
 import org.apache.hadoop.util.GenericOptionsParser;
 
 public class WordCount {
-  
 
-  public static class TokenizerMapper 
+  public static class TokenizerMapper
        extends Mapper<Object, Text, Text, IntWritable>{
-    
+
     private final static IntWritable one = new IntWritable(1);
     private Text word = new Text();
-      
+
     public void map(Object key, Text value, Context context
                     ) throws IOException, InterruptedException {
       StringTokenizer itr = new StringTokenizer(value.toString());
@@ -49,12 +49,12 @@ public class WordCount {
       }
     }
   }
-  
-  public static class IntSumReducer 
+
+  public static class IntSumReducer
        extends Reducer<Text,IntWritable,Text,IntWritable> {
     private IntWritable result = new IntWritable();
 
-    public void reduce(Text key, Iterable<IntWritable> values, 
+    public void reduce(Text key, Iterable<IntWritable> values,
                        Context context
                        ) throws IOException, InterruptedException {
       int sum = 0;

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/9a7e7bcc/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/newoutput/SimpleOutput.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/newoutput/SimpleOutput.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/newoutput/SimpleOutput.java
index d00ffc0..1dd94e1 100644
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/newoutput/SimpleOutput.java
+++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/newoutput/SimpleOutput.java
@@ -14,6 +14,7 @@ import org.apache.hadoop.fs.FileSystem.Statistics;
 import org.apache.hadoop.mapred.FileOutputCommitter;
 import org.apache.hadoop.mapred.FileOutputFormat;
 import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.JobContext;
 import org.apache.hadoop.mapred.TaskAttemptID;
 import org.apache.hadoop.mapred.TaskID;
 import org.apache.hadoop.mapreduce.OutputCommitter;
@@ -22,7 +23,6 @@ import org.apache.hadoop.mapreduce.TaskAttemptContext;
 import org.apache.hadoop.mapreduce.TaskType;
 import org.apache.hadoop.mapreduce.lib.output.FileOutputFormatCounter;
 import org.apache.hadoop.util.ReflectionUtils;
-import org.apache.tez.common.TezJobConfig;
 import org.apache.tez.common.TezUtils;
 import org.apache.tez.common.counters.TaskCounter;
 import org.apache.tez.common.counters.TezCounter;
@@ -129,6 +129,12 @@ public class SimpleOutput implements LogicalOutput {
           (isMapperOutput ? TaskType.MAP : TaskType.REDUCE),
           outputContext.getTaskIndex()),
           outputContext.getTaskAttemptNumber());
+      jobConf.set(JobContext.TASK_ATTEMPT_ID, taskAttemptId.toString());
+      jobConf.set(JobContext.TASK_ID, taskAttemptId.getTaskID().toString());
+      jobConf.setBoolean(JobContext.TASK_ISMAP, isMapperOutput);
+      jobConf.setInt(JobContext.TASK_PARTITION,
+          taskAttemptId.getTaskID().getId());
+      jobConf.set(JobContext.ID, taskAttemptId.getJobID().toString());
 
       oldApiTaskAttemptContext =
           new org.apache.tez.mapreduce.hadoop.newmapred.TaskAttemptContextImpl(
@@ -137,7 +143,8 @@ public class SimpleOutput implements LogicalOutput {
       oldOutputFormat = jobConf.getOutputFormat();
 
       List<Statistics> matchedStats = null;
-      if (oldOutputFormat instanceof org.apache.hadoop.mapred.FileOutputFormat) {
+      if (oldOutputFormat
+          instanceof org.apache.hadoop.mapred.FileOutputFormat) {
         matchedStats =
             Utils.getFsStatistics(
                 org.apache.hadoop.mapred.FileOutputFormat.getOutputPath(
@@ -194,11 +201,19 @@ public class SimpleOutput implements LogicalOutput {
         FileOutputFormat.setWorkOutputPath(job, outputPath);
       }
     }
-    this.committer.setupTask(newApiTaskAttemptContext);
+    if (useNewApi) {
+      this.committer.setupTask(newApiTaskAttemptContext);
+    } else {
+      this.committer.setupTask(oldApiTaskAttemptContext);
+    }
   }
 
   public boolean isCommitRequired() throws IOException {
-    return committer.needsTaskCommit(newApiTaskAttemptContext);
+    if (useNewApi) {
+      return committer.needsTaskCommit(newApiTaskAttemptContext);
+    } else {
+      return committer.needsTaskCommit(oldApiTaskAttemptContext);
+    }
   }
 
   private TaskAttemptContext createTaskAttemptContext() {

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/9a7e7bcc/tez-yarn-client/src/main/java/org/apache/tez/mapreduce/DAGJobStatus.java
----------------------------------------------------------------------
diff --git a/tez-yarn-client/src/main/java/org/apache/tez/mapreduce/DAGJobStatus.java b/tez-yarn-client/src/main/java/org/apache/tez/mapreduce/DAGJobStatus.java
index 1b264c0..0b768c0 100644
--- a/tez-yarn-client/src/main/java/org/apache/tez/mapreduce/DAGJobStatus.java
+++ b/tez-yarn-client/src/main/java/org/apache/tez/mapreduce/DAGJobStatus.java
@@ -42,14 +42,14 @@ public class DAGJobStatus extends JobStatus {
   private final String jobFile;
   private final DAGStatus dagStatus;
   private final ApplicationReport report;
-  
+
   public DAGJobStatus(ApplicationReport report, DAGStatus dagStatus, String jobFile) {
     super();
     this.dagStatus = dagStatus;
     this.jobFile = jobFile;
     this.report = report;
   }
-  
+
   @Override
   protected synchronized void setMapProgress(float p) {
     throw new UnsupportedOperationException();
@@ -149,7 +149,7 @@ public class DAGJobStatus extends JobStatus {
   @Override
   public synchronized float getCleanupProgress() {
     if (dagStatus.getState() == DAGStatus.State.SUCCEEDED ||
-        dagStatus.getState() == DAGStatus.State.FAILED || 
+        dagStatus.getState() == DAGStatus.State.FAILED ||
         dagStatus.getState() == DAGStatus.State.KILLED ||
         dagStatus.getState() == DAGStatus.State.ERROR) {
       return 1.0f;
@@ -237,7 +237,7 @@ public class DAGJobStatus extends JobStatus {
   @Override
   public synchronized boolean isJobComplete() {
     return (dagStatus.getState() == DAGStatus.State.SUCCEEDED ||
-        dagStatus.getState() == DAGStatus.State.FAILED || 
+        dagStatus.getState() == DAGStatus.State.FAILED ||
         dagStatus.getState() == DAGStatus.State.KILLED ||
         dagStatus.getState() == DAGStatus.State.ERROR);
   }
@@ -369,7 +369,7 @@ public class DAGJobStatus extends JobStatus {
     buffer.append("needed-mem" + getNeededMem());
     return buffer.toString();
   }
-  
+
   private float getProgress(String vertexName) {
     Progress progress = dagStatus.getVertexProgress().get(vertexName);
     if(progress == null) {
@@ -380,7 +380,7 @@ public class DAGJobStatus extends JobStatus {
     if(totalTasks != 0) {
       return progress.getSucceededTaskCount()/totalTasks;
     }
-    return 1;
+    return 0;
   }
 
 }


[25/50] [abbrv] git commit: TEZ-443. Merge tez-dag-api and tez-engine-api into a single module - tez-api (part of TEZ-398). (sseth)

Posted by ss...@apache.org.
TEZ-443.  Merge tez-dag-api and tez-engine-api into a single module -
tez-api (part of TEZ-398). (sseth)


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

Branch: refs/heads/master
Commit: d316f723508c77eb90936a9477812195714b59a2
Parents: b4950f9
Author: Siddharth Seth <ss...@apache.org>
Authored: Mon Sep 23 10:44:42 2013 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Mon Sep 23 10:44:42 2013 -0700

----------------------------------------------------------------------
 pom.xml                                         |  10 +-
 tez-api/findbugs-exclude.xml                    |  16 +
 tez-api/pom.xml                                 |  90 +++
 .../org/apache/tez/client/AMConfiguration.java  | 100 ++++
 .../java/org/apache/tez/client/TezClient.java   | 144 +++++
 .../org/apache/tez/client/TezClientUtils.java   | 560 +++++++++++++++++++
 .../java/org/apache/tez/client/TezSession.java  | 184 ++++++
 .../tez/client/TezSessionConfiguration.java     |  57 ++
 .../org/apache/tez/common/TezJobConfig.java     | 314 +++++++++++
 .../tez/common/counters/AbstractCounter.java    |  52 ++
 .../common/counters/AbstractCounterGroup.java   | 208 +++++++
 .../tez/common/counters/AbstractCounters.java   | 385 +++++++++++++
 .../tez/common/counters/CounterGroup.java       |  32 ++
 .../tez/common/counters/CounterGroupBase.java   | 108 ++++
 .../common/counters/CounterGroupFactory.java    | 180 ++++++
 .../apache/tez/common/counters/DAGCounter.java  |  39 ++
 .../tez/common/counters/FileSystemCounter.java  |  30 +
 .../common/counters/FileSystemCounterGroup.java | 327 +++++++++++
 .../common/counters/FrameworkCounterGroup.java  | 275 +++++++++
 .../tez/common/counters/GenericCounter.java     | 109 ++++
 .../apache/tez/common/counters/JobCounter.java  |  45 ++
 .../common/counters/LimitExceededException.java |  36 ++
 .../org/apache/tez/common/counters/Limits.java  | 112 ++++
 .../tez/common/counters/ResourceBundles.java    |  94 ++++
 .../apache/tez/common/counters/TaskCounter.java |  66 +++
 .../apache/tez/common/counters/TezCounter.java  |  83 +++
 .../apache/tez/common/counters/TezCounters.java | 144 +++++
 .../main/java/org/apache/tez/dag/api/DAG.java   | 377 +++++++++++++
 .../apache/tez/dag/api/DagTypeConverters.java   | 278 +++++++++
 .../main/java/org/apache/tez/dag/api/Edge.java  |  59 ++
 .../org/apache/tez/dag/api/EdgeProperty.java    | 147 +++++
 .../org/apache/tez/dag/api/InputDescriptor.java |  32 ++
 .../apache/tez/dag/api/OutputDescriptor.java    |  32 ++
 .../apache/tez/dag/api/ProcessorDescriptor.java |  31 +
 .../apache/tez/dag/api/TezConfiguration.java    | 223 ++++++++
 .../org/apache/tez/dag/api/TezConstants.java    |  29 +
 .../apache/tez/dag/api/TezEntityDescriptor.java |  42 ++
 .../org/apache/tez/dag/api/TezException.java    |  31 +
 .../tez/dag/api/TezUncheckedException.java      |  33 ++
 .../java/org/apache/tez/dag/api/Vertex.java     | 153 +++++
 .../apache/tez/dag/api/VertexLocationHint.java  | 154 +++++
 .../apache/tez/dag/api/client/DAGClient.java    |  67 +++
 .../apache/tez/dag/api/client/DAGStatus.java    | 130 +++++
 .../org/apache/tez/dag/api/client/Progress.java |  67 +++
 .../apache/tez/dag/api/client/VertexStatus.java |  78 +++
 .../rpc/DAGClientAMProtocolBlockingPB.java      |  30 +
 .../dag/api/client/rpc/DAGClientRPCImpl.java    | 291 ++++++++++
 .../java/org/apache/tez/engine/api/Event.java   |  28 +
 .../java/org/apache/tez/engine/api/Input.java   |  71 +++
 .../tez/engine/api/LogicalIOProcessor.java      |  43 ++
 .../org/apache/tez/engine/api/LogicalInput.java |  37 ++
 .../apache/tez/engine/api/LogicalOutput.java    |  36 ++
 .../java/org/apache/tez/engine/api/Output.java  |  71 +++
 .../org/apache/tez/engine/api/Processor.java    |  55 ++
 .../java/org/apache/tez/engine/api/Reader.java  |  26 +
 .../apache/tez/engine/api/TezInputContext.java  |  32 ++
 .../apache/tez/engine/api/TezOutputContext.java |  33 ++
 .../tez/engine/api/TezProcessorContext.java     |  41 ++
 .../apache/tez/engine/api/TezTaskContext.java   | 130 +++++
 .../java/org/apache/tez/engine/api/Writer.java  |  26 +
 .../engine/api/events/DataMovementEvent.java    | 109 ++++
 .../tez/engine/api/events/InputFailedEvent.java |  89 +++
 .../api/events/InputInformationEvent.java       |  41 ++
 .../engine/api/events/InputReadErrorEvent.java  |  65 +++
 .../common/objectregistry/ObjectLifeCycle.java  |  37 ++
 .../common/objectregistry/ObjectRegistry.java   |  56 ++
 .../objectregistry/ObjectRegistryFactory.java   |  32 ++
 tez-api/src/main/proto/DAGApiRecords.proto      | 183 ++++++
 .../src/main/proto/DAGClientAMProtocol.proto    |  81 +++
 tez-api/src/main/proto/Events.proto             |  44 ++
 .../org/apache/tez/dag/api/TestDAGPlan.java     | 155 +++++
 .../org/apache/tez/dag/api/TestDAGVerify.java   | 417 ++++++++++++++
 tez-common/pom.xml                              |   2 +-
 .../java/org/apache/tez/common/Constants.java   |  57 --
 .../org/apache/tez/common/ContainerContext.java |  64 ---
 .../java/org/apache/tez/common/InputSpec.java   |  85 ---
 .../java/org/apache/tez/common/OutputSpec.java  |  84 ---
 .../org/apache/tez/common/TezJobConfig.java     | 314 -----------
 .../org/apache/tez/common/TezTaskContext.java   |  88 ---
 .../org/apache/tez/common/TezTaskStatus.java    | 104 ----
 .../tez/common/counters/AbstractCounter.java    |  52 --
 .../common/counters/AbstractCounterGroup.java   | 208 -------
 .../tez/common/counters/AbstractCounters.java   | 385 -------------
 .../tez/common/counters/CounterGroup.java       |  32 --
 .../tez/common/counters/CounterGroupBase.java   | 108 ----
 .../common/counters/CounterGroupFactory.java    | 180 ------
 .../apache/tez/common/counters/DAGCounter.java  |  39 --
 .../tez/common/counters/FileSystemCounter.java  |  30 -
 .../common/counters/FileSystemCounterGroup.java | 327 -----------
 .../common/counters/FrameworkCounterGroup.java  | 275 ---------
 .../tez/common/counters/GenericCounter.java     | 109 ----
 .../apache/tez/common/counters/JobCounter.java  |  45 --
 .../common/counters/LimitExceededException.java |  36 --
 .../org/apache/tez/common/counters/Limits.java  | 112 ----
 .../tez/common/counters/ResourceBundles.java    |  94 ----
 .../apache/tez/common/counters/TaskCounter.java |  66 ---
 .../apache/tez/common/counters/TezCounter.java  |  83 ---
 .../apache/tez/common/counters/TezCounters.java | 144 -----
 .../org/apache/tez/records/TezContainerId.java  |  78 ---
 tez-dag-api/findbugs-exclude.xml                |  16 -
 tez-dag-api/pom.xml                             |  88 ---
 .../org/apache/tez/client/AMConfiguration.java  | 100 ----
 .../java/org/apache/tez/client/TezClient.java   | 144 -----
 .../org/apache/tez/client/TezClientUtils.java   | 560 -------------------
 .../java/org/apache/tez/client/TezSession.java  | 184 ------
 .../tez/client/TezSessionConfiguration.java     |  57 --
 .../main/java/org/apache/tez/dag/api/DAG.java   | 377 -------------
 .../apache/tez/dag/api/DagTypeConverters.java   | 278 ---------
 .../main/java/org/apache/tez/dag/api/Edge.java  |  59 --
 .../org/apache/tez/dag/api/EdgeProperty.java    | 147 -----
 .../org/apache/tez/dag/api/InputDescriptor.java |  32 --
 .../apache/tez/dag/api/OutputDescriptor.java    |  32 --
 .../apache/tez/dag/api/ProcessorDescriptor.java |  31 -
 .../apache/tez/dag/api/TezConfiguration.java    | 223 --------
 .../org/apache/tez/dag/api/TezConstants.java    |  29 -
 .../apache/tez/dag/api/TezEntityDescriptor.java |  42 --
 .../org/apache/tez/dag/api/TezException.java    |  31 -
 .../tez/dag/api/TezUncheckedException.java      |  33 --
 .../java/org/apache/tez/dag/api/Vertex.java     | 153 -----
 .../apache/tez/dag/api/VertexLocationHint.java  | 154 -----
 .../apache/tez/dag/api/client/DAGClient.java    |  67 ---
 .../apache/tez/dag/api/client/DAGStatus.java    | 130 -----
 .../org/apache/tez/dag/api/client/Progress.java |  67 ---
 .../apache/tez/dag/api/client/VertexStatus.java |  78 ---
 .../rpc/DAGClientAMProtocolBlockingPB.java      |  30 -
 .../dag/api/client/rpc/DAGClientRPCImpl.java    | 291 ----------
 tez-dag-api/src/main/proto/DAGApiRecords.proto  | 183 ------
 .../src/main/proto/DAGClientAMProtocol.proto    |  81 ---
 .../org/apache/tez/dag/api/TestDAGPlan.java     | 155 -----
 .../org/apache/tez/dag/api/TestDAGVerify.java   | 417 --------------
 tez-dag/pom.xml                                 |  12 +-
 .../dag/app/TaskAttemptListenerImpTezDag.java   |  38 --
 .../org/apache/tez/dag/app/dag/EdgeManager.java |   6 +-
 .../event/TaskAttemptEventOutputConsumable.java |  36 --
 .../dag/app/dag/impl/BroadcastEdgeManager.java  |   6 +-
 .../org/apache/tez/dag/app/dag/impl/Edge.java   |   6 +-
 .../dag/app/dag/impl/OneToOneEdgeManager.java   |   6 +-
 .../app/dag/impl/ScatterGatherEdgeManager.java  |   6 +-
 .../dag/app/dag/impl/ShuffleVertexManager.java  |   6 +-
 .../apache/tez/dag/app/dag/impl/VertexImpl.java |   4 +-
 .../TezDependentTaskCompletionEvent.java        | 228 ++++++++
 ...TezTaskDependencyCompletionEventsUpdate.java |  64 +++
 .../dag/app/rm/container/TestAMContainer.java   |  14 +-
 tez-dist/src/main/assembly/tez-dist-full.xml    |   6 +-
 tez-dist/src/main/assembly/tez-dist.xml         |   6 +-
 tez-engine-api/findbugs-exclude.xml             |  16 -
 tez-engine-api/pom.xml                          |  91 ---
 .../java/org/apache/tez/engine/api/Input.java   |  83 ---
 .../java/org/apache/tez/engine/api/Master.java  |  39 --
 .../java/org/apache/tez/engine/api/Output.java  |  65 ---
 .../org/apache/tez/engine/api/Processor.java    |  62 --
 .../java/org/apache/tez/engine/api/Task.java    |  79 ---
 .../common/objectregistry/ObjectLifeCycle.java  |  37 --
 .../common/objectregistry/ObjectRegistry.java   |  56 --
 .../objectregistry/ObjectRegistryFactory.java   |  32 --
 .../org/apache/tez/engine/newapi/Event.java     |  28 -
 .../org/apache/tez/engine/newapi/Input.java     |  71 ---
 .../tez/engine/newapi/LogicalIOProcessor.java   |  43 --
 .../apache/tez/engine/newapi/LogicalInput.java  |  37 --
 .../apache/tez/engine/newapi/LogicalOutput.java |  36 --
 .../org/apache/tez/engine/newapi/Output.java    |  71 ---
 .../org/apache/tez/engine/newapi/Processor.java |  58 --
 .../org/apache/tez/engine/newapi/Reader.java    |  26 -
 .../tez/engine/newapi/TezInputContext.java      |  32 --
 .../tez/engine/newapi/TezOutputContext.java     |  33 --
 .../tez/engine/newapi/TezProcessorContext.java  |  41 --
 .../tez/engine/newapi/TezTaskContext.java       | 130 -----
 .../org/apache/tez/engine/newapi/Writer.java    |  26 -
 .../engine/newapi/events/DataMovementEvent.java | 109 ----
 .../engine/newapi/events/InputFailedEvent.java  |  89 ---
 .../newapi/events/InputInformationEvent.java    |  41 --
 .../newapi/events/InputReadErrorEvent.java      |  65 ---
 .../tez/engine/records/OutputContext.java       |  61 --
 .../TezDependentTaskCompletionEvent.java        | 228 --------
 ...TezTaskDependencyCompletionEventsUpdate.java |  64 ---
 tez-engine-api/src/main/proto/Events.proto      |  44 --
 tez-engine/pom.xml                              |   6 +-
 .../java/org/apache/tez/common/Constants.java   |  57 ++
 .../org/apache/tez/common/ContainerContext.java |  64 +++
 .../tez/common/TezTaskUmbilicalProtocol.java    |  20 -
 .../org/apache/tez/engine/api/KVReader.java     |   2 +-
 .../org/apache/tez/engine/api/KVWriter.java     |   2 +-
 .../api/events/TaskAttemptCompletedEvent.java   |   2 +-
 .../api/events/TaskAttemptFailedEvent.java      |   2 +-
 .../api/events/TaskStatusUpdateEvent.java       |   2 +-
 .../apache/tez/engine/api/impl/TezEvent.java    |  10 +-
 .../engine/api/impl/TezInputContextImpl.java    |   4 +-
 .../engine/api/impl/TezOutputContextImpl.java   |   4 +-
 .../api/impl/TezProcessorContextImpl.java       |   4 +-
 .../tez/engine/api/impl/TezTaskContextImpl.java |   2 +-
 .../broadcast/input/BroadcastInputManager.java  |   2 +-
 .../BroadcastShuffleInputEventHandler.java      |   8 +-
 .../input/BroadcastShuffleManager.java          |   6 +-
 .../broadcast/output/FileBasedKVWriter.java     |   2 +-
 .../tez/engine/common/TezEngineUtils.java       |   4 +-
 .../tez/engine/common/combine/Combiner.java     |   1 -
 .../common/localshuffle/LocalShuffle.java       |   2 +-
 .../tez/engine/common/shuffle/impl/Fetcher.java |   2 +-
 .../common/shuffle/impl/MergeManager.java       |   2 +-
 .../tez/engine/common/shuffle/impl/Shuffle.java |   4 +-
 .../shuffle/impl/ShuffleInputEventHandler.java  |  10 +-
 .../common/shuffle/impl/ShuffleScheduler.java   |   6 +-
 .../common/shuffle/server/ShuffleHandler.java   |   2 +-
 .../engine/common/sort/impl/ExternalSorter.java |   2 +-
 .../common/sort/impl/PipelinedSorter.java       |   2 +-
 .../common/sort/impl/dflt/DefaultSorter.java    |   2 +-
 .../sort/impl/dflt/InMemoryShuffleSorter.java   |   2 +-
 .../tez/engine/lib/input/LocalMergedInput.java  |   6 +-
 .../engine/lib/input/ShuffledMergedInput.java   |   6 +-
 .../lib/input/ShuffledUnorderedKVInput.java     |   8 +-
 .../engine/lib/output/InMemorySortedOutput.java |  10 +-
 .../lib/output/LocalOnFileSorterOutput.java     |   2 +-
 .../engine/lib/output/OnFileSortedOutput.java   |   8 +-
 .../lib/output/OnFileUnorderedKVOutput.java     |   8 +-
 .../LogicalIOProcessorRuntimeTask.java          |  20 +-
 tez-mapreduce/pom.xml                           |   2 +-
 .../org/apache/tez/common/TezTaskStatus.java    | 105 ++++
 .../tez/mapreduce/combine/MRCombiner.java       |   6 +-
 .../tez/mapreduce/hadoop/TezTypeConverters.java |   9 -
 .../tez/mapreduce/hadoop/mapred/MRReporter.java |   4 +-
 .../hadoop/mapreduce/MapContextImpl.java        |   2 +-
 .../mapreduce/TaskAttemptContextImpl.java       |   2 +-
 .../mapreduce/TaskInputOutputContextImpl.java   |   2 +-
 .../apache/tez/mapreduce/input/SimpleInput.java |   6 +-
 .../tez/mapreduce/output/SimpleOutput.java      |   6 +-
 .../apache/tez/mapreduce/processor/MRTask.java  |  13 +-
 .../tez/mapreduce/processor/MRTaskReporter.java |   8 +-
 .../mapreduce/processor/map/MapProcessor.java   |  10 +-
 .../processor/reduce/ReduceProcessor.java       |  10 +-
 .../tez/mapreduce/TestUmbilicalProtocol.java    |  17 -
 tez-yarn-client/pom.xml                         |   2 +-
 231 files changed, 8681 insertions(+), 9660 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index fe41471..63f17eb 100644
--- a/pom.xml
+++ b/pom.xml
@@ -90,12 +90,7 @@
     <dependencies>
       <dependency>
         <groupId>org.apache.tez</groupId>
-        <artifactId>tez-dag-api</artifactId>
-        <version>${project.version}</version>
-      </dependency>
-      <dependency>
-        <groupId>org.apache.tez</groupId>
-        <artifactId>tez-engine-api</artifactId>
+        <artifactId>tez-api</artifactId>
         <version>${project.version}</version>
       </dependency>
       <dependency>
@@ -250,8 +245,7 @@
   </dependencyManagement>
 
   <modules>
-    <module>tez-dag-api</module>
-    <module>tez-engine-api</module>
+    <module>tez-api</module>
     <module>tez-common</module>
     <module>tez-engine</module>
     <module>tez-yarn-client</module>

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/findbugs-exclude.xml
----------------------------------------------------------------------
diff --git a/tez-api/findbugs-exclude.xml b/tez-api/findbugs-exclude.xml
new file mode 100644
index 0000000..5b11308
--- /dev/null
+++ b/tez-api/findbugs-exclude.xml
@@ -0,0 +1,16 @@
+<!--
+  Licensed 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. See accompanying LICENSE file.
+-->
+<FindBugsFilter>
+
+</FindBugsFilter>

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/pom.xml
----------------------------------------------------------------------
diff --git a/tez-api/pom.xml b/tez-api/pom.xml
new file mode 100644
index 0000000..069b0d4
--- /dev/null
+++ b/tez-api/pom.xml
@@ -0,0 +1,90 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  Licensed 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. See accompanying LICENSE file.
+-->
+
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <groupId>org.apache.tez</groupId>
+    <artifactId>tez</artifactId>
+    <version>0.2.0-SNAPSHOT</version>
+  </parent>
+  <artifactId>tez-api</artifactId>
+
+  <dependencies>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-common</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-yarn-api</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-yarn-client</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-yarn-common</artifactId>
+    </dependency>
+    <dependency>
+     <groupId>com.google.protobuf</groupId>
+     <artifactId>protobuf-java</artifactId>
+    </dependency>
+  </dependencies>
+
+  <build>
+    <plugins>
+      <plugin>
+        <groupId>org.apache.rat</groupId>
+        <artifactId>apache-rat-plugin</artifactId>
+        <configuration>
+        </configuration>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.hadoop</groupId>
+        <artifactId>hadoop-maven-plugins</artifactId>
+        <executions>
+          <execution>
+            <id>compile-protoc</id>
+            <phase>generate-sources</phase>
+            <goals>
+              <goal>protoc</goal>
+            </goals>
+            <configuration>
+              <protocVersion>${protobuf.version}</protocVersion>
+              <protocCommand>${protoc.path}</protocCommand>
+              <imports>
+                <param>${basedir}/src/main/proto</param>
+              </imports>
+              <source>
+                <directory>${basedir}/src/main/proto</directory>
+                <includes>
+                  <include>DAGApiRecords.proto</include>
+                  <include>DAGClientAMProtocol.proto</include>
+                  <include>Events.proto</include>
+                </includes>
+              </source>
+              <output>${project.build.directory}/generated-sources/java</output>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
+    </plugins>
+  </build>
+
+</project>

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/src/main/java/org/apache/tez/client/AMConfiguration.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/client/AMConfiguration.java b/tez-api/src/main/java/org/apache/tez/client/AMConfiguration.java
new file mode 100644
index 0000000..f452c74
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/client/AMConfiguration.java
@@ -0,0 +1,100 @@
+/**
+ * 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.client;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.security.Credentials;
+import org.apache.hadoop.yarn.api.records.LocalResource;
+import org.apache.tez.dag.api.TezConfiguration;
+import org.apache.tez.dag.api.TezUncheckedException;
+
+public class AMConfiguration {
+
+  private final Path stagingDir;
+  private final String queueName;
+  private final Map<String, String> env;
+  private final Map<String, LocalResource> localResources;
+  private final TezConfiguration amConf;
+  private final Credentials credentials;
+
+  public AMConfiguration(String queueName, Map<String, String> env,
+      Map<String, LocalResource> localResources,
+      TezConfiguration conf, Credentials credentials) {
+    this.queueName = queueName;
+    if (conf != null) {
+      this.amConf = conf;
+    } else {
+      this.amConf = new TezConfiguration();
+    }
+
+    if (env != null) {
+      this.env = env;
+    } else {
+      this.env = new HashMap<String, String>(0);
+    }
+    this.localResources = localResources;
+    String stagingDirStr = amConf.get(TezConfiguration.TEZ_AM_STAGING_DIR);
+    if (stagingDirStr == null || stagingDirStr.isEmpty()) {
+      throw new TezUncheckedException("Staging directory for AM resources"
+          + " not specified in config"
+          + ", property=" + TezConfiguration.TEZ_AM_STAGING_DIR);
+    }
+    try {
+      FileSystem fs = FileSystem.get(amConf);
+      this.stagingDir = fs.resolvePath(new Path(stagingDirStr));
+    } catch (IOException e) {
+      throw new TezUncheckedException(e);
+    }
+    this.credentials = credentials;
+  }
+
+  public Path getStagingDir() {
+    return stagingDir;
+  }
+
+  public String getQueueName() {
+    return queueName;
+  }
+
+  public Map<String, String> getEnv() {
+    return env;
+  }
+
+  public Map<String, LocalResource> getLocalResources() {
+    return localResources;
+  }
+
+  public TezConfiguration getAMConf() {
+    return amConf;
+  }
+
+  public Credentials getCredentials() {
+    return credentials;
+  }
+
+  public void isCompatible(AMConfiguration other) {
+    // TODO implement
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/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
new file mode 100644
index 0000000..df260ec
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/client/TezClient.java
@@ -0,0 +1,144 @@
+/**
+* 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.client;
+
+import java.io.IOException;
+import java.text.NumberFormat;
+import java.util.Map;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
+import org.apache.hadoop.yarn.api.records.LocalResource;
+import org.apache.hadoop.yarn.client.api.YarnClient;
+import org.apache.hadoop.yarn.client.api.impl.YarnClientImpl;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.tez.dag.api.DAG;
+import org.apache.tez.dag.api.TezConfiguration;
+import org.apache.tez.dag.api.TezException;
+import org.apache.tez.dag.api.client.DAGClient;
+import org.apache.tez.dag.api.client.rpc.DAGClientRPCImpl;
+
+public class TezClient {
+  private static final Log LOG = LogFactory.getLog(TezClient.class);
+
+  private final TezConfiguration conf;
+  private final YarnConfiguration yarnConf;
+  private YarnClient yarnClient;
+  Map<String, LocalResource> tezJarResources = null;
+
+  /**
+   * <p>
+   * Create an instance of the TezClient which will be used to communicate with
+   * a specific instance of YARN, or TezService when that exists.
+   * </p>
+   * <p>
+   * Separate instances of TezClient should be created to communicate with
+   * different instances of YARN
+   * </p>
+   *
+   * @param conf
+   *          the configuration which will be used to establish which YARN or
+   *          Tez service instance this client is associated with.
+   */
+  public TezClient(TezConfiguration conf) {
+    this.conf = conf;
+    this.yarnConf = new YarnConfiguration(conf);
+    yarnClient = new YarnClientImpl();
+    yarnClient.init(yarnConf);
+    yarnClient.start();
+  }
+
+
+  public DAGClient submitDAGApplication(DAG dag, AMConfiguration amConfig)
+      throws TezException, IOException {
+    ApplicationId appId = createApplication();
+    return submitDAGApplication(appId, dag, amConfig);
+  }
+
+  @Private
+  // To be used only by YarnRunner
+  public DAGClient submitDAGApplication(ApplicationId appId,
+      DAG dag, AMConfiguration amConfig)
+          throws TezException, IOException {
+    try {
+      ApplicationSubmissionContext appContext =
+          TezClientUtils.createApplicationSubmissionContext(conf, appId, dag,
+              dag.getName(), amConfig, getTezJarResources());
+      LOG.info("Submitting DAG to YARN"
+          + ", applicationId=" + appId);
+      yarnClient.submitApplication(appContext);
+    } catch (YarnException e) {
+      throw new TezException(e);
+    }
+    return getDAGClient(appId);
+  }
+
+  /**
+   * Create a new YARN application
+   * @return <code>ApplicationId</code> for the new YARN application
+   * @throws YarnException
+   * @throws IOException
+   */
+  public ApplicationId createApplication() throws TezException, IOException {
+    try {
+      return yarnClient.createApplication().
+          getNewApplicationResponse().getApplicationId();
+    } catch (YarnException e) {
+      throw new TezException(e);
+    }
+  }
+
+  private synchronized Map<String, LocalResource> getTezJarResources()
+      throws IOException {
+    if (tezJarResources == null) {
+      tezJarResources = TezClientUtils.setupTezJarsLocalResources(conf);
+    }
+    return tezJarResources;
+  }
+
+  @Private
+  public DAGClient getDAGClient(ApplicationId appId)
+      throws IOException, TezException {
+      return new DAGClientRPCImpl(appId, getDefaultTezDAGID(appId),
+                                   conf);
+  }
+
+  // DO NOT CHANGE THIS. This code is replicated from TezDAGID.java
+  private static final char SEPARATOR = '_';
+  private static final String DAG = "dag";
+  private static final NumberFormat idFormat = NumberFormat.getInstance();
+  static {
+    idFormat.setGroupingUsed(false);
+    idFormat.setMinimumIntegerDigits(6);
+  }
+
+  String getDefaultTezDAGID(ApplicationId appId) {
+     return (new StringBuilder(DAG)).append(SEPARATOR).
+                   append(appId.getClusterTimestamp()).
+                   append(SEPARATOR).
+                   append(appId.getId()).
+                   append(SEPARATOR).
+                   append(idFormat.format(1)).toString();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/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
new file mode 100644
index 0000000..7c6a5ed
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java
@@ -0,0 +1,560 @@
+/**
+ * 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.client;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.TreeMap;
+import java.util.Vector;
+import java.util.Map.Entry;
+
+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.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.LocatedFileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.io.DataOutputBuffer;
+import org.apache.hadoop.ipc.ProtobufRpcEngine;
+import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.yarn.api.ApplicationConstants;
+import org.apache.hadoop.yarn.api.ApplicationConstants.Environment;
+import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.ApplicationReport;
+import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
+import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
+import org.apache.hadoop.yarn.api.records.LocalResource;
+import org.apache.hadoop.yarn.api.records.LocalResourceType;
+import org.apache.hadoop.yarn.api.records.LocalResourceVisibility;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.api.records.YarnApplicationState;
+import org.apache.hadoop.yarn.client.api.YarnClient;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.util.Apps;
+import org.apache.hadoop.yarn.util.ConverterUtils;
+import org.apache.hadoop.yarn.util.Records;
+import org.apache.log4j.Level;
+import org.apache.tez.dag.api.DAG;
+import org.apache.tez.dag.api.TezConfiguration;
+import org.apache.tez.dag.api.TezConstants;
+import org.apache.tez.dag.api.TezException;
+import org.apache.tez.dag.api.TezUncheckedException;
+import org.apache.tez.dag.api.Vertex;
+import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolBlockingPB;
+import org.apache.tez.dag.api.records.DAGProtos.ConfigurationProto;
+import org.apache.tez.dag.api.records.DAGProtos.DAGPlan;
+import org.apache.tez.dag.api.records.DAGProtos.PlanKeyValuePair;
+
+import com.google.common.annotations.VisibleForTesting;
+
+public class TezClientUtils {
+
+  private static Log LOG = LogFactory.getLog(TezClientUtils.class);
+
+  public static final FsPermission TEZ_AM_DIR_PERMISSION =
+      FsPermission.createImmutable((short) 0700); // rwx--------
+  public static final FsPermission TEZ_AM_FILE_PERMISSION =
+      FsPermission.createImmutable((short) 0644); // rw-r--r--
+
+  private static final int UTF8_CHUNK_SIZE = 16 * 1024;
+
+  /**
+   * Setup LocalResource map for Tez jars based on provided Configuration
+   * @param conf Configuration to use to access Tez jars' locations
+   * @return Map of LocalResources to use when launching Tez AM
+   * @throws IOException
+   */
+  static Map<String, LocalResource> setupTezJarsLocalResources(
+      TezConfiguration conf)
+      throws IOException {
+    Map<String, LocalResource> tezJarResources =
+        new TreeMap<String, LocalResource>();
+    if (conf.getBoolean(YarnConfiguration.IS_MINI_YARN_CLUSTER, false)) {
+      return tezJarResources;
+    }
+
+    // Add tez jars to local resource
+    String[] tezJarUris = conf.getStrings(
+        TezConfiguration.TEZ_LIB_URIS);
+    if (tezJarUris == null
+        || tezJarUris.length == 0) {
+      throw new TezUncheckedException("Invalid configuration of tez jars"
+          + ", " + TezConfiguration.TEZ_LIB_URIS
+          + " is not defined in the configurartion");
+    }
+
+    for (String tezJarUri : tezJarUris) {
+      URI uri;
+      try {
+        uri = new URI(tezJarUri.trim());
+      } catch (URISyntaxException e) {
+        String message = "Invalid URI defined in configuration for"
+            + " location of TEZ jars. providedURI=" + tezJarUri;
+        LOG.error(message);
+        throw new TezUncheckedException(message, e);
+      }
+      if (!uri.isAbsolute()) {
+        String message = "Non-absolute URI defined in configuration for"
+            + " location of TEZ jars. providedURI=" + tezJarUri;
+        LOG.error(message);
+        throw new TezUncheckedException(message);
+      }
+      Path p = new Path(uri);
+      FileSystem pathfs = p.getFileSystem(conf);
+      RemoteIterator<LocatedFileStatus> iter = pathfs.listFiles(p, false);
+      while (iter.hasNext()) {
+        LocatedFileStatus fStatus = iter.next();
+        String rsrcName = fStatus.getPath().getName();
+        // FIXME currently not checking for duplicates due to quirks
+        // in assembly generation
+        if (tezJarResources.containsKey(rsrcName)) {
+          String message = "Duplicate resource found"
+              + ", resourceName=" + rsrcName
+              + ", existingPath=" +
+              tezJarResources.get(rsrcName).getResource().toString()
+              + ", newPath=" + fStatus.getPath();
+          LOG.warn(message);
+          // throw new TezUncheckedException(message);
+        }
+        tezJarResources.put(rsrcName,
+            LocalResource.newInstance(
+                ConverterUtils.getYarnUrlFromPath(fStatus.getPath()),
+                LocalResourceType.FILE,
+                LocalResourceVisibility.PUBLIC,
+                fStatus.getLen(),
+                fStatus.getModificationTime()));
+      }
+    }
+    if (tezJarResources.isEmpty()) {
+      LOG.warn("No tez jars found in configured locations"
+          + ". Ignoring for now. Errors may occur");
+    }
+    return tezJarResources;
+  }
+
+  /**
+   * Verify or create the Staging area directory on the configured Filesystem
+   * @param stagingArea Staging area directory path
+   * @return
+   * @throws IOException
+   */
+  public static FileSystem ensureStagingDirExists(Configuration conf,
+      Path stagingArea)
+      throws IOException {
+    FileSystem fs = stagingArea.getFileSystem(conf);
+    String realUser;
+    String currentUser;
+    UserGroupInformation ugi = UserGroupInformation.getLoginUser();
+    realUser = ugi.getShortUserName();
+    currentUser = UserGroupInformation.getCurrentUser().getShortUserName();
+    if (fs.exists(stagingArea)) {
+      FileStatus fsStatus = fs.getFileStatus(stagingArea);
+      String owner = fsStatus.getOwner();
+      if (!(owner.equals(currentUser) || owner.equals(realUser))) {
+        throw new IOException("The ownership on the staging directory "
+            + stagingArea + " is not as expected. " + "It is owned by " + owner
+            + ". The directory must " + "be owned by the submitter "
+            + currentUser + " or " + "by " + realUser);
+      }
+      if (!fsStatus.getPermission().equals(TEZ_AM_DIR_PERMISSION)) {
+        LOG.info("Permissions on staging directory " + stagingArea + " are "
+            + "incorrect: " + fsStatus.getPermission()
+            + ". Fixing permissions " + "to correct value "
+            + TEZ_AM_DIR_PERMISSION);
+        fs.setPermission(stagingArea, TEZ_AM_DIR_PERMISSION);
+      }
+    } else {
+      fs.mkdirs(stagingArea, new FsPermission(TEZ_AM_DIR_PERMISSION));
+    }
+    return fs;
+  }
+
+  /**
+   * Create an ApplicationSubmissionContext to launch a Tez AM
+   * @param conf
+   * @param appId
+   * @param dag
+   * @param appStagingDir
+   * @param ts
+   * @param amQueueName
+   * @param amName
+   * @param amArgs
+   * @param amEnv
+   * @param amLocalResources
+   * @param appConf
+   * @return
+   * @throws IOException
+   * @throws YarnException
+   */
+  static ApplicationSubmissionContext createApplicationSubmissionContext(
+      Configuration conf, ApplicationId appId, DAG dag, String amName,
+      AMConfiguration amConfig,
+      Map<String, LocalResource> tezJarResources)
+          throws IOException, YarnException{
+
+    FileSystem fs = TezClientUtils.ensureStagingDirExists(conf,
+        amConfig.getStagingDir());
+
+    // Setup resource requirements
+    Resource capability = Records.newRecord(Resource.class);
+    capability.setMemory(
+        amConfig.getAMConf().getInt(TezConfiguration.TEZ_AM_RESOURCE_MEMORY_MB,
+            TezConfiguration.TEZ_AM_RESOURCE_MEMORY_MB_DEFAULT));
+    capability.setVirtualCores(
+        amConfig.getAMConf().getInt(TezConfiguration.TEZ_AM_RESOURCE_CPU_VCORES,
+            TezConfiguration.TEZ_AM_RESOURCE_CPU_VCORES_DEFAULT));
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("AppMaster capability = " + capability);
+    }
+
+    ByteBuffer securityTokens = null;
+    // Setup security tokens
+    if (amConfig.getCredentials() != null) {
+      DataOutputBuffer dob = new DataOutputBuffer();
+      amConfig.getCredentials().writeTokenStorageToStream(dob);
+      securityTokens = ByteBuffer.wrap(dob.getData(), 0,
+          dob.getLength());
+    }
+
+    // Setup the command to run the AM
+    List<String> vargs = new ArrayList<String>(8);
+    vargs.add(Environment.JAVA_HOME.$() + "/bin/java");
+
+    String amLogLevel = amConfig.getAMConf().get(
+        TezConfiguration.TEZ_AM_LOG_LEVEL,
+        TezConfiguration.TEZ_AM_LOG_LEVEL_DEFAULT);
+    addLog4jSystemProperties(amLogLevel, vargs);
+
+    vargs.add(amConfig.getAMConf().get(TezConfiguration.TEZ_AM_JAVA_OPTS,
+        TezConfiguration.DEFAULT_TEZ_AM_JAVA_OPTS));
+
+    vargs.add(TezConfiguration.TEZ_APPLICATION_MASTER_CLASS);
+    vargs.add("1>" + ApplicationConstants.LOG_DIR_EXPANSION_VAR +
+        File.separator + ApplicationConstants.STDOUT);
+    vargs.add("2>" + ApplicationConstants.LOG_DIR_EXPANSION_VAR +
+        File.separator + ApplicationConstants.STDERR);
+
+
+    Vector<String> vargsFinal = new Vector<String>(8);
+    // Final command
+    StringBuilder mergedCommand = new StringBuilder();
+    for (CharSequence str : vargs) {
+      mergedCommand.append(str).append(" ");
+    }
+    vargsFinal.add(mergedCommand.toString());
+
+    LOG.debug("Command to launch container for ApplicationMaster is : "
+        + mergedCommand);
+
+    // Setup the CLASSPATH in environment
+    // i.e. add { Hadoop jars, job jar, CWD } to classpath.
+    Map<String, String> environment = new HashMap<String, String>();
+
+    boolean isMiniCluster =
+        conf.getBoolean(YarnConfiguration.IS_MINI_YARN_CLUSTER, false);
+    if (isMiniCluster) {
+      Apps.addToEnvironment(environment, Environment.CLASSPATH.name(),
+          System.getProperty("java.class.path"));
+    }
+
+    Apps.addToEnvironment(environment,
+        Environment.CLASSPATH.name(),
+        Environment.PWD.$());
+
+    Apps.addToEnvironment(environment,
+        Environment.CLASSPATH.name(),
+        Environment.PWD.$() + File.separator + "*");
+
+    // Add YARN/COMMON/HDFS jars to path
+    if (!isMiniCluster) {
+      for (String c : conf.getStrings(
+          YarnConfiguration.YARN_APPLICATION_CLASSPATH,
+          YarnConfiguration.DEFAULT_YARN_APPLICATION_CLASSPATH)) {
+        Apps.addToEnvironment(environment, Environment.CLASSPATH.name(),
+            c.trim());
+      }
+    }
+
+    if (amConfig.getEnv() != null) {
+      for (Map.Entry<String, String> entry : amConfig.getEnv().entrySet()) {
+        Apps.addToEnvironment(environment, entry.getKey(), entry.getValue());
+      }
+    }
+
+    Map<String, LocalResource> localResources =
+        new TreeMap<String, LocalResource>();
+
+    if (amConfig.getLocalResources() != null) {
+      localResources.putAll(amConfig.getLocalResources());
+    }
+    localResources.putAll(tezJarResources);
+
+    // emit conf as PB file
+    Configuration finalTezConf = createFinalTezConfForApp(amConfig.getAMConf());
+    Path binaryConfPath =  new Path(amConfig.getStagingDir(),
+        TezConfiguration.TEZ_PB_BINARY_CONF_NAME + "." + appId.toString());
+    FSDataOutputStream amConfPBOutBinaryStream = null;
+    try {
+      ConfigurationProto.Builder confProtoBuilder =
+          ConfigurationProto.newBuilder();
+      Iterator<Entry<String, String>> iter = finalTezConf.iterator();
+      while (iter.hasNext()) {
+        Entry<String, String> entry = iter.next();
+        PlanKeyValuePair.Builder kvp = PlanKeyValuePair.newBuilder();
+        kvp.setKey(entry.getKey());
+        kvp.setValue(entry.getValue());
+        confProtoBuilder.addConfKeyValues(kvp);
+      }
+      //binary output
+      amConfPBOutBinaryStream = FileSystem.create(fs, binaryConfPath,
+          new FsPermission(TEZ_AM_FILE_PERMISSION));
+      confProtoBuilder.build().writeTo(amConfPBOutBinaryStream);
+    } finally {
+      if(amConfPBOutBinaryStream != null){
+        amConfPBOutBinaryStream.close();
+      }
+    }
+
+    LocalResource binaryConfLRsrc =
+        TezClientUtils.createLocalResource(fs,
+            binaryConfPath, LocalResourceType.FILE,
+            LocalResourceVisibility.APPLICATION);
+    localResources.put(TezConfiguration.TEZ_PB_BINARY_CONF_NAME,
+        binaryConfLRsrc);
+
+    if(dag != null) {
+      // Add tez jars to vertices too
+      for (Vertex v : dag.getVertices()) {
+        v.getTaskLocalResources().putAll(tezJarResources);
+        v.getTaskLocalResources().put(TezConfiguration.TEZ_PB_BINARY_CONF_NAME,
+            binaryConfLRsrc);
+      }
+
+      // emit protobuf DAG file style
+      Path binaryPath =  new Path(amConfig.getStagingDir(),
+          TezConfiguration.TEZ_PB_PLAN_BINARY_NAME + "." + appId.toString());
+      amConfig.getAMConf().set(TezConfiguration.TEZ_AM_PLAN_REMOTE_PATH,
+          binaryPath.toUri().toString());
+
+      DAGPlan dagPB = dag.createDag(null);
+
+      FSDataOutputStream dagPBOutBinaryStream = null;
+
+      try {
+        //binary output
+        dagPBOutBinaryStream = FileSystem.create(fs, binaryPath,
+            new FsPermission(TEZ_AM_FILE_PERMISSION));
+        dagPB.writeTo(dagPBOutBinaryStream);
+      } finally {
+        if(dagPBOutBinaryStream != null){
+          dagPBOutBinaryStream.close();
+        }
+      }
+
+      localResources.put(TezConfiguration.TEZ_PB_PLAN_BINARY_NAME,
+          TezClientUtils.createLocalResource(fs,
+              binaryPath, LocalResourceType.FILE,
+              LocalResourceVisibility.APPLICATION));
+
+      if (Level.DEBUG.isGreaterOrEqual(Level.toLevel(amLogLevel))) {
+        Path textPath = localizeDagPlanAsText(dagPB, fs,
+            amConfig.getStagingDir(), appId);
+        localResources.put(TezConfiguration.TEZ_PB_PLAN_TEXT_NAME,
+            TezClientUtils.createLocalResource(fs,
+                textPath, LocalResourceType.FILE,
+                LocalResourceVisibility.APPLICATION));
+      }
+    } else {
+      Apps.addToEnvironment(environment,
+          TezConstants.TEZ_AM_IS_SESSION_ENV, "set");
+    }
+
+    Map<ApplicationAccessType, String> acls
+        = new HashMap<ApplicationAccessType, String>();
+
+    // Setup ContainerLaunchContext for AM container
+    ContainerLaunchContext amContainer =
+        ContainerLaunchContext.newInstance(localResources, environment,
+            vargsFinal, null, securityTokens, acls);
+
+    // Set up the ApplicationSubmissionContext
+    ApplicationSubmissionContext appContext = Records
+        .newRecord(ApplicationSubmissionContext.class);
+
+    appContext.setApplicationType(TezConfiguration.TEZ_APPLICATION_TYPE);
+    appContext.setApplicationId(appId);
+    appContext.setResource(capability);
+    appContext.setQueue(amConfig.getQueueName());
+    appContext.setApplicationName(amName);
+    appContext.setCancelTokensWhenComplete(amConfig.getAMConf().getBoolean(
+        TezConfiguration.TEZ_AM_CANCEL_DELEGATION_TOKEN,
+        TezConfiguration.TEZ_AM_CANCEL_DELEGATION_TOKEN_DEFAULT));
+    appContext.setAMContainerSpec(amContainer);
+
+    return appContext;
+
+  }
+
+  @VisibleForTesting
+  static void addLog4jSystemProperties(String logLevel,
+      List<String> vargs) {
+    vargs.add("-Dlog4j.configuration="
+        + TezConfiguration.TEZ_CONTAINER_LOG4J_PROPERTIES_FILE);
+    vargs.add("-D" + YarnConfiguration.YARN_APP_CONTAINER_LOG_DIR + "="
+        + ApplicationConstants.LOG_DIR_EXPANSION_VAR);
+    vargs.add("-D" + TezConfiguration.TEZ_ROOT_LOGGER_NAME + "=" + logLevel
+        + "," + TezConfiguration.TEZ_CONTAINER_LOGGER_NAME);
+  }
+
+  static Configuration createFinalTezConfForApp(TezConfiguration amConf) {
+    Configuration conf = new Configuration(false);
+    conf.setQuietMode(true);
+
+    assert amConf != null;
+    Iterator<Entry<String, String>> iter = amConf.iterator();
+    while (iter.hasNext()) {
+      Entry<String, String> entry = iter.next();
+      // Copy all tez config parameters.
+      if (entry.getKey().startsWith(TezConfiguration.TEZ_PREFIX)) {
+        conf.set(entry.getKey(), entry.getValue());
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Adding tez dag am parameter: " + entry.getKey()
+              + ", with value: " + entry.getValue());
+        }
+      }
+    }
+    return conf;
+  }
+
+  /**
+   * Helper function to create a YARN LocalResource
+   * @param fs FileSystem object
+   * @param p Path of resource to localize
+   * @param type LocalResource Type
+   * @return
+   * @throws IOException
+   */
+  static LocalResource createLocalResource(FileSystem fs, Path p,
+      LocalResourceType type,
+      LocalResourceVisibility visibility) throws IOException {
+    LocalResource rsrc = Records.newRecord(LocalResource.class);
+    FileStatus rsrcStat = fs.getFileStatus(p);
+    rsrc.setResource(ConverterUtils.getYarnUrlFromPath(fs.resolvePath(rsrcStat
+        .getPath())));
+    rsrc.setSize(rsrcStat.getLen());
+    rsrc.setTimestamp(rsrcStat.getModificationTime());
+    rsrc.setType(type);
+    rsrc.setVisibility(visibility);
+    return rsrc;
+  }
+
+  private static Path localizeDagPlanAsText(DAGPlan dagPB, FileSystem fs,
+      Path appStagingDir, ApplicationId appId) throws IOException {
+    Path textPath = new Path(appStagingDir,
+        TezConfiguration.TEZ_PB_PLAN_TEXT_NAME + "." + appId.toString());
+    FSDataOutputStream dagPBOutTextStream = null;
+    try {
+      dagPBOutTextStream = FileSystem.create(fs, textPath, new FsPermission(
+          TEZ_AM_FILE_PERMISSION));
+      String dagPBStr = dagPB.toString();
+      int dagPBStrLen = dagPBStr.length();
+      if (dagPBStrLen <= UTF8_CHUNK_SIZE) {
+        dagPBOutTextStream.writeUTF(dagPBStr);
+      } else {
+        int startIndex = 0;
+        while (startIndex < dagPBStrLen) {
+          int endIndex = startIndex + UTF8_CHUNK_SIZE;
+          if (endIndex > dagPBStrLen) {
+            endIndex = dagPBStrLen;
+          }
+          dagPBOutTextStream.writeUTF(dagPBStr.substring(startIndex, endIndex));
+          startIndex += UTF8_CHUNK_SIZE;
+        }
+      }
+    } finally {
+      if (dagPBOutTextStream != null) {
+        dagPBOutTextStream.close();
+      }
+    }
+    return textPath;
+  }
+
+  static DAGClientAMProtocolBlockingPB getAMProxy(YarnClient yarnClient,
+      Configuration conf,
+      ApplicationId applicationId) throws TezException, IOException {
+    ApplicationReport appReport;
+    try {
+      appReport = yarnClient.getApplicationReport(
+          applicationId);
+
+      if(appReport == null) {
+        throw new TezUncheckedException("Could not retrieve application report"
+            + " from YARN, applicationId=" + applicationId);
+      }
+      YarnApplicationState appState = appReport.getYarnApplicationState();
+      if(appState != YarnApplicationState.RUNNING) {
+        if (appState == YarnApplicationState.FINISHED
+            || appState == YarnApplicationState.KILLED
+            || appState == YarnApplicationState.FAILED) {
+          throw new TezUncheckedException("Application not running"
+              + ", applicationId=" + applicationId
+              + ", yarnApplicationState=" + appReport.getYarnApplicationState()
+              + ", finalApplicationStatus="
+              + appReport.getFinalApplicationStatus()
+              + ", trackingUrl=" + appReport.getTrackingUrl());
+        }
+        return null;
+      }
+    } catch (YarnException e) {
+      throw new TezException(e);
+    }
+    return getAMProxy(conf, appReport.getHost(), appReport.getRpcPort());
+  }
+
+  static DAGClientAMProtocolBlockingPB getAMProxy(Configuration conf,
+      String amHost, int amRpcPort) throws IOException {
+    InetSocketAddress addr = new InetSocketAddress(amHost,
+        amRpcPort);
+
+    RPC.setProtocolEngine(conf, DAGClientAMProtocolBlockingPB.class,
+        ProtobufRpcEngine.class);
+    DAGClientAMProtocolBlockingPB proxy =
+        (DAGClientAMProtocolBlockingPB) RPC.getProxy(
+            DAGClientAMProtocolBlockingPB.class, 0, addr, conf);
+    return proxy;
+  }
+
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/src/main/java/org/apache/tez/client/TezSession.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/client/TezSession.java b/tez-api/src/main/java/org/apache/tez/client/TezSession.java
new file mode 100644
index 0000000..acf523d
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/client/TezSession.java
@@ -0,0 +1,184 @@
+/**
+ * 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.client;
+
+import java.io.IOException;
+import java.util.Map;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
+import org.apache.hadoop.yarn.api.records.LocalResource;
+import org.apache.hadoop.yarn.client.api.YarnClient;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.tez.dag.api.DAG;
+import org.apache.tez.dag.api.TezConfiguration;
+import org.apache.tez.dag.api.TezException;
+import org.apache.tez.dag.api.TezUncheckedException;
+import org.apache.tez.dag.api.Vertex;
+import org.apache.tez.dag.api.client.DAGClient;
+import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolBlockingPB;
+import org.apache.tez.dag.api.client.rpc.DAGClientRPCImpl;
+import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolRPC.ShutdownSessionRequestProto;
+import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolRPC.SubmitDAGRequestProto;
+import org.apache.tez.dag.api.records.DAGProtos.DAGPlan;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.protobuf.ServiceException;
+
+public class TezSession {
+
+  private static final Log LOG = LogFactory.getLog(TezSession.class);
+
+  private final String sessionName;
+  private ApplicationId applicationId;
+  private LocalResource tezConfPBLRsrc = null;
+  private final TezSessionConfiguration sessionConfig;
+  private YarnClient yarnClient;
+  private Map<String, LocalResource> tezJarResources;
+  private boolean sessionStarted = false;
+
+  public TezSession(String sessionName,
+      ApplicationId applicationId,
+      TezSessionConfiguration sessionConfig) {
+    this.sessionName = sessionName;
+    this.sessionConfig = sessionConfig;
+    this.applicationId = applicationId;
+  }
+
+  public TezSession(String sessionName,
+      TezSessionConfiguration sessionConfig) {
+    this(sessionName, null, sessionConfig);
+  }
+
+  public synchronized void start() throws TezException, IOException {
+    yarnClient = YarnClient.createYarnClient();
+    yarnClient.init(sessionConfig.getYarnConfiguration());
+    yarnClient.start();
+
+    tezJarResources = TezClientUtils.setupTezJarsLocalResources(
+        sessionConfig.getTezConfiguration());
+
+    try {
+      if (applicationId == null) {
+        applicationId = yarnClient.createApplication().
+            getNewApplicationResponse().getApplicationId();
+      }
+
+      ApplicationSubmissionContext appContext =
+          TezClientUtils.createApplicationSubmissionContext(
+              sessionConfig.getTezConfiguration(), applicationId,
+              null, sessionName, sessionConfig.getAMConfiguration(),
+              tezJarResources);
+      tezConfPBLRsrc = appContext.getAMContainerSpec().getLocalResources().get(
+          TezConfiguration.TEZ_PB_BINARY_CONF_NAME);
+      yarnClient.submitApplication(appContext);
+    } catch (YarnException e) {
+      throw new TezException(e);
+    }
+    sessionStarted = true;
+  }
+
+  public synchronized DAGClient submitDAG(DAG dag)
+      throws TezException, IOException {
+    if (!sessionStarted) {
+      throw new TezUncheckedException("Session not started");
+    }
+
+    String dagId = null;
+    LOG.info("Submitting dag to TezSession"
+        + ", sessionName=" + sessionName
+        + ", applicationId=" + applicationId);
+    // Add tez jars to vertices too
+    for (Vertex v : dag.getVertices()) {
+      v.getTaskLocalResources().putAll(tezJarResources);
+      if (null != tezConfPBLRsrc) {
+        v.getTaskLocalResources().put(TezConfiguration.TEZ_PB_BINARY_CONF_NAME,
+            tezConfPBLRsrc);
+      }
+    }
+    DAGPlan dagPlan = dag.createDag(sessionConfig.getTezConfiguration());
+    SubmitDAGRequestProto requestProto =
+        SubmitDAGRequestProto.newBuilder().setDAGPlan(dagPlan).build();
+
+    DAGClientAMProtocolBlockingPB proxy;
+    while (true) {
+      proxy = TezClientUtils.getAMProxy(yarnClient,
+          sessionConfig.getYarnConfiguration(), applicationId);
+      if (proxy != null) {
+        break;
+      }
+      try {
+        Thread.sleep(100l);
+      } catch (InterruptedException e) {
+        // Ignore
+      }
+    }
+
+    try {
+      dagId = proxy.submitDAG(null, requestProto).getDagId();
+    } catch (ServiceException e) {
+      throw new TezException(e);
+    }
+    LOG.info("Submitted dag to TezSession"
+        + ", sessionName=" + sessionName
+        + ", applicationId=" + applicationId
+        + ", dagId=" + dagId);
+    return new DAGClientRPCImpl(applicationId, dagId,
+        sessionConfig.getTezConfiguration());
+  }
+
+  public synchronized void stop() throws TezException, IOException {
+    LOG.info("Shutting down Tez Session"
+        + ", sessionName=" + sessionName
+        + ", applicationId=" + applicationId);
+    DAGClientAMProtocolBlockingPB proxy = TezClientUtils.getAMProxy(yarnClient,
+        sessionConfig.getYarnConfiguration(), applicationId);
+    if (proxy != null) {
+      try {
+        ShutdownSessionRequestProto request =
+            ShutdownSessionRequestProto.newBuilder().build();
+        proxy.shutdownSession(null, request);
+        return;
+      } catch (ServiceException e) {
+        LOG.info("Failed to shutdown Tez Session via proxy", e);
+      }
+    }
+    LOG.info("Could not connect to AM, killing session via YARN"
+        + ", sessionName=" + sessionName
+        + ", applicationId=" + applicationId);
+    try {
+      yarnClient.killApplication(applicationId);
+    } catch (YarnException e) {
+      throw new TezException(e);
+    }
+  }
+
+  public String getSessionName() {
+    return sessionName;
+  }
+
+  @Private
+  @VisibleForTesting
+  public synchronized ApplicationId getApplicationId() {
+    return applicationId;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/src/main/java/org/apache/tez/client/TezSessionConfiguration.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/client/TezSessionConfiguration.java b/tez-api/src/main/java/org/apache/tez/client/TezSessionConfiguration.java
new file mode 100644
index 0000000..61ca60b
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/client/TezSessionConfiguration.java
@@ -0,0 +1,57 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.client;
+
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.tez.dag.api.TezConfiguration;
+
+public class TezSessionConfiguration {
+
+  private final AMConfiguration amConfiguration;
+  private final YarnConfiguration yarnConfig;
+  private final TezConfiguration tezConfig;
+
+  public TezSessionConfiguration(AMConfiguration amConfiguration,
+      TezConfiguration tezConfig) {
+    this.amConfiguration = amConfiguration;
+    this.tezConfig = tezConfig;
+    this.yarnConfig = new YarnConfiguration(tezConfig);
+  }
+
+  TezSessionConfiguration(AMConfiguration amConfiguration,
+      TezConfiguration tezConfig,
+      YarnConfiguration yarnConf) {
+    this.amConfiguration = amConfiguration;
+    this.tezConfig = tezConfig;
+    this.yarnConfig = yarnConf;
+  }
+
+  public AMConfiguration getAMConfiguration() {
+    return amConfiguration;
+  }
+
+  public YarnConfiguration getYarnConfiguration() {
+    return yarnConfig;
+  }
+
+  public TezConfiguration getTezConfiguration() {
+    return tezConfig;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/src/main/java/org/apache/tez/common/TezJobConfig.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/common/TezJobConfig.java b/tez-api/src/main/java/org/apache/tez/common/TezJobConfig.java
new file mode 100644
index 0000000..7c4540c
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/common/TezJobConfig.java
@@ -0,0 +1,314 @@
+/**
+ * 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.common;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Evolving;
+
+
+/**
+ * Meant for user configurable job properties. For others look at {@link Constants}
+ *
+ */
+
+// TODO EVENTUALLY A description for each property.
+@Private
+@Evolving
+public class TezJobConfig {
+
+
+
+
+  /** The number of milliseconds between progress reports. */
+  public static final int PROGRESS_INTERVAL = 3000;
+
+  public static final long DEFAULT_COMBINE_RECORDS_BEFORE_PROGRESS = 10000;
+
+  /**
+   * Configuration key to enable/disable IFile readahead.
+   */
+  public static final String TEZ_ENGINE_IFILE_READAHEAD =
+      "tez.engine.ifile.readahead";
+  public static final boolean DEFAULT_TEZ_ENGINE_IFILE_READAHEAD = true;
+
+  /**
+   * Configuration key to set the IFile readahead length in bytes.
+   */
+  public static final String TEZ_ENGINE_IFILE_READAHEAD_BYTES =
+      "tez.engine.ifile.readahead.bytes";
+  public static final int DEFAULT_TEZ_ENGINE_IFILE_READAHEAD_BYTES =
+      4 * 1024 * 1024;
+
+  /**
+   * 
+   */
+  public static final String RECORDS_BEFORE_PROGRESS = 
+      "tez.task.merge.progress.records";
+  public static final long DEFAULT_RECORDS_BEFORE_PROGRESS = 10000; 
+
+  /**
+   * List of directories avialble to the engine. 
+   */
+  public static final String LOCAL_DIRS = "tez.engine.local.dirs";
+  public static final String DEFAULT_LOCAL_DIRS = "/tmp";
+
+  /**
+   * One local dir for the speicfic job.
+   */
+  public static final String JOB_LOCAL_DIR = "tez.engine.job.local.dir";
+  
+  /**
+   * The directory which contains the localized files for this task.
+   */
+  @Private
+  public static final String TASK_LOCAL_RESOURCE_DIR = "tez.engine.task-local-resource.dir";
+  public static final String DEFAULT_TASK_LOCAL_RESOURCE_DIR = "/tmp";
+  
+  public static final String TEZ_TASK_WORKING_DIR = "tez.engine.task.working.dir";
+
+  /**
+   * 
+   */
+  public static final String TEZ_ENGINE_IO_SORT_FACTOR = 
+      "tez.engine.io.sort.factor";
+  public static final int DEFAULT_TEZ_ENGINE_IO_SORT_FACTOR = 100;
+
+  /**
+   * 
+   */
+  public static final String TEZ_ENGINE_SORT_SPILL_PERCENT = 
+      "tez.engine.sort.spill.percent";
+  public static float DEFAULT_TEZ_ENGINE_SORT_SPILL_PERCENT = 0.8f; 
+
+  /**
+   * 
+   */
+  public static final String TEZ_ENGINE_IO_SORT_MB = "tez.engine.io.sort.mb";
+  public static final int DEFAULT_TEZ_ENGINE_IO_SORT_MB = 100;
+
+  /**
+   * 
+   */
+  public static final String TEZ_ENGINE_INDEX_CACHE_MEMORY_LIMIT_BYTES = 
+      "tez.engine.index.cache.memory.limit.bytes";
+  public static final int DEFAULT_TEZ_ENGINE_INDEX_CACHE_MEMORY_LIMIT_BYTES = 
+      1024 * 1024;
+
+  /**
+   * 
+   */
+  public static final String TEZ_ENGINE_COMBINE_MIN_SPILLS = 
+      "tez.engine.combine.min.spills";
+  public static final int  DEFAULT_TEZ_ENGINE_COMBINE_MIN_SPILLS = 3;
+  
+  /**
+   * 
+   */
+  public static final String TEZ_ENGINE_SORT_THREADS = 
+	      "tez.engine.sort.threads";
+  public static final int DEFAULT_TEZ_ENGINE_SORT_THREADS = 1;
+
+  /**
+   * Specifies a partitioner class, which is used in Tez engine components like OnFileSortedOutput
+   */
+  public static final String TEZ_ENGINE_PARTITIONER_CLASS = "tez.engine.partitioner.class";
+  
+  /**
+   * Specifies a combiner class (primarily for Shuffle)
+   */
+  public static final String TEZ_ENGINE_COMBINER_CLASS = "tez.engine.combiner.class";
+  
+  public static final String TEZ_ENGINE_NUM_EXPECTED_PARTITIONS = "tez.engine.num.expected.partitions";
+  
+  /**
+   * 
+   */
+  public static final String COUNTERS_MAX_KEY = "tez.engine.job.counters.max";
+  public static final int COUNTERS_MAX_DEFAULT = 120;
+
+  /**
+   * 
+   */
+  public static final String COUNTER_GROUP_NAME_MAX_KEY = "tez.engine.job.counters.group.name.max";
+  public static final int COUNTER_GROUP_NAME_MAX_DEFAULT = 128;
+
+  /**
+   * 
+   */
+  public static final String COUNTER_NAME_MAX_KEY = "tez.engine.job.counters.counter.name.max";
+  public static final int COUNTER_NAME_MAX_DEFAULT = 64;
+
+  /**
+   * 
+   */
+  public static final String COUNTER_GROUPS_MAX_KEY = "tez.engine.job.counters.groups.max";
+  public static final int COUNTER_GROUPS_MAX_DEFAULT = 50;
+
+  
+  /**
+   * Temporary interface for MR only (not chained Tez) to indicate whether
+   * in-memory shuffle should be used.
+   */
+  @Private
+  public static final String TEZ_ENGINE_SHUFFLE_USE_IN_MEMORY =
+      "tez.engine.shuffle.use.in-memory";
+  public static final boolean DEFAULT_TEZ_ENGINE_SHUFFLE_USE_IN_MEMORY = false;
+
+  // TODO NEWTEZ Remove these config parameters. Will be part of an event.
+  @Private
+  public static final String TEZ_ENGINE_SHUFFLE_PARTITION_RANGE = 
+      "tez.engine.shuffle.partition-range";
+  public static int TEZ_ENGINE_SHUFFLE_PARTITION_RANGE_DEFAULT = 1;
+
+  /**
+   * 
+   */
+  public static final String TEZ_ENGINE_SHUFFLE_PARALLEL_COPIES = 
+      "tez.engine.shuffle.parallel.copies";
+  public static final int DEFAULT_TEZ_ENGINE_SHUFFLE_PARALLEL_COPIES = 20;
+
+  /**
+   * TODO Is this user configurable.
+   */
+  public static final String TEZ_ENGINE_METRICS_SESSION_ID = 
+      "tez.engine.metrics.session.id";
+  public static final String DEFAULT_TEZ_ENGINE_METRICS_SESSION_ID = "";
+
+  /**
+   * 
+   */
+  public static final String TEZ_ENGINE_SHUFFLE_FETCH_FAILURES = 
+      "tez.engine.shuffle.fetch.failures.limit";
+  public final static int DEFAULT_TEZ_ENGINE_SHUFFLE_FETCH_FAILURES_LIMIT = 10;
+
+  /**
+   * 
+   */
+  public static final String TEZ_ENGINE_SHUFFLE_NOTIFY_READERROR = 
+      "tez.engine.shuffle.notify.readerror";
+  public static final boolean DEFAULT_TEZ_ENGINE_SHUFFLE_NOTIFY_READERROR = true;
+
+  /**
+   * 
+   */
+  public static final String TEZ_ENGINE_SHUFFLE_CONNECT_TIMEOUT = 
+      "tez.engine.shuffle.connect.timeout";
+  public static final int DEFAULT_TEZ_ENGINE_SHUFFLE_STALLED_COPY_TIMEOUT = 
+      3 * 60 * 1000;
+
+  /**
+   * 
+   */
+  public static final String TEZ_ENGINE_SHUFFLE_READ_TIMEOUT = "tez.engine.shuffle.read.timeout";
+  public final static int DEFAULT_TEZ_ENGINE_SHUFFLE_READ_TIMEOUT = 
+      3 * 60 * 1000;
+
+  /**
+   * 
+   */
+  public static final String TEZ_ENGINE_SHUFFLE_ENABLE_SSL = 
+      "tez.engine.shuffle.ssl.enable";
+  public static final boolean DEFAULT_TEZ_ENGINE_SHUFFLE_ENABLE_SSL = false;
+
+  /**
+   * 
+   */
+  public static final String TEZ_ENGINE_SHUFFLE_INPUT_BUFFER_PERCENT = 
+      "tez.engine.shuffle.input.buffer.percent";
+  public static final float DEFAULT_TEZ_ENGINE_SHUFFLE_INPUT_BUFFER_PERCENT =
+      0.90f;
+
+  /**
+   * 
+   */
+  public static final String TEZ_ENGINE_SHUFFLE_MEMORY_LIMIT_PERCENT = 
+      "tez.engine.shuffle.memory.limit.percent";
+  public static final float DEFAULT_TEZ_ENGINE_SHUFFLE_MEMORY_LIMIT_PERCENT = 
+      0.25f;
+
+  /**
+   * 
+   */
+  public static final String TEZ_ENGINE_SHUFFLE_MERGE_PERCENT = 
+      "tez.engine.shuffle.merge.percent";
+  public static final float DEFAULT_TEZ_ENGINE_SHUFFLE_MERGE_PERCENT = 0.90f;
+  
+  /**
+   * TODO TEZAM3 default value ?
+   */
+  public static final String TEZ_ENGINE_SHUFFLE_MEMTOMEM_SEGMENTS = 
+      "tez.engine.shuffle.memory-to-memory.segments";
+
+  /**
+   * 
+   */
+  public static final String TEZ_ENGINE_SHUFFLE_ENABLE_MEMTOMEM = 
+      "tez.engine.shuffle.memory-to-memory.enable";
+  public static final boolean DEFAULT_TEZ_ENGINE_SHUFFLE_ENABLE_MEMTOMEM = 
+      false;
+
+  /**
+   * 
+   */
+  public static final String TEZ_ENGINE_INPUT_BUFFER_PERCENT = 
+      "tez.engine.task.input.buffer.percent";
+  public static final float DEFAULT_TEZ_ENGINE_INPUT_BUFFER_PERCENT = 0.0f;
+
+  // TODO Rename. 
+  public static final String TEZ_ENGINE_GROUP_COMPARATOR_CLASS = 
+      "tez.engine.group.comparator.class";
+  
+  // TODO Better name.
+  public static final String TEZ_ENGINE_INTERNAL_SORTER_CLASS = 
+      "tez.engine.internal.sorter.class";
+  
+  public static final String TEZ_ENGINE_INTERMEDIATE_OUTPUT_KEY_COMPARATOR_CLASS = 
+      "tez.engine.intermediate-output.key.comparator.class";
+  public static final String TEZ_ENGINE_INTERMEDIATE_INPUT_KEY_COMPARATOR_CLASS = 
+      "tez.engine.intermediate-input.key.comparator.class";
+
+  public static final String TEZ_ENGINE_INTERMEDIATE_OUTPUT_KEY_CLASS = 
+      "tez.engine.intermediate-output.key.class";
+  public static final String TEZ_ENGINE_INTERMEDIATE_INPUT_KEY_CLASS = 
+      "tez.engine.intermediate-input.key.class";
+  
+  public static final String TEZ_ENGINE_INTERMEDIATE_OUTPUT_VALUE_CLASS = 
+      "tez.engine.intermediate-output.value.class";
+  public static final String TEZ_ENGINE_INTERMEDIATE_INPUT_VALUE_CLASS = 
+      "tez.engine.intermediate-input.value.class";
+  
+  public static final String TEZ_ENGINE_INTERMEDIATE_OUTPUT_SHOULD_COMPRESS = 
+      "tez.engine.intermediate-output.should-compress";
+  public static final String TEZ_ENGINE_INTERMEDIATE_INPUT_IS_COMPRESSED = 
+      "tez.engine.intermdiate-input.is-compressed";
+  
+  public static final String TEZ_ENGINE_INTERMEDIATE_OUTPUT_COMPRESS_CODEC = 
+      "tez.engine.intermediate-output.compress.codec";
+  public static final String TEZ_ENGINE_INTERMEDIATE_INPUT_COMPRESS_CODEC = 
+      "tez.engine.intermediate-input.compress.codec";
+
+  public static final String TEZ_ENGINE_INTERMEDIATE_INPUT_KEY_SECONDARY_COMPARATOR_CLASS = 
+      "tez.engine.intermediate-input.key.secondary.comparator.class";
+  
+  // TODO This should be in DAGConfiguration
+  /* config for tracking the local file where all the credentials for the job
+   * credentials.
+   */
+  public static final String DAG_CREDENTIALS_BINARY =  "tez.dag.credentials.binary";
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/src/main/java/org/apache/tez/common/counters/AbstractCounter.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/common/counters/AbstractCounter.java b/tez-api/src/main/java/org/apache/tez/common/counters/AbstractCounter.java
new file mode 100644
index 0000000..e64a26c
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/common/counters/AbstractCounter.java
@@ -0,0 +1,52 @@
+/*
+ * 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.common.counters;
+
+import com.google.common.base.Objects;
+import org.apache.hadoop.classification.InterfaceAudience;
+
+/**
+ * An abstract counter class to provide common implementation of
+ * the counter interface in both mapred and mapreduce packages.
+ */
+@InterfaceAudience.Private
+public abstract class AbstractCounter implements TezCounter {
+
+  @Deprecated
+  @Override
+  public void setDisplayName(String name) {}
+
+  @Override
+  public synchronized boolean equals(Object genericRight) {
+    if (genericRight instanceof TezCounter) {
+      synchronized (genericRight) {
+        TezCounter right = (TezCounter) genericRight;
+        return getName().equals(right.getName()) &&
+               getDisplayName().equals(right.getDisplayName()) &&
+               getValue() == right.getValue();
+      }
+    }
+    return false;
+  }
+
+  @Override
+  public synchronized int hashCode() {
+    return Objects.hashCode(getName(), getDisplayName(), getValue());
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/src/main/java/org/apache/tez/common/counters/AbstractCounterGroup.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/common/counters/AbstractCounterGroup.java b/tez-api/src/main/java/org/apache/tez/common/counters/AbstractCounterGroup.java
new file mode 100644
index 0000000..d8896ed
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/common/counters/AbstractCounterGroup.java
@@ -0,0 +1,208 @@
+/*
+ * 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.common.counters;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.WritableUtils;
+
+import com.google.common.collect.Iterators;
+
+/**
+ * An abstract class to provide common implementation of the
+ * generic counter group in both mapred and mapreduce package.
+ *
+ * @param <T> type of the counter for the group
+ */
+@InterfaceAudience.Private
+public abstract class AbstractCounterGroup<T extends TezCounter>
+    implements CounterGroupBase<T> {
+
+  private final String name;
+  private String displayName;
+  private final ConcurrentMap<String, T> counters =
+      new ConcurrentSkipListMap<String, T>();
+  private final Limits limits;
+
+  public AbstractCounterGroup(String name, String displayName,
+                              Limits limits) {
+    this.name = name;
+    this.displayName = displayName;
+    this.limits = limits;
+  }
+
+  @Override
+  public String getName() {
+    return name;
+  }
+
+  @Override
+  public synchronized String getDisplayName() {
+    return displayName;
+  }
+
+  @Override
+  public synchronized void setDisplayName(String displayName) {
+    this.displayName = displayName;
+  }
+
+  @Override
+  public synchronized void addCounter(T counter) {
+    counters.put(counter.getName(), counter);
+    limits.incrCounters();
+  }
+
+  @Override
+  public synchronized T addCounter(String counterName, String displayName,
+                                   long value) {
+    String saveName = Limits.filterCounterName(counterName);
+    T counter = findCounterImpl(saveName, false);
+    if (counter == null) {
+      return addCounterImpl(saveName, displayName, value);
+    }
+    counter.setValue(value);
+    return counter;
+  }
+
+  private T addCounterImpl(String name, String displayName, long value) {
+    T counter = newCounter(name, displayName, value);
+    addCounter(counter);
+    return counter;
+  }
+
+  @Override
+  public synchronized T findCounter(String counterName, String displayName) {
+    // Take lock to avoid two threads not finding a counter and trying to add
+    // the same counter.
+    String saveName = Limits.filterCounterName(counterName);
+    T counter = findCounterImpl(saveName, false);
+    if (counter == null) {
+      return addCounterImpl(saveName, displayName, 0);
+    }
+    return counter;
+  }
+
+  @Override
+  public T findCounter(String counterName, boolean create) {
+    return findCounterImpl(Limits.filterCounterName(counterName), create);
+  }
+
+  // Lock the object. Cannot simply use concurrent constructs on the counters
+  // data-structure (like putIfAbsent) because of localization, limits etc.
+  private synchronized T findCounterImpl(String counterName, boolean create) {
+    T counter = counters.get(counterName);
+    if (counter == null && create) {
+      String localized =
+          ResourceBundles.getCounterName(getName(), counterName, counterName);
+      return addCounterImpl(counterName, localized, 0);
+    }
+    return counter;
+  }
+
+  @Override
+  public T findCounter(String counterName) {
+    return findCounter(counterName, true);
+  }
+
+  /**
+   * Abstract factory method to create a new counter of type T
+   * @param counterName of the counter
+   * @param displayName of the counter
+   * @param value of the counter
+   * @return a new counter
+   */
+  protected abstract T newCounter(String counterName, String displayName,
+                                  long value);
+
+  /**
+   * Abstract factory method to create a new counter of type T
+   * @return a new counter object
+   */
+  protected abstract T newCounter();
+
+  @Override
+  public Iterator<T> iterator() {
+    return counters.values().iterator();
+  }
+
+  /**
+   * GenericGroup ::= displayName #counter counter*
+   */
+  @Override
+  public synchronized void write(DataOutput out) throws IOException {
+    Text.writeString(out, displayName);
+    WritableUtils.writeVInt(out, counters.size());
+    for(TezCounter counter: counters.values()) {
+      counter.write(out);
+    }
+  }
+
+  @Override
+  public synchronized void readFields(DataInput in) throws IOException {
+    displayName = Text.readString(in);
+    counters.clear();
+    int size = WritableUtils.readVInt(in);
+    for (int i = 0; i < size; i++) {
+      T counter = newCounter();
+      counter.readFields(in);
+      counters.put(counter.getName(), counter);
+      limits.incrCounters();
+    }
+  }
+
+  @Override
+  public synchronized int size() {
+    return counters.size();
+  }
+
+  @Override
+  public synchronized boolean equals(Object genericRight) {
+    if (genericRight instanceof CounterGroupBase<?>) {
+      @SuppressWarnings("unchecked")
+      CounterGroupBase<T> right = (CounterGroupBase<T>) genericRight;
+      return Iterators.elementsEqual(iterator(), right.iterator());
+    }
+    return false;
+  }
+
+  @Override
+  public synchronized int hashCode() {
+    return counters.hashCode();
+  }
+
+  @Override
+  public void incrAllCounters(CounterGroupBase<T> rightGroup) {
+    try {
+      for (TezCounter right : rightGroup) {
+        TezCounter left = findCounter(right.getName(), right.getDisplayName());
+        left.increment(right.getValue());
+      }
+    } catch (LimitExceededException e) {
+      counters.clear();
+      throw e;
+    }
+  }
+}


[41/50] [abbrv] Rename tez-engine-api to tez-runtime-api and tez-engine is split into 2: - tez-engine-library for user-visible Input/Output/Processor implementations - tez-engine-internals for framework internals

Posted by ss...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/server/ShuffleHandler.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/server/ShuffleHandler.java b/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/server/ShuffleHandler.java
deleted file mode 100644
index a6d1c5b..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/server/ShuffleHandler.java
+++ /dev/null
@@ -1,572 +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.engine.common.shuffle.server;
-
-import static org.jboss.netty.buffer.ChannelBuffers.wrappedBuffer;
-import static org.jboss.netty.handler.codec.http.HttpHeaders.Names.CONTENT_TYPE;
-import static org.jboss.netty.handler.codec.http.HttpMethod.GET;
-import static org.jboss.netty.handler.codec.http.HttpResponseStatus.BAD_REQUEST;
-import static org.jboss.netty.handler.codec.http.HttpResponseStatus.FORBIDDEN;
-import static org.jboss.netty.handler.codec.http.HttpResponseStatus.INTERNAL_SERVER_ERROR;
-import static org.jboss.netty.handler.codec.http.HttpResponseStatus.METHOD_NOT_ALLOWED;
-import static org.jboss.netty.handler.codec.http.HttpResponseStatus.NOT_FOUND;
-import static org.jboss.netty.handler.codec.http.HttpResponseStatus.OK;
-import static org.jboss.netty.handler.codec.http.HttpResponseStatus.UNAUTHORIZED;
-import static org.jboss.netty.handler.codec.http.HttpVersion.HTTP_1_1;
-
-import java.io.IOException;
-import java.net.InetSocketAddress;
-import java.net.URL;
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.Executors;
-import java.util.concurrent.ThreadFactory;
-import java.util.concurrent.TimeUnit;
-
-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.io.DataInputByteBuffer;
-import org.apache.hadoop.io.DataOutputBuffer;
-import org.apache.hadoop.metrics2.MetricsSystem;
-import org.apache.hadoop.metrics2.annotation.Metric;
-import org.apache.hadoop.metrics2.annotation.Metrics;
-import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
-import org.apache.hadoop.metrics2.lib.MutableCounterInt;
-import org.apache.hadoop.metrics2.lib.MutableCounterLong;
-import org.apache.hadoop.metrics2.lib.MutableGaugeInt;
-import org.apache.hadoop.security.ssl.SSLFactory;
-import org.apache.hadoop.security.token.Token;
-import org.apache.hadoop.yarn.api.records.ApplicationId;
-import org.apache.hadoop.yarn.server.api.ApplicationInitializationContext;
-import org.apache.hadoop.yarn.server.api.ApplicationTerminationContext;
-import org.apache.hadoop.yarn.server.api.AuxiliaryService;
-import org.apache.tez.common.TezJobConfig;
-import org.apache.tez.engine.api.TezOutputContext;
-import org.apache.tez.engine.common.security.JobTokenIdentifier;
-import org.apache.tez.engine.common.security.JobTokenSecretManager;
-import org.apache.tez.engine.common.security.SecureShuffleUtils;
-import org.apache.tez.engine.common.shuffle.impl.ShuffleHeader;
-import org.apache.tez.engine.common.sort.impl.ExternalSorter;
-import org.apache.tez.engine.shuffle.common.ShuffleUtils;
-import org.jboss.netty.bootstrap.ServerBootstrap;
-import org.jboss.netty.buffer.ChannelBuffers;
-import org.jboss.netty.channel.Channel;
-import org.jboss.netty.channel.ChannelFactory;
-import org.jboss.netty.channel.ChannelFuture;
-import org.jboss.netty.channel.ChannelFutureListener;
-import org.jboss.netty.channel.ChannelHandlerContext;
-import org.jboss.netty.channel.ChannelPipeline;
-import org.jboss.netty.channel.ChannelPipelineFactory;
-import org.jboss.netty.channel.Channels;
-import org.jboss.netty.channel.ExceptionEvent;
-import org.jboss.netty.channel.MessageEvent;
-import org.jboss.netty.channel.SimpleChannelUpstreamHandler;
-import org.jboss.netty.channel.group.ChannelGroup;
-import org.jboss.netty.channel.group.DefaultChannelGroup;
-import org.jboss.netty.channel.socket.nio.NioServerSocketChannelFactory;
-import org.jboss.netty.handler.codec.frame.TooLongFrameException;
-import org.jboss.netty.handler.codec.http.DefaultHttpResponse;
-import org.jboss.netty.handler.codec.http.HttpChunkAggregator;
-import org.jboss.netty.handler.codec.http.HttpRequest;
-import org.jboss.netty.handler.codec.http.HttpRequestDecoder;
-import org.jboss.netty.handler.codec.http.HttpResponse;
-import org.jboss.netty.handler.codec.http.HttpResponseEncoder;
-import org.jboss.netty.handler.codec.http.HttpResponseStatus;
-import org.jboss.netty.handler.codec.http.QueryStringDecoder;
-import org.jboss.netty.handler.ssl.SslHandler;
-import org.jboss.netty.handler.stream.ChunkedStream;
-import org.jboss.netty.handler.stream.ChunkedWriteHandler;
-import org.jboss.netty.util.CharsetUtil;
-
-import com.google.common.util.concurrent.ThreadFactoryBuilder;
-
-public class ShuffleHandler extends AuxiliaryService {
-
-  private static final Log LOG = LogFactory.getLog(ShuffleHandler.class);
-  
-  public static final String SHUFFLE_MANAGE_OS_CACHE = "mapreduce.shuffle.manage.os.cache";
-  public static final boolean DEFAULT_SHUFFLE_MANAGE_OS_CACHE = true;
-
-  public static final String SHUFFLE_READAHEAD_BYTES = "mapreduce.shuffle.readahead.bytes";
-  public static final int DEFAULT_SHUFFLE_READAHEAD_BYTES = 4 * 1024 * 1024;
-
-  private int port;
-  private ChannelFactory selector;
-  private final ChannelGroup accepted = new DefaultChannelGroup();
-  private HttpPipelineFactory pipelineFact;
-  private int sslFileBufferSize;
-
-  public static final String MAPREDUCE_SHUFFLE_SERVICEID =
-      "mapreduce.shuffle";
-
-  private static final Map<String,String> userRsrc =
-    new ConcurrentHashMap<String,String>();
-  private static final JobTokenSecretManager secretManager =
-    new JobTokenSecretManager();
-  private SecretKey tokenSecret;
-
-  public static final String SHUFFLE_PORT_CONFIG_KEY = "mapreduce.shuffle.port";
-  public static final int DEFAULT_SHUFFLE_PORT = 8080;
-
-  public static final String SUFFLE_SSL_FILE_BUFFER_SIZE_KEY =
-    "mapreduce.shuffle.ssl.file.buffer.size";
-
-  public static final int DEFAULT_SUFFLE_SSL_FILE_BUFFER_SIZE = 60 * 1024;
-
-  private ExternalSorter sorter;
-  
-  @Metrics(about="Shuffle output metrics", context="mapred")
-  static class ShuffleMetrics implements ChannelFutureListener {
-    @Metric("Shuffle output in bytes")
-        MutableCounterLong shuffleOutputBytes;
-    @Metric("# of failed shuffle outputs")
-        MutableCounterInt shuffleOutputsFailed;
-    @Metric("# of succeeeded shuffle outputs")
-        MutableCounterInt shuffleOutputsOK;
-    @Metric("# of current shuffle connections")
-        MutableGaugeInt shuffleConnections;
-
-    @Override
-    public void operationComplete(ChannelFuture future) throws Exception {
-      if (future.isSuccess()) {
-        shuffleOutputsOK.incr();
-      } else {
-        shuffleOutputsFailed.incr();
-      }
-      shuffleConnections.decr();
-    }
-  }
-
-  final ShuffleMetrics metrics;
-
-  ShuffleHandler(MetricsSystem ms) {
-    super("httpshuffle");
-    metrics = ms.register(new ShuffleMetrics());
-  }
-
-  public ShuffleHandler(ExternalSorter sorter) {
-    this(DefaultMetricsSystem.instance());
-    this.sorter = sorter;
-  }
-
-  /**
-   * Serialize the shuffle port into a ByteBuffer for use later on.
-   * @param port the port to be sent to the ApplciationMaster
-   * @return the serialized form of the port.
-   */
-  public static ByteBuffer serializeMetaData(int port) throws IOException {
-    //TODO these bytes should be versioned
-    DataOutputBuffer port_dob = new DataOutputBuffer();
-    port_dob.writeInt(port);
-    return ByteBuffer.wrap(port_dob.getData(), 0, port_dob.getLength());
-  }
-
-  /**
-   * A helper function to deserialize the metadata returned by ShuffleHandler.
-   * @param meta the metadata returned by the ShuffleHandler
-   * @return the port the Shuffle Handler is listening on to serve shuffle data.
-   */
-  public static int deserializeMetaData(ByteBuffer meta) throws IOException {
-    //TODO this should be returning a class not just an int
-    DataInputByteBuffer in = new DataInputByteBuffer();
-    in.reset(meta);
-    int port = in.readInt();
-    return port;
-  }
-
-  /**
-   * A helper function to serialize the JobTokenIdentifier to be sent to the
-   * ShuffleHandler as ServiceData.
-   * @param jobToken the job token to be used for authentication of
-   * shuffle data requests.
-   * @return the serialized version of the jobToken.
-   */
-  public static ByteBuffer serializeServiceData(Token<JobTokenIdentifier> jobToken) throws IOException {
-    //TODO these bytes should be versioned
-    DataOutputBuffer jobToken_dob = new DataOutputBuffer();
-    jobToken.write(jobToken_dob);
-    return ByteBuffer.wrap(jobToken_dob.getData(), 0, jobToken_dob.getLength());
-  }
-
-  static Token<JobTokenIdentifier> deserializeServiceData(ByteBuffer secret) throws IOException {
-    DataInputByteBuffer in = new DataInputByteBuffer();
-    in.reset(secret);
-    Token<JobTokenIdentifier> jt = new Token<JobTokenIdentifier>();
-    jt.readFields(in);
-    return jt;
-  }
-
-  
-  @Override
-  public void initializeApplication(
-      ApplicationInitializationContext initAppContext) {
-    // TODO these bytes should be versioned
-    try {
-      String user = initAppContext.getUser();
-      ApplicationId appId = initAppContext.getApplicationId();
-      ByteBuffer secret = initAppContext.getApplicationDataForService();
-      Token<JobTokenIdentifier> jt = deserializeServiceData(secret);
-      // TODO: Once SHuffle is out of NM, this can use MR APIs
-      userRsrc.put(appId.toString(), user);
-      LOG.info("Added token for " + appId.toString());
-      secretManager.addTokenForJob(appId.toString(), jt);
-    } catch (IOException e) {
-      LOG.error("Error during initApp", e);
-      // TODO add API to AuxiliaryServices to report failures
-    }
-  }
-
-  @Override
-  public void stopApplication(ApplicationTerminationContext context) {
-    ApplicationId appId = context.getApplicationId();
-    secretManager.removeTokenForJob(appId.toString());
-    userRsrc.remove(appId.toString());
-  }
-
-  public void initialize(TezOutputContext outputContext, Configuration conf) throws IOException {
-    this.init(new Configuration(conf));
-    tokenSecret = ShuffleUtils.getJobTokenSecretFromTokenBytes(outputContext.getServiceConsumerMetaData(MAPREDUCE_SHUFFLE_SERVICEID));
-  }
-
-  @Override
-  public synchronized void serviceInit(Configuration conf) {
-    ThreadFactory bossFactory = new ThreadFactoryBuilder()
-      .setNameFormat("ShuffleHandler Netty Boss #%d")
-      .build();
-    ThreadFactory workerFactory = new ThreadFactoryBuilder()
-      .setNameFormat("ShuffleHandler Netty Worker #%d")
-      .build();
-    
-    selector = new NioServerSocketChannelFactory(
-        Executors.newCachedThreadPool(bossFactory),
-        Executors.newCachedThreadPool(workerFactory));    
-  }
-
-  // TODO change AbstractService to throw InterruptedException
-  @Override
-  public synchronized void serviceStart() {
-    Configuration conf = getConfig();
-    ServerBootstrap bootstrap = new ServerBootstrap(selector);
-    try {
-      pipelineFact = new HttpPipelineFactory(conf);
-    } catch (Exception ex) {
-      throw new RuntimeException(ex);
-    }
-    bootstrap.setPipelineFactory(pipelineFact);
-    // Let OS pick the port
-    Channel ch = bootstrap.bind(new InetSocketAddress(0));
-    accepted.add(ch);
-    port = ((InetSocketAddress)ch.getLocalAddress()).getPort();
-    conf.set(SHUFFLE_PORT_CONFIG_KEY, Integer.toString(port));
-    pipelineFact.SHUFFLE.setPort(port);
-    LOG.info(getName() + " listening on port " + port);
-
-    sslFileBufferSize = conf.getInt(SUFFLE_SSL_FILE_BUFFER_SIZE_KEY,
-                                    DEFAULT_SUFFLE_SSL_FILE_BUFFER_SIZE);
-  }
-
-  @Override
-  public synchronized void serviceStop() {
-    accepted.close().awaitUninterruptibly(10, TimeUnit.SECONDS);
-    ServerBootstrap bootstrap = new ServerBootstrap(selector);
-    bootstrap.releaseExternalResources();
-    pipelineFact.destroy();
-  }
-
-  @Override
-  public synchronized ByteBuffer getMetaData() {
-    try {
-      return serializeMetaData(port); 
-    } catch (IOException e) {
-      LOG.error("Error during getMeta", e);
-      // TODO add API to AuxiliaryServices to report failures
-      return null;
-    }
-  }
-
-  class HttpPipelineFactory implements ChannelPipelineFactory {
-
-    final Shuffle SHUFFLE;
-    private SSLFactory sslFactory;
-
-    public HttpPipelineFactory(Configuration conf) throws Exception {
-      SHUFFLE = new Shuffle(conf);
-      if (conf.getBoolean(TezJobConfig.TEZ_ENGINE_SHUFFLE_ENABLE_SSL,
-              TezJobConfig.DEFAULT_TEZ_ENGINE_SHUFFLE_ENABLE_SSL)) {
-        sslFactory = new SSLFactory(SSLFactory.Mode.SERVER, conf);
-        sslFactory.init();
-      }
-    }
-
-    public void destroy() {
-      if (sslFactory != null) {
-        sslFactory.destroy();
-      }
-    }
-
-    @Override
-    public ChannelPipeline getPipeline() throws Exception {
-      ChannelPipeline pipeline = Channels.pipeline();
-      if (sslFactory != null) {
-        pipeline.addLast("ssl", new SslHandler(sslFactory.createSSLEngine()));
-      }
-      pipeline.addLast("decoder", new HttpRequestDecoder());
-      pipeline.addLast("aggregator", new HttpChunkAggregator(1 << 16));
-      pipeline.addLast("encoder", new HttpResponseEncoder());
-      pipeline.addLast("chunking", new ChunkedWriteHandler());
-      pipeline.addLast("shuffle", SHUFFLE);
-      return pipeline;
-      // TODO factor security manager into pipeline
-      // TODO factor out encode/decode to permit binary shuffle
-      // TODO factor out decode of index to permit alt. models
-    }
-
-  }
-
-  class Shuffle extends SimpleChannelUpstreamHandler {
-
-    private final Configuration conf;
-    private int port;
-
-    public Shuffle(Configuration conf) {
-      this.conf = conf;
-      this.port = conf.getInt(SHUFFLE_PORT_CONFIG_KEY, DEFAULT_SHUFFLE_PORT);
-    }
-    
-    public void setPort(int port) {
-      this.port = port;
-    }
-
-    private List<String> splitMaps(List<String> mapq) {
-      if (null == mapq) {
-        return null;
-      }
-      final List<String> ret = new ArrayList<String>();
-      for (String s : mapq) {
-        Collections.addAll(ret, s.split(","));
-      }
-      return ret;
-    }
-
-    @Override
-    public void messageReceived(ChannelHandlerContext ctx, MessageEvent evt)
-        throws Exception {
-      HttpRequest request = (HttpRequest) evt.getMessage();
-      if (request.getMethod() != GET) {
-          sendError(ctx, METHOD_NOT_ALLOWED);
-          return;
-      }
-      // Check whether the shuffle version is compatible
-      if (!ShuffleHeader.DEFAULT_HTTP_HEADER_NAME.equals(
-          request.getHeader(ShuffleHeader.HTTP_HEADER_NAME))
-          || !ShuffleHeader.DEFAULT_HTTP_HEADER_VERSION.equals(
-              request.getHeader(ShuffleHeader.HTTP_HEADER_VERSION))) {
-        sendError(ctx, "Incompatible shuffle request version", BAD_REQUEST);
-      }
-      final Map<String,List<String>> q =
-        new QueryStringDecoder(request.getUri()).getParameters();
-      final List<String> mapIds = splitMaps(q.get("map"));
-      final List<String> reduceQ = q.get("reduce");
-      final List<String> jobQ = q.get("job");
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("RECV: " + request.getUri() +
-            "\n  mapId: " + mapIds +
-            "\n  reduceId: " + reduceQ +
-            "\n  jobId: " + jobQ);
-      }
-
-      if (mapIds == null || reduceQ == null || jobQ == null) {
-        sendError(ctx, "Required param job, map and reduce", BAD_REQUEST);
-        return;
-      }
-      if (reduceQ.size() != 1 || jobQ.size() != 1) {
-        sendError(ctx, "Too many job/reduce parameters", BAD_REQUEST);
-        return;
-      }
-      int reduceId;
-      String jobId;
-      try {
-        reduceId = Integer.parseInt(reduceQ.get(0));
-        jobId = jobQ.get(0);
-      } catch (NumberFormatException e) {
-        sendError(ctx, "Bad reduce parameter", BAD_REQUEST);
-        return;
-      } catch (IllegalArgumentException e) {
-        sendError(ctx, "Bad job parameter", BAD_REQUEST);
-        return;
-      }
-
-      final String reqUri = request.getUri();
-      if (null == reqUri) {
-        // TODO? add upstream?
-        sendError(ctx, FORBIDDEN);
-        return;
-      }
-      HttpResponse response = new DefaultHttpResponse(HTTP_1_1, OK);
-      try {
-        verifyRequest(jobId, ctx, request, response,
-            new URL("http", "", this.port, reqUri));
-      } catch (IOException e) {
-        LOG.warn("Shuffle failure ", e);
-        sendError(ctx, e.getMessage(), UNAUTHORIZED);
-        return;
-      }
-
-      Channel ch = evt.getChannel();
-      ch.write(response);
-      // TODO refactor the following into the pipeline
-      ChannelFuture lastMap = null;
-      for (String mapId : mapIds) {
-        try {
-          // TODO: Error handling - validate mapId via TezTaskAttemptId.forName
-          
-          // TODO NEWTEZ Fix this. TaskAttemptId is no longer valid. mapId validation will not work anymore.
-//          if (!mapId.equals(sorter.getTaskAttemptId().toString())) {
-//            String errorMessage =
-//                "Illegal shuffle request mapId: " + mapId
-//                    + " while actual mapId is " + sorter.getTaskAttemptId(); 
-//            LOG.warn(errorMessage);
-//            sendError(ctx, errorMessage, BAD_REQUEST);
-//            return;
-//          }
-
-          lastMap =
-            sendMapOutput(ctx, ch, userRsrc.get(jobId), jobId, mapId, reduceId);
-          if (null == lastMap) {
-            sendError(ctx, NOT_FOUND);
-            return;
-          }
-        } catch (IOException e) {
-          LOG.error("Shuffle error ", e);
-          sendError(ctx, e.getMessage(), INTERNAL_SERVER_ERROR);
-          return;
-        }
-      }
-      lastMap.addListener(metrics);
-      lastMap.addListener(ChannelFutureListener.CLOSE);
-    }
-
-    private void verifyRequest(String appid, ChannelHandlerContext ctx,
-        HttpRequest request, HttpResponse response, URL requestUri)
-        throws IOException {
-      if (null == tokenSecret) {
-        LOG.info("Request for unknown token " + appid);
-        throw new IOException("could not find jobid");
-      }
-      // string to encrypt
-      String enc_str = SecureShuffleUtils.buildMsgFrom(requestUri);
-      // hash from the fetcher
-      String urlHashStr =
-        request.getHeader(SecureShuffleUtils.HTTP_HEADER_URL_HASH);
-      if (urlHashStr == null) {
-        LOG.info("Missing header hash for " + appid);
-        throw new IOException("fetcher cannot be authenticated");
-      }
-      if (LOG.isDebugEnabled()) {
-        int len = urlHashStr.length();
-        LOG.debug("verifying request. enc_str=" + enc_str + "; hash=..." +
-            urlHashStr.substring(len-len/2, len-1));
-      }
-      // verify - throws exception
-      SecureShuffleUtils.verifyReply(urlHashStr, enc_str, tokenSecret);
-      // verification passed - encode the reply
-      String reply =
-        SecureShuffleUtils.generateHash(urlHashStr.getBytes(), tokenSecret);
-      response.setHeader(SecureShuffleUtils.HTTP_HEADER_REPLY_URL_HASH, reply);
-      addVersionToHeader(response);
-      if (LOG.isDebugEnabled()) {
-        int len = reply.length();
-        LOG.debug("Fetcher request verfied. enc_str=" + enc_str + ";reply=" +
-            reply.substring(len-len/2, len-1));
-      }
-    }
-
-    protected ChannelFuture sendMapOutput(ChannelHandlerContext ctx, Channel ch,
-        String user, String jobId, String mapId, int reduce)
-        throws IOException {
-      final ShuffleHeader header = sorter.getShuffleHeader(reduce);
-      final DataOutputBuffer dob = new DataOutputBuffer();
-      header.write(dob);
-      ch.write(wrappedBuffer(dob.getData(), 0, dob.getLength()));
-
-      ChannelFuture writeFuture =
-          ch.write(
-              new ChunkedStream(
-                  sorter.getSortedStream(reduce), sslFileBufferSize
-                  )
-              );
-      metrics.shuffleConnections.incr();
-      metrics.shuffleOutputBytes.incr(header.getCompressedLength()); // optimistic
-      return writeFuture;
-    }
-
-    private void sendError(ChannelHandlerContext ctx,
-        HttpResponseStatus status) {
-      sendError(ctx, "", status);
-    }
-
-    private void sendError(ChannelHandlerContext ctx, String message,
-        HttpResponseStatus status) {
-      HttpResponse response = new DefaultHttpResponse(HTTP_1_1, status);
-      response.setHeader(CONTENT_TYPE, "text/plain; charset=UTF-8");
-      addVersionToHeader(response);
-      response.setContent(
-        ChannelBuffers.copiedBuffer(message, CharsetUtil.UTF_8));
-      // Close the connection as soon as the error message is sent.
-      ctx.getChannel().write(response).addListener(ChannelFutureListener.CLOSE);
-    }
-    
-    private void addVersionToHeader(HttpResponse response) {
-      // Put shuffle version into http header
-      response.setHeader(ShuffleHeader.HTTP_HEADER_NAME,
-          ShuffleHeader.DEFAULT_HTTP_HEADER_NAME);
-      response.setHeader(ShuffleHeader.HTTP_HEADER_VERSION,
-          ShuffleHeader.DEFAULT_HTTP_HEADER_VERSION);      
-    }
-
-    @Override
-    public void exceptionCaught(ChannelHandlerContext ctx, ExceptionEvent e)
-        throws Exception {
-      Channel ch = e.getChannel();
-      Throwable cause = e.getCause();
-      if (cause instanceof TooLongFrameException) {
-        sendError(ctx, BAD_REQUEST);
-        return;
-      }
-
-      LOG.error("Shuffle error: ", cause);
-      if (ch.isConnected()) {
-        LOG.error("Shuffle error " + e);
-        sendError(ctx, INTERNAL_SERVER_ERROR);
-      }
-    }
-
-  }
-
-  public int getPort() {
-    return port;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/ExternalSorter.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/ExternalSorter.java b/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/ExternalSorter.java
deleted file mode 100644
index 5aa0ddf..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/ExternalSorter.java
+++ /dev/null
@@ -1,194 +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.engine.common.sort.impl;
-
-import java.io.File;
-import java.io.IOException;
-import java.io.InputStream;
-
-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.fs.FileSystem;
-import org.apache.hadoop.fs.LocalFileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.RawLocalFileSystem;
-import org.apache.hadoop.io.RawComparator;
-import org.apache.hadoop.io.compress.CompressionCodec;
-import org.apache.hadoop.io.compress.DefaultCodec;
-import org.apache.hadoop.io.serializer.SerializationFactory;
-import org.apache.hadoop.io.serializer.Serializer;
-import org.apache.hadoop.util.IndexedSorter;
-import org.apache.hadoop.util.Progressable;
-import org.apache.hadoop.util.QuickSort;
-import org.apache.hadoop.util.ReflectionUtils;
-import org.apache.tez.common.TezJobConfig;
-import org.apache.tez.common.counters.TaskCounter;
-import org.apache.tez.common.counters.TezCounter;
-import org.apache.tez.engine.api.Partitioner;
-import org.apache.tez.engine.api.TezOutputContext;
-import org.apache.tez.engine.common.ConfigUtils;
-import org.apache.tez.engine.common.TezEngineUtils;
-import org.apache.tez.engine.common.combine.Combiner;
-import org.apache.tez.engine.common.shuffle.impl.ShuffleHeader;
-import org.apache.tez.engine.common.sort.impl.IFile.Writer;
-import org.apache.tez.engine.common.task.local.output.TezTaskOutput;
-import org.apache.tez.engine.hadoop.compat.NullProgressable;
-
-@SuppressWarnings({"unchecked", "rawtypes"})
-public abstract class ExternalSorter {
-
-  private static final Log LOG = LogFactory.getLog(ExternalSorter.class);
-
-  public abstract void close() throws IOException;
-
-  public abstract void flush() throws IOException;
-
-  public abstract void write(Object key, Object value) throws IOException;
-
-  protected Progressable nullProgressable = new NullProgressable();
-  protected TezOutputContext outputContext;
-  protected Combiner combiner;
-  protected Partitioner partitioner;
-  protected Configuration conf;
-  protected FileSystem rfs;
-  protected TezTaskOutput mapOutputFile;
-  protected int partitions;
-  protected Class keyClass;
-  protected Class valClass;
-  protected RawComparator comparator;
-  protected SerializationFactory serializationFactory;
-  protected Serializer keySerializer;
-  protected Serializer valSerializer;
-
-  protected IndexedSorter sorter;
-
-  // Compression for map-outputs
-  protected CompressionCodec codec;
-
-  // Counters
-  // TODO TEZ Rename all counter variables [Mapping of counter to MR for compatibility in the MR layer]
-  protected TezCounter mapOutputByteCounter;
-  protected TezCounter mapOutputRecordCounter;
-  protected TezCounter fileOutputByteCounter;
-  protected TezCounter spilledRecordsCounter;
-
-  public void initialize(TezOutputContext outputContext, Configuration conf, int numOutputs) throws IOException {
-    this.outputContext = outputContext;
-    this.conf = conf;
-    this.partitions = numOutputs;
-
-    rfs = ((LocalFileSystem)FileSystem.getLocal(this.conf)).getRaw();
-
-    // sorter
-    sorter = ReflectionUtils.newInstance(this.conf.getClass(
-        TezJobConfig.TEZ_ENGINE_INTERNAL_SORTER_CLASS, QuickSort.class,
-        IndexedSorter.class), this.conf);
-
-    comparator = ConfigUtils.getIntermediateOutputKeyComparator(this.conf);
-
-    // k/v serialization
-    keyClass = ConfigUtils.getIntermediateOutputKeyClass(this.conf);
-    valClass = ConfigUtils.getIntermediateOutputValueClass(this.conf);
-    serializationFactory = new SerializationFactory(this.conf);
-    keySerializer = serializationFactory.getSerializer(keyClass);
-    valSerializer = serializationFactory.getSerializer(valClass);
-
-    //    counters
-    mapOutputByteCounter =
-        outputContext.getCounters().findCounter(TaskCounter.MAP_OUTPUT_BYTES);
-    mapOutputRecordCounter =
-        outputContext.getCounters().findCounter(TaskCounter.MAP_OUTPUT_RECORDS);
-    fileOutputByteCounter =
-        outputContext.getCounters().findCounter(TaskCounter.MAP_OUTPUT_MATERIALIZED_BYTES);
-    spilledRecordsCounter =
-        outputContext.getCounters().findCounter(TaskCounter.SPILLED_RECORDS);
-    // compression
-    if (ConfigUtils.shouldCompressIntermediateOutput(this.conf)) {
-      Class<? extends CompressionCodec> codecClass =
-          ConfigUtils.getIntermediateOutputCompressorClass(this.conf, DefaultCodec.class);
-      codec = ReflectionUtils.newInstance(codecClass, this.conf);
-    } else {
-      codec = null;
-    }
-
-    // Task outputs
-    mapOutputFile = TezEngineUtils.instantiateTaskOutputManager(conf, outputContext);
-    
-    LOG.info("Instantiating Partitioner: [" + conf.get(TezJobConfig.TEZ_ENGINE_PARTITIONER_CLASS) + "]");
-    this.conf.setInt(TezJobConfig.TEZ_ENGINE_NUM_EXPECTED_PARTITIONS, this.partitions);
-    this.partitioner = TezEngineUtils.instantiatePartitioner(this.conf);
-    this.combiner = TezEngineUtils.instantiateCombiner(this.conf, outputContext);
-  }
-
-  /**
-   * Exception indicating that the allocated sort buffer is insufficient to hold
-   * the current record.
-   */
-  @SuppressWarnings("serial")
-  public static class MapBufferTooSmallException extends IOException {
-    public MapBufferTooSmallException(String s) {
-      super(s);
-    }
-  }
-
-  @Private
-  public TezTaskOutput getMapOutput() {
-    return mapOutputFile;
-  }
-
-  protected void runCombineProcessor(TezRawKeyValueIterator kvIter,
-      Writer writer) throws IOException {
-    try {
-      combiner.combine(kvIter, writer);
-    } catch (InterruptedException e) {
-      throw new IOException(e);
-    }
-  }
-
-  /**
-   * Rename srcPath to dstPath on the same volume. This is the same as
-   * RawLocalFileSystem's rename method, except that it will not fall back to a
-   * copy, and it will create the target directory if it doesn't exist.
-   */
-  protected void sameVolRename(Path srcPath, Path dstPath) throws IOException {
-    RawLocalFileSystem rfs = (RawLocalFileSystem) this.rfs;
-    File src = rfs.pathToFile(srcPath);
-    File dst = rfs.pathToFile(dstPath);
-    if (!dst.getParentFile().exists()) {
-      if (!dst.getParentFile().mkdirs()) {
-        throw new IOException("Unable to rename " + src + " to " + dst
-            + ": couldn't create parent directory");
-      }
-    }
-
-    if (!src.renameTo(dst)) {
-      throw new IOException("Unable to rename " + src + " to " + dst);
-    }
-  }
-
-  public InputStream getSortedStream(int partition) {
-    throw new UnsupportedOperationException("getSortedStream isn't supported!");
-  }
-
-  public ShuffleHeader getShuffleHeader(int reduce) {
-    throw new UnsupportedOperationException("getShuffleHeader isn't supported!");
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/IFile.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/IFile.java b/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/IFile.java
deleted file mode 100644
index 7d10606..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/IFile.java
+++ /dev/null
@@ -1,559 +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.engine.common.sort.impl;
-
-import java.io.DataInput;
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
-import java.io.EOFException;
-import java.io.IOException;
-import java.io.InputStream;
-import java.util.concurrent.atomic.AtomicBoolean;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FSDataInputStream;
-import org.apache.hadoop.fs.FSDataOutputStream;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.DataInputBuffer;
-import org.apache.hadoop.io.DataOutputBuffer;
-import org.apache.hadoop.io.BufferUtils;
-import org.apache.hadoop.io.IOUtils;
-import org.apache.hadoop.io.WritableUtils;
-import org.apache.hadoop.io.compress.CodecPool;
-import org.apache.hadoop.io.compress.CompressionCodec;
-import org.apache.hadoop.io.compress.CompressionOutputStream;
-import org.apache.hadoop.io.compress.Compressor;
-import org.apache.hadoop.io.compress.Decompressor;
-import org.apache.hadoop.io.serializer.SerializationFactory;
-import org.apache.hadoop.io.serializer.Serializer;
-import org.apache.tez.common.counters.TezCounter;
-
-/**
- * <code>IFile</code> is the simple <key-len, value-len, key, value> format
- * for the intermediate map-outputs in Map-Reduce.
- *
- * There is a <code>Writer</code> to write out map-outputs in this format and 
- * a <code>Reader</code> to read files of this format.
- */
-@InterfaceAudience.Private
-@InterfaceStability.Unstable
-public class IFile {
-  private static final Log LOG = LogFactory.getLog(IFile.class);
-  public static final int EOF_MARKER = -1; // End of File Marker
-  public static final int RLE_MARKER = -2; // Repeat same key marker
-  public static final DataInputBuffer REPEAT_KEY = new DataInputBuffer();
-    
-  /**
-   * <code>IFile.Writer</code> to write out intermediate map-outputs. 
-   */
-  @InterfaceAudience.Private
-  @InterfaceStability.Unstable
-  @SuppressWarnings({"unchecked", "rawtypes"})
-  public static class Writer {
-    FSDataOutputStream out;
-    boolean ownOutputStream = false;
-    long start = 0;
-    FSDataOutputStream rawOut;
-    AtomicBoolean closed = new AtomicBoolean(false);
-    
-    CompressionOutputStream compressedOut;
-    Compressor compressor;
-    boolean compressOutput = false;
-    
-    long decompressedBytesWritten = 0;
-    long compressedBytesWritten = 0;
-
-    // Count records written to disk
-    private long numRecordsWritten = 0;
-    private final TezCounter writtenRecordsCounter;
-
-    IFileOutputStream checksumOut;
-
-    Class keyClass;
-    Class valueClass;
-    Serializer keySerializer;
-    Serializer valueSerializer;
-    
-    DataOutputBuffer buffer = new DataOutputBuffer();
-    DataOutputBuffer previous = new DataOutputBuffer();
-    
-    // de-dup keys or not
-    private boolean rle = false;
-
-    public Writer(Configuration conf, FileSystem fs, Path file, 
-                  Class keyClass, Class valueClass,
-                  CompressionCodec codec,
-                  TezCounter writesCounter) throws IOException {
-      this(conf, fs.create(file), keyClass, valueClass, codec,
-           writesCounter);
-      ownOutputStream = true;
-    }
-    
-    protected Writer(TezCounter writesCounter) {
-      writtenRecordsCounter = writesCounter;
-    }
-
-    public Writer(Configuration conf, FSDataOutputStream out, 
-        Class keyClass, Class valueClass,
-        CompressionCodec codec, TezCounter writesCounter)
-        throws IOException {
-      this.writtenRecordsCounter = writesCounter;
-      this.checksumOut = new IFileOutputStream(out);
-      this.rawOut = out;
-      this.start = this.rawOut.getPos();
-      if (codec != null) {
-        this.compressor = CodecPool.getCompressor(codec);
-        if (this.compressor != null) {
-          this.compressor.reset();
-          this.compressedOut = codec.createOutputStream(checksumOut, compressor);
-          this.out = new FSDataOutputStream(this.compressedOut,  null);
-          this.compressOutput = true;
-        } else {
-          LOG.warn("Could not obtain compressor from CodecPool");
-          this.out = new FSDataOutputStream(checksumOut,null);
-        }
-      } else {
-        this.out = new FSDataOutputStream(checksumOut,null);
-      }
-      
-      this.keyClass = keyClass;
-      this.valueClass = valueClass;
-
-      if (keyClass != null) {
-        SerializationFactory serializationFactory = 
-          new SerializationFactory(conf);
-        this.keySerializer = serializationFactory.getSerializer(keyClass);
-        this.keySerializer.open(buffer);
-        this.valueSerializer = serializationFactory.getSerializer(valueClass);
-        this.valueSerializer.open(buffer);
-      }
-    }
-
-    public Writer(Configuration conf, FileSystem fs, Path file) 
-    throws IOException {
-      this(conf, fs, file, null, null, null, null);
-    }
-
-    public void close() throws IOException {
-      if (closed.getAndSet(true)) {
-        throw new IOException("Writer was already closed earlier");
-      }
-
-      // When IFile writer is created by BackupStore, we do not have
-      // Key and Value classes set. So, check before closing the
-      // serializers
-      if (keyClass != null) {
-        keySerializer.close();
-        valueSerializer.close();
-      }
-
-      // Write EOF_MARKER for key/value length
-      WritableUtils.writeVInt(out, EOF_MARKER);
-      WritableUtils.writeVInt(out, EOF_MARKER);
-      decompressedBytesWritten += 2 * WritableUtils.getVIntSize(EOF_MARKER);
-      
-      //Flush the stream
-      out.flush();
-  
-      if (compressOutput) {
-        // Flush
-        compressedOut.finish();
-        compressedOut.resetState();
-      }
-      
-      // Close the underlying stream iff we own it...
-      if (ownOutputStream) {
-        out.close();
-      }
-      else {
-        // Write the checksum
-        checksumOut.finish();
-      }
-
-      compressedBytesWritten = rawOut.getPos() - start;
-
-      if (compressOutput) {
-        // Return back the compressor
-        CodecPool.returnCompressor(compressor);
-        compressor = null;
-      }
-
-      out = null;
-      if(writtenRecordsCounter != null) {
-        writtenRecordsCounter.increment(numRecordsWritten);
-      }
-    }
-
-    public void append(Object key, Object value) throws IOException {
-      if (key.getClass() != keyClass)
-        throw new IOException("wrong key class: "+ key.getClass()
-                              +" is not "+ keyClass);
-      if (value.getClass() != valueClass)
-        throw new IOException("wrong value class: "+ value.getClass()
-                              +" is not "+ valueClass);
-      
-      boolean sameKey = false;
-
-      // Append the 'key'
-      keySerializer.serialize(key);
-      int keyLength = buffer.getLength();
-      if (keyLength < 0) {
-        throw new IOException("Negative key-length not allowed: " + keyLength + 
-                              " for " + key);
-      }     
-      
-      if(keyLength == previous.getLength()) {
-        sameKey = (BufferUtils.compare(previous, buffer) == 0);       
-      }
-      
-      if(!sameKey) {
-        BufferUtils.copy(buffer, previous);
-      }
-
-      // Append the 'value'
-      valueSerializer.serialize(value);
-      int valueLength = buffer.getLength() - keyLength;
-      if (valueLength < 0) {
-        throw new IOException("Negative value-length not allowed: " + 
-                              valueLength + " for " + value);
-      }
-      
-      if(sameKey) {        
-        WritableUtils.writeVInt(out, RLE_MARKER);                   // Same key as previous
-        WritableUtils.writeVInt(out, valueLength);                  // value length
-        out.write(buffer.getData(), keyLength, buffer.getLength()); // only the value
-        // Update bytes written
-        decompressedBytesWritten += 0 + valueLength + 
-                                    WritableUtils.getVIntSize(RLE_MARKER) + 
-                                    WritableUtils.getVIntSize(valueLength);
-      } else {        
-        // Write the record out        
-        WritableUtils.writeVInt(out, keyLength);                  // key length
-        WritableUtils.writeVInt(out, valueLength);                // value length
-        out.write(buffer.getData(), 0, buffer.getLength());       // data
-        // Update bytes written
-        decompressedBytesWritten += keyLength + valueLength + 
-                                    WritableUtils.getVIntSize(keyLength) + 
-                                    WritableUtils.getVIntSize(valueLength);
-      }
-
-      // Reset
-      buffer.reset();
-      
-      
-      ++numRecordsWritten;
-    }
-    
-    public void append(DataInputBuffer key, DataInputBuffer value)
-    throws IOException {
-      int keyLength = key.getLength() - key.getPosition();
-      if (keyLength < 0) {
-        throw new IOException("Negative key-length not allowed: " + keyLength + 
-                              " for " + key);
-      }
-      
-      int valueLength = value.getLength() - value.getPosition();
-      if (valueLength < 0) {
-        throw new IOException("Negative value-length not allowed: " + 
-                              valueLength + " for " + value);
-      }
-      
-      boolean sameKey = false;
-      
-      if(rle && keyLength == previous.getLength()) {
-        sameKey = (keyLength != 0) && (BufferUtils.compare(previous, key) == 0);        
-      }
-      
-      if(rle && sameKey) {
-        WritableUtils.writeVInt(out, RLE_MARKER);
-        WritableUtils.writeVInt(out, valueLength);        
-        out.write(value.getData(), value.getPosition(), valueLength);
-
-        // Update bytes written
-        decompressedBytesWritten += 0 + valueLength
-            + WritableUtils.getVIntSize(RLE_MARKER)
-            + WritableUtils.getVIntSize(valueLength);
-      } else {
-        WritableUtils.writeVInt(out, keyLength);
-        WritableUtils.writeVInt(out, valueLength);
-        out.write(key.getData(), key.getPosition(), keyLength);
-        out.write(value.getData(), value.getPosition(), valueLength);
-
-        // Update bytes written
-        decompressedBytesWritten += keyLength + valueLength
-            + WritableUtils.getVIntSize(keyLength)
-            + WritableUtils.getVIntSize(valueLength);
-                
-        BufferUtils.copy(key, previous);        
-      }
-      ++numRecordsWritten;
-    }
-    
-    // Required for mark/reset
-    public DataOutputStream getOutputStream () {
-      return out;
-    }
-    
-    // Required for mark/reset
-    public void updateCountersForExternalAppend(long length) {
-      ++numRecordsWritten;
-      decompressedBytesWritten += length;
-    }
-    
-    public long getRawLength() {
-      return decompressedBytesWritten;
-    }
-    
-    public long getCompressedLength() {
-      return compressedBytesWritten;
-    }
-    
-    public void setRLE(boolean rle) {
-      this.rle = rle;
-      previous.reset();
-    }
-
-  }
-
-  /**
-   * <code>IFile.Reader</code> to read intermediate map-outputs. 
-   */
-  @InterfaceAudience.Private
-  @InterfaceStability.Unstable
-  public static class Reader {
-    
-    public enum KeyState {NO_KEY, NEW_KEY, SAME_KEY};
-    
-    private static final int DEFAULT_BUFFER_SIZE = 128*1024;
-
-    // Count records read from disk
-    private long numRecordsRead = 0;
-    private final TezCounter readRecordsCounter;
-
-    final InputStream in;        // Possibly decompressed stream that we read
-    Decompressor decompressor;
-    public long bytesRead = 0;
-    protected final long fileLength;
-    protected boolean eof = false;
-    final IFileInputStream checksumIn;
-    
-    protected byte[] buffer = null;
-    protected int bufferSize = DEFAULT_BUFFER_SIZE;
-    protected DataInputStream dataIn;
-
-    protected int recNo = 1;
-    protected int prevKeyLength;
-    protected int currentKeyLength;
-    protected int currentValueLength;
-    byte keyBytes[] = new byte[0];
-    
-    
-    /**
-     * Construct an IFile Reader.
-     * 
-     * @param conf Configuration File 
-     * @param fs  FileSystem
-     * @param file Path of the file to be opened. This file should have
-     *             checksum bytes for the data at the end of the file.
-     * @param codec codec
-     * @param readsCounter Counter for records read from disk
-     * @throws IOException
-     */
-    public Reader(Configuration conf, FileSystem fs, Path file,
-                  CompressionCodec codec,
-                  TezCounter readsCounter) throws IOException {
-      this(conf, fs.open(file), 
-           fs.getFileStatus(file).getLen(),
-           codec, readsCounter);
-    }
-
-    /**
-     * Construct an IFile Reader.
-     * 
-     * @param conf Configuration File 
-     * @param in   The input stream
-     * @param length Length of the data in the stream, including the checksum
-     *               bytes.
-     * @param codec codec
-     * @param readsCounter Counter for records read from disk
-     * @throws IOException
-     */
-    public Reader(Configuration conf, InputStream in, long length, 
-                  CompressionCodec codec,
-                  TezCounter readsCounter) throws IOException {
-      readRecordsCounter = readsCounter;
-      checksumIn = new IFileInputStream(in,length, conf);
-      if (codec != null) {
-        decompressor = CodecPool.getDecompressor(codec);
-        if (decompressor != null) {
-          this.in = codec.createInputStream(checksumIn, decompressor);
-        } else {
-          LOG.warn("Could not obtain decompressor from CodecPool");
-          this.in = checksumIn;
-        }
-      } else {
-        this.in = checksumIn;
-      }
-      this.dataIn = new DataInputStream(this.in);
-      this.fileLength = length;
-      
-      if (conf != null) {
-        bufferSize = conf.getInt("io.file.buffer.size", DEFAULT_BUFFER_SIZE);
-      }
-    }
-    
-    public long getLength() { 
-      return fileLength - checksumIn.getSize();
-    }
-    
-    public long getPosition() throws IOException {    
-      return checksumIn.getPosition(); 
-    }
-    
-    /**
-     * Read upto len bytes into buf starting at offset off.
-     * 
-     * @param buf buffer 
-     * @param off offset
-     * @param len length of buffer
-     * @return the no. of bytes read
-     * @throws IOException
-     */
-    private int readData(byte[] buf, int off, int len) throws IOException {
-      int bytesRead = 0;
-      while (bytesRead < len) {
-        int n = IOUtils.wrappedReadForCompressedData(in, buf, off + bytesRead,
-            len - bytesRead);
-        if (n < 0) {
-          return bytesRead;
-        }
-        bytesRead += n;
-      }
-      return len;
-    }
-    
-    protected boolean positionToNextRecord(DataInput dIn) throws IOException {
-      // Sanity check
-      if (eof) {
-        throw new EOFException("Completed reading " + bytesRead);
-      }
-      
-      // Read key and value lengths
-      prevKeyLength = currentKeyLength;
-      currentKeyLength = WritableUtils.readVInt(dIn);
-      currentValueLength = WritableUtils.readVInt(dIn);
-      bytesRead += WritableUtils.getVIntSize(currentKeyLength) +
-                   WritableUtils.getVIntSize(currentValueLength);
-      
-      // Check for EOF
-      if (currentKeyLength == EOF_MARKER && currentValueLength == EOF_MARKER) {
-        eof = true;
-        return false;
-      }      
-      
-      // Sanity check
-      if (currentKeyLength != RLE_MARKER && currentKeyLength < 0) {
-        throw new IOException("Rec# " + recNo + ": Negative key-length: " + 
-                              currentKeyLength);
-      }
-      if (currentValueLength < 0) {
-        throw new IOException("Rec# " + recNo + ": Negative value-length: " + 
-                              currentValueLength);
-      }
-            
-      return true;
-    }
-    
-    public boolean nextRawKey(DataInputBuffer key) throws IOException {
-      return readRawKey(key) != KeyState.NO_KEY;
-    }
-    
-    public KeyState readRawKey(DataInputBuffer key) throws IOException {
-      if (!positionToNextRecord(dataIn)) {
-        return KeyState.NO_KEY;
-      }
-      if(currentKeyLength == RLE_MARKER) {
-        currentKeyLength = prevKeyLength;
-        // no data to read
-        key.reset(keyBytes, currentKeyLength);
-        return KeyState.SAME_KEY;
-      }
-      if (keyBytes.length < currentKeyLength) {
-        keyBytes = new byte[currentKeyLength << 1];
-      }
-      int i = readData(keyBytes, 0, currentKeyLength);
-      if (i != currentKeyLength) {
-        throw new IOException ("Asked for " + currentKeyLength + " Got: " + i);
-      }
-      key.reset(keyBytes, currentKeyLength);
-      bytesRead += currentKeyLength;
-      return KeyState.NEW_KEY;
-    }
-    
-    public void nextRawValue(DataInputBuffer value) throws IOException {
-      final byte[] valBytes = 
-        ((value.getData().length < currentValueLength) || (value.getData() == keyBytes))
-        ? new byte[currentValueLength << 1]
-        : value.getData();
-      int i = readData(valBytes, 0, currentValueLength);
-      if (i != currentValueLength) {
-        throw new IOException ("Asked for " + currentValueLength + " Got: " + i);
-      }
-      value.reset(valBytes, currentValueLength);
-      
-      // Record the bytes read
-      bytesRead += currentValueLength;
-
-      ++recNo;
-      ++numRecordsRead;
-    }
-    
-    public void close() throws IOException {
-      // Close the underlying stream
-      in.close();
-      
-      // Release the buffer
-      dataIn = null;
-      buffer = null;
-      if(readRecordsCounter != null) {
-        readRecordsCounter.increment(numRecordsRead);
-      }
-
-      // Return the decompressor
-      if (decompressor != null) {
-        decompressor.reset();
-        CodecPool.returnDecompressor(decompressor);
-        decompressor = null;
-      }
-    }
-    
-    public void reset(int offset) {
-      return;
-    }
-
-    public void disableChecksumValidation() {
-      checksumIn.disableChecksumValidation();
-    }
-
-  }    
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/IFileInputStream.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/IFileInputStream.java b/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/IFileInputStream.java
deleted file mode 100644
index dfb69f1..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/IFileInputStream.java
+++ /dev/null
@@ -1,276 +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.engine.common.sort.impl;
-
-import java.io.EOFException;
-import java.io.FileDescriptor;
-import java.io.FileInputStream;
-import java.io.IOException;
-import java.io.InputStream;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.ChecksumException;
-import org.apache.hadoop.fs.HasFileDescriptor;
-import org.apache.hadoop.io.IOUtils;
-import org.apache.hadoop.io.ReadaheadPool;
-import org.apache.hadoop.io.ReadaheadPool.ReadaheadRequest;
-import org.apache.hadoop.util.DataChecksum;
-import org.apache.tez.common.TezJobConfig;
-/**
- * A checksum input stream, used for IFiles.
- * Used to validate the checksum of files created by {@link IFileOutputStream}. 
-*/
-@InterfaceAudience.Private
-@InterfaceStability.Unstable
-public class IFileInputStream extends InputStream {
-  
-  private final InputStream in; //The input stream to be verified for checksum.
-  private final FileDescriptor inFd; // the file descriptor, if it is known
-  private final long length; //The total length of the input file
-  private final long dataLength;
-  private DataChecksum sum;
-  private long currentOffset = 0;
-  private final byte b[] = new byte[1];
-  private byte csum[] = null;
-  private int checksumSize;
-  private byte[] buffer;
-  private int offset;
-
-  private ReadaheadRequest curReadahead = null;
-  private ReadaheadPool raPool = ReadaheadPool.getInstance();
-  private boolean readahead;
-  private int readaheadLength;
-
-  public static final Log LOG = LogFactory.getLog(IFileInputStream.class);
-
-  private boolean disableChecksumValidation = false;
-  
-  /**
-   * Create a checksum input stream that reads
-   * @param in The input stream to be verified for checksum.
-   * @param len The length of the input stream including checksum bytes.
-   */
-  public IFileInputStream(InputStream in, long len, Configuration conf) {
-    this.in = in;
-    this.inFd = getFileDescriptorIfAvail(in);
-    sum = DataChecksum.newDataChecksum(DataChecksum.Type.CRC32, 
-        Integer.MAX_VALUE);
-    checksumSize = sum.getChecksumSize();
-    buffer = new byte[4096];
-    offset = 0;
-    length = len;
-    dataLength = length - checksumSize;
-
-    conf = (conf != null) ? conf : new Configuration();
-    readahead = conf.getBoolean(TezJobConfig.TEZ_ENGINE_IFILE_READAHEAD,
-        TezJobConfig.DEFAULT_TEZ_ENGINE_IFILE_READAHEAD);
-    readaheadLength = conf.getInt(TezJobConfig.TEZ_ENGINE_IFILE_READAHEAD_BYTES,
-        TezJobConfig.DEFAULT_TEZ_ENGINE_IFILE_READAHEAD_BYTES);
-
-    doReadahead();
-  }
-
-  private static FileDescriptor getFileDescriptorIfAvail(InputStream in) {
-    FileDescriptor fd = null;
-    try {
-      if (in instanceof HasFileDescriptor) {
-        fd = ((HasFileDescriptor)in).getFileDescriptor();
-      } else if (in instanceof FileInputStream) {
-        fd = ((FileInputStream)in).getFD();
-      }
-    } catch (IOException e) {
-      LOG.info("Unable to determine FileDescriptor", e);
-    }
-    return fd;
-  }
-
-  /**
-   * Close the input stream. Note that we need to read to the end of the
-   * stream to validate the checksum.
-   */
-  @Override
-  public void close() throws IOException {
-
-    if (curReadahead != null) {
-      curReadahead.cancel();
-    }
-    if (currentOffset < dataLength) {
-      byte[] t = new byte[Math.min((int)
-            (Integer.MAX_VALUE & (dataLength - currentOffset)), 32 * 1024)];
-      while (currentOffset < dataLength) {
-        int n = read(t, 0, t.length);
-        if (0 == n) {
-          throw new EOFException("Could not validate checksum");
-        }
-      }
-    }
-    in.close();
-  }
-  
-  @Override
-  public long skip(long n) throws IOException {
-   throw new IOException("Skip not supported for IFileInputStream");
-  }
-  
-  public long getPosition() {
-    return (currentOffset >= dataLength) ? dataLength : currentOffset;
-  }
-  
-  public long getSize() {
-    return checksumSize;
-  }
-
-  private void checksum(byte[] b, int off, int len) {
-    if(len >= buffer.length) {
-      sum.update(buffer, 0, offset);
-      offset = 0;
-      sum.update(b, off, len);
-      return;
-    }
-    final int remaining = buffer.length - offset;
-    if(len > remaining) {
-      sum.update(buffer, 0, offset);
-      offset = 0;
-    }
-    /* now we should have len < buffer.length */
-    System.arraycopy(b, off, buffer, offset, len);
-    offset += len;
-  }
-  
-  /**
-   * Read bytes from the stream.
-   * At EOF, checksum is validated, but the checksum
-   * bytes are not passed back in the buffer. 
-   */
-  public int read(byte[] b, int off, int len) throws IOException {
-
-    if (currentOffset >= dataLength) {
-      return -1;
-    }
-
-    doReadahead();
-
-    return doRead(b,off,len);
-  }
-
-  private void doReadahead() {
-    if (raPool != null && inFd != null && readahead) {
-      curReadahead = raPool.readaheadStream(
-          "ifile", inFd,
-          currentOffset, readaheadLength, dataLength,
-          curReadahead);
-    }
-  }
-
-  /**
-   * Read bytes from the stream.
-   * At EOF, checksum is validated and sent back
-   * as the last four bytes of the buffer. The caller should handle
-   * these bytes appropriately
-   */
-  public int readWithChecksum(byte[] b, int off, int len) throws IOException {
-
-    if (currentOffset == length) {
-      return -1;
-    }
-    else if (currentOffset >= dataLength) {
-      // If the previous read drained off all the data, then just return
-      // the checksum now. Note that checksum validation would have 
-      // happened in the earlier read
-      int lenToCopy = (int) (checksumSize - (currentOffset - dataLength));
-      if (len < lenToCopy) {
-        lenToCopy = len;
-      }
-      System.arraycopy(csum, (int) (currentOffset - dataLength), b, off, 
-          lenToCopy);
-      currentOffset += lenToCopy;
-      return lenToCopy;
-    }
-
-    int bytesRead = doRead(b,off,len);
-
-    if (currentOffset == dataLength) {
-      if (len >= bytesRead + checksumSize) {
-        System.arraycopy(csum, 0, b, off + bytesRead, checksumSize);
-        bytesRead += checksumSize;
-        currentOffset += checksumSize;
-      }
-    }
-    return bytesRead;
-  }
-
-  private int doRead(byte[]b, int off, int len) throws IOException {
-    
-    // If we are trying to read past the end of data, just read
-    // the left over data
-    if (currentOffset + len > dataLength) {
-      len = (int) dataLength - (int)currentOffset;
-    }
-    
-    int bytesRead = in.read(b, off, len);
-
-    if (bytesRead < 0) {
-      throw new ChecksumException("Checksum Error", 0);
-    }
-
-    checksum(b, off, bytesRead);
-
-    currentOffset += bytesRead;
-
-    if (disableChecksumValidation) {
-      return bytesRead;
-    }
-    
-    if (currentOffset == dataLength) {
-      // The last four bytes are checksum. Strip them and verify
-      sum.update(buffer, 0, offset);
-      csum = new byte[checksumSize];
-      IOUtils.readFully(in, csum, 0, checksumSize);
-      if (!sum.compare(csum, 0)) {
-        throw new ChecksumException("Checksum Error", 0);
-      }
-    }
-    return bytesRead;
-  }
-
-
-  @Override
-  public int read() throws IOException {    
-    b[0] = 0;
-    int l = read(b,0,1);
-    if (l < 0)  return l;
-    
-    // Upgrade the b[0] to an int so as not to misinterpret the
-    // first bit of the byte as a sign bit
-    int result = 0xFF & b[0];
-    return result;
-  }
-
-  public byte[] getChecksum() {
-    return csum;
-  }
-
-  void disableChecksumValidation() {
-    disableChecksumValidation = true;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/IFileOutputStream.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/IFileOutputStream.java b/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/IFileOutputStream.java
deleted file mode 100644
index 3b39900..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/IFileOutputStream.java
+++ /dev/null
@@ -1,129 +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.engine.common.sort.impl;
-
-import java.io.FilterOutputStream;
-import java.io.IOException;
-import java.io.OutputStream;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.util.DataChecksum;
-/**
- * A Checksum output stream.
- * Checksum for the contents of the file is calculated and
- * appended to the end of the file on close of the stream.
- * Used for IFiles
- */
-@InterfaceAudience.Private
-@InterfaceStability.Unstable
-public class IFileOutputStream extends FilterOutputStream {
-
-  /**
-   * The output stream to be checksummed.
-   */
-  private final DataChecksum sum;
-  private byte[] barray;
-  private byte[] buffer;
-  private int offset;
-  private boolean closed = false;
-  private boolean finished = false;
-
-  /**
-   * Create a checksum output stream that writes
-   * the bytes to the given stream.
-   * @param out
-   */
-  public IFileOutputStream(OutputStream out) {
-    super(out);
-    sum = DataChecksum.newDataChecksum(DataChecksum.Type.CRC32,
-        Integer.MAX_VALUE);
-    barray = new byte[sum.getChecksumSize()];
-    buffer = new byte[4096];
-    offset = 0;
-  }
-
-  @Override
-  public void close() throws IOException {
-    if (closed) {
-      return;
-    }
-    closed = true;
-    finish();
-    out.close();
-  }
-
-  /**
-   * Finishes writing data to the output stream, by writing
-   * the checksum bytes to the end. The underlying stream is not closed.
-   * @throws IOException
-   */
-  public void finish() throws IOException {
-    if (finished) {
-      return;
-    }
-    finished = true;
-    sum.update(buffer, 0, offset);
-    sum.writeValue(barray, 0, false);
-    out.write (barray, 0, sum.getChecksumSize());
-    out.flush();
-  }
-
-  private void checksum(byte[] b, int off, int len) {
-    if(len >= buffer.length) {
-      sum.update(buffer, 0, offset);
-      offset = 0;
-      sum.update(b, off, len);
-      return;
-    }
-    final int remaining = buffer.length - offset;
-    if(len > remaining) {
-      sum.update(buffer, 0, offset);
-      offset = 0;
-    }
-    /*
-    // FIXME if needed re-enable this in debug mode
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("XXX checksum" +
-          " b=" + b + " off=" + off +
-          " buffer=" + " offset=" + offset +
-          " len=" + len);
-    }
-    */
-    /* now we should have len < buffer.length */
-    System.arraycopy(b, off, buffer, offset, len);
-    offset += len;
-  }
-
-  /**
-   * Write bytes to the stream.
-   */
-  @Override
-  public void write(byte[] b, int off, int len) throws IOException {
-    checksum(b, off, len);
-    out.write(b,off,len);
-  }
-
-  @Override
-  public void write(int b) throws IOException {
-    barray[0] = (byte) (b & 0xFF);
-    write(barray,0,1);
-  }
-
-}


[48/50] [abbrv] TEZ-431. Implement fault tolerance, retries and event flow for dealing with failed inputs (bikas)

Posted by ss...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/3749a18f/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 520473d..cff71ab 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
@@ -22,7 +22,6 @@ import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.EnumSet;
-import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Iterator;
 import java.util.LinkedHashMap;
@@ -74,6 +73,7 @@ import org.apache.tez.dag.app.TaskHeartbeatHandler;
 import org.apache.tez.dag.app.dag.DAG;
 import org.apache.tez.dag.app.dag.EdgeManager;
 import org.apache.tez.dag.app.dag.Task;
+import org.apache.tez.dag.app.dag.TaskAttemptStateInternal;
 import org.apache.tez.dag.app.dag.TaskTerminationCause;
 import org.apache.tez.dag.app.dag.Vertex;
 import org.apache.tez.dag.app.dag.VertexScheduler;
@@ -83,6 +83,7 @@ import org.apache.tez.dag.app.dag.event.DAGEvent;
 import org.apache.tez.dag.app.dag.event.DAGEventDiagnosticsUpdate;
 import org.apache.tez.dag.app.dag.event.DAGEventType;
 import org.apache.tez.dag.app.dag.event.DAGEventVertexCompleted;
+import org.apache.tez.dag.app.dag.event.DAGEventVertexReRunning;
 import org.apache.tez.dag.app.dag.event.TaskAttemptEvent;
 import org.apache.tez.dag.app.dag.event.TaskAttemptEventAttemptFailed;
 import org.apache.tez.dag.app.dag.event.TaskAttemptEventStatusUpdate;
@@ -95,8 +96,8 @@ import org.apache.tez.dag.app.dag.event.VertexEventRouteEvent;
 import org.apache.tez.dag.app.dag.event.VertexEventSourceTaskAttemptCompleted;
 import org.apache.tez.dag.app.dag.event.VertexEventSourceVertexStarted;
 import org.apache.tez.dag.app.dag.event.VertexEventTaskAttemptCompleted;
-import org.apache.tez.dag.app.dag.event.VertexEventTaskAttemptFetchFailure;
 import org.apache.tez.dag.app.dag.event.VertexEventTaskCompleted;
+import org.apache.tez.dag.app.dag.event.VertexEventTaskReschedule;
 import org.apache.tez.dag.app.dag.event.VertexEventTermination;
 import org.apache.tez.dag.app.dag.event.VertexEventType;
 import org.apache.tez.dag.history.DAGHistoryEvent;
@@ -114,7 +115,6 @@ import org.apache.tez.runtime.api.impl.EventMetaData;
 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 org.apache.tez.runtime.records.TezDependentTaskCompletionEvent;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.HashMultiset;
@@ -130,18 +130,9 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
 
   private static final String LINE_SEPARATOR = System
       .getProperty("line.separator");
-  private static final TezDependentTaskCompletionEvent[]
-      EMPTY_TASK_ATTEMPT_COMPLETION_EVENTS =
-      new TezDependentTaskCompletionEvent[0];
 
   private static final Log LOG = LogFactory.getLog(VertexImpl.class);
 
-  //The maximum fraction of fetch failures allowed for a map
-  private static final double MAX_ALLOWED_FETCH_FAILURES_FRACTION = 0.5;
-
-  // Maximum no. of fetch-failure notifications after which map task is failed
-  private static final int MAX_FETCH_FAILURES_NOTIFICATIONS = 3;
-
   //final fields
   private final Clock clock;
 
@@ -160,7 +151,6 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
   private boolean lazyTasksCopyNeeded = false;
   // must be a linked map for ordering
   volatile LinkedHashMap<TezTaskID, Task> tasks = new LinkedHashMap<TezTaskID, Task>();
-  private List<byte[]> taskUserPayloads = null;
   private Object fullCountersLock = new Object();
   private TezCounters fullCounters = null;
   private Resource taskResource;
@@ -172,15 +162,11 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
   private int numStartedSourceVertices = 0;
   private int distanceFromRoot = 0;
 
-  private List<TezDependentTaskCompletionEvent> sourceTaskAttemptCompletionEvents;
   private final List<String> diagnostics = new ArrayList<String>();
 
   //task/attempt related datastructures
   @VisibleForTesting
-  final Map<TezTaskID, Integer> successSourceAttemptCompletionEventNoMap =
-    new HashMap<TezTaskID, Integer>();
-  private final Map<TezTaskAttemptID, Integer> fetchFailuresMapping =
-    new HashMap<TezTaskAttemptID, Integer>();
+  int numSuccessSourceAttemptCompletions = 0;
 
   List<InputSpec> inputSpecList;
   List<OutputSpec> outputSpecList;
@@ -212,7 +198,7 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
               VertexEventType.V_TERMINATE,
               new TerminateNewVertexTransition())
           .addTransition(VertexState.NEW, VertexState.ERROR,
-              VertexEventType.INTERNAL_ERROR,
+              VertexEventType.V_INTERNAL_ERROR,
               INTERNAL_ERROR_TRANSITION)
 
           // Transitions from INITED state
@@ -227,7 +213,7 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
               VertexEventType.V_TERMINATE,
               new TerminateInitedVertexTransition())
           .addTransition(VertexState.INITED, VertexState.ERROR,
-              VertexEventType.INTERNAL_ERROR,
+              VertexEventType.V_INTERNAL_ERROR,
               INTERNAL_ERROR_TRANSITION)
 
           // Transitions from RUNNING state
@@ -249,12 +235,9 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
           .addTransition(VertexState.RUNNING, VertexState.RUNNING,
               VertexEventType.V_TASK_RESCHEDULED,
               new TaskRescheduledTransition())
-          .addTransition(VertexState.RUNNING, VertexState.RUNNING,
-              VertexEventType.V_TASK_ATTEMPT_FETCH_FAILURE,
-              new TaskAttemptFetchFailureTransition())
           .addTransition(
               VertexState.RUNNING,
-              VertexState.ERROR, VertexEventType.INTERNAL_ERROR,
+              VertexState.ERROR, VertexEventType.V_INTERNAL_ERROR,
               INTERNAL_ERROR_TRANSITION)
           .addTransition(
               VertexState.RUNNING,
@@ -275,48 +258,49 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
               SOURCE_TASK_ATTEMPT_COMPLETED_EVENT_TRANSITION)
           .addTransition(
               VertexState.TERMINATING,
-              VertexState.ERROR, VertexEventType.INTERNAL_ERROR,
+              VertexState.ERROR, VertexEventType.V_INTERNAL_ERROR,
               INTERNAL_ERROR_TRANSITION)
           // Ignore-able events
           .addTransition(VertexState.TERMINATING, VertexState.TERMINATING,
               EnumSet.of(VertexEventType.V_TERMINATE,
-                  VertexEventType.V_TASK_RESCHEDULED,
-                  VertexEventType.V_TASK_ATTEMPT_FETCH_FAILURE))
+                  VertexEventType.V_TASK_RESCHEDULED))
 
           // Transitions from SUCCEEDED state
           .addTransition(
               VertexState.SUCCEEDED,
-              VertexState.ERROR, VertexEventType.INTERNAL_ERROR,
+              VertexState.ERROR, VertexEventType.V_INTERNAL_ERROR,
               INTERNAL_ERROR_TRANSITION)
+          .addTransition(VertexState.SUCCEEDED, 
+              EnumSet.of(VertexState.RUNNING, VertexState.FAILED), 
+              VertexEventType.V_TASK_RESCHEDULED,
+              new TaskRescheduledAfterVertexSuccessTransition())
+
           // Ignore-able events
           .addTransition(VertexState.SUCCEEDED, VertexState.SUCCEEDED,
               EnumSet.of(VertexEventType.V_TERMINATE,
-                  VertexEventType.V_TASK_ATTEMPT_FETCH_FAILURE,
                   VertexEventType.V_TASK_ATTEMPT_COMPLETED,
                   VertexEventType.V_TASK_COMPLETED))
 
           // Transitions from FAILED state
           .addTransition(
               VertexState.FAILED,
-              VertexState.ERROR, VertexEventType.INTERNAL_ERROR,
+              VertexState.ERROR, VertexEventType.V_INTERNAL_ERROR,
               INTERNAL_ERROR_TRANSITION)
           // Ignore-able events
           .addTransition(VertexState.FAILED, VertexState.FAILED,
               EnumSet.of(VertexEventType.V_TERMINATE,
-                  VertexEventType.V_TASK_ATTEMPT_FETCH_FAILURE,
                   VertexEventType.V_TASK_ATTEMPT_COMPLETED,
                   VertexEventType.V_TASK_COMPLETED))
 
           // Transitions from KILLED state
           .addTransition(
               VertexState.KILLED,
-              VertexState.ERROR, VertexEventType.INTERNAL_ERROR,
+              VertexState.ERROR, VertexEventType.V_INTERNAL_ERROR,
               INTERNAL_ERROR_TRANSITION)
           // Ignore-able events
           .addTransition(VertexState.KILLED, VertexState.KILLED,
               EnumSet.of(VertexEventType.V_TERMINATE,
                   VertexEventType.V_START,
-                  VertexEventType.V_TASK_ATTEMPT_FETCH_FAILURE,
                   VertexEventType.V_TASK_ATTEMPT_COMPLETED,
                   VertexEventType.V_TASK_COMPLETED))
 
@@ -330,8 +314,7 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
                   VertexEventType.V_TASK_ATTEMPT_COMPLETED,
                   VertexEventType.V_TASK_RESCHEDULED,
                   VertexEventType.V_DIAGNOSTIC_UPDATE,
-                  VertexEventType.V_TASK_ATTEMPT_FETCH_FAILURE,
-                  VertexEventType.INTERNAL_ERROR))
+                  VertexEventType.V_INTERNAL_ERROR))
           // create the topology tables
           .installTopology();
 
@@ -550,32 +533,6 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
   }
 
   @Override
-  public TezDependentTaskCompletionEvent[] getTaskAttemptCompletionEvents(
-      TezTaskAttemptID attemptID, int fromEventId, int maxEvents) {
-    TezDependentTaskCompletionEvent[] events = EMPTY_TASK_ATTEMPT_COMPLETION_EVENTS;
-    readLock.lock();
-    try {
-      if (sourceTaskAttemptCompletionEvents.size() > fromEventId) {
-        int actualMax = Math.min(maxEvents,
-            (sourceTaskAttemptCompletionEvents.size() - fromEventId));
-        events = sourceTaskAttemptCompletionEvents.subList(fromEventId,
-            actualMax + fromEventId).toArray(events);
-        // create a copy if user payload is different per task
-        if(taskUserPayloads != null && events.length > 0) {
-          int taskId = attemptID.getTaskID().getId();
-          byte[] userPayload = taskUserPayloads.get(taskId);
-          TezDependentTaskCompletionEvent event = events[0].clone();
-          event.setUserPayload(userPayload);
-          events[0] = event;
-        }
-      }
-      return events;
-    } finally {
-      readLock.unlock();
-    }
-  }
-
-  @Override
   public List<String> getDiagnostics() {
     readLock.lock();
     try {
@@ -683,6 +640,8 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
     }
   }
 
+  // TODO Create InputReadyVertexManager that schedules when there is something 
+  // to read and use that as default instead of ImmediateStart.TEZ-480
   @Override
   public void scheduleTasks(Collection<TezTaskID> taskIDs) {
     readLock.lock();
@@ -808,7 +767,7 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
         LOG.error("Can't handle " + message, e);
         addDiagnostic(message);
         eventHandler.handle(new VertexEvent(this.vertexId,
-            VertexEventType.INTERNAL_ERROR));
+            VertexEventType.V_INTERNAL_ERROR));
       }
 
       if (oldState != getInternalState()) {
@@ -1028,10 +987,6 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
 
         checkTaskLimits();
 
-        // TODO should depend on source num tasks
-        vertex.sourceTaskAttemptCompletionEvents =
-            new ArrayList<TezDependentTaskCompletionEvent>(vertex.numTasks + 10);
-
         // create the Tasks but don't start them yet
         createTasks(vertex);
 
@@ -1269,98 +1224,43 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
   SingleArcTransition<VertexImpl, VertexEvent> {
     @Override
     public void transition(VertexImpl vertex, VertexEvent event) {
-      TezDependentTaskCompletionEvent tce =
+      VertexEventTaskAttemptCompleted completionEvent =
           ((VertexEventSourceTaskAttemptCompleted) event).getCompletionEvent();
-      if(LOG.isDebugEnabled()) {
-        LOG.debug("Adding completion event to vertex: " + vertex.getName()
-            + " attempt: " + tce.getTaskAttemptID());
-      }
-      // Add the TaskAttemptCompletionEvent
-      //eventId is equal to index in the arraylist
-      tce.setEventId(vertex.sourceTaskAttemptCompletionEvents.size());
-      vertex.sourceTaskAttemptCompletionEvents.add(tce);
-      // TODO this needs to be ordered/grouped by source vertices or else
-      // my tasks will not know which events are for which vertices' tasks. This
-      // differentiation was not needed for MR because there was only 1 M stage.
-      // if the tce is sent to the task then a solution could be to add vertex
-      // name to the tce
-      // need to send vertex name and task index in that vertex
-
-      TezTaskAttemptID attemptId = tce.getTaskAttemptID();
-      TezTaskID taskId = attemptId.getTaskID();
-      //make the previous completion event as obsolete if it exists
-      if (TezDependentTaskCompletionEvent.Status.SUCCEEDED.equals(tce.getStatus())) {
-        vertex.vertexScheduler.onSourceTaskCompleted(attemptId, tce);
-        Object successEventNo =
-            vertex.successSourceAttemptCompletionEventNoMap.remove(taskId);
-        if (successEventNo != null) {
-          TezDependentTaskCompletionEvent successEvent =
-              vertex.sourceTaskAttemptCompletionEvents.get((Integer) successEventNo);
-          successEvent.setTaskStatus(TezDependentTaskCompletionEvent.Status.OBSOLETE);
-        }
-        vertex.successSourceAttemptCompletionEventNoMap.put(taskId, tce.getEventId());
+      LOG.info("Source task attempt completed for vertex: " + vertex.getVertexId()
+            + " attempt: " + completionEvent.getTaskAttemptId()
+            + " with state: " + completionEvent.getTaskAttemptState());
+      
+      if (TaskAttemptStateInternal.SUCCEEDED.equals(completionEvent
+          .getTaskAttemptState())) {
+        vertex.numSuccessSourceAttemptCompletions++;
+        vertex.vertexScheduler.onSourceTaskCompleted(completionEvent
+            .getTaskAttemptId());
       }
 
     }
   }
 
-  // TODO Why is TA event coming directly to Vertex instead of TA -> Task -> Vertex
   private static class TaskAttemptCompletedEventTransition implements
       SingleArcTransition<VertexImpl, VertexEvent> {
     @Override
     public void transition(VertexImpl vertex, VertexEvent event) {
-      TezDependentTaskCompletionEvent tce =
-        ((VertexEventTaskAttemptCompleted) event).getCompletionEvent();
+      VertexEventTaskAttemptCompleted completionEvent =
+        ((VertexEventTaskAttemptCompleted) event);
 
-      // TODO this should only be sent for successful events? looks like all
-      // need to be sent in the existing shuffle code
+      // If different tasks were connected to different destination vertices
+      // then this would need to be sent via the edges
       // Notify all target vertices
       if (vertex.targetVertices != null) {
         for (Vertex targetVertex : vertex.targetVertices.keySet()) {
           vertex.eventHandler.handle(
               new VertexEventSourceTaskAttemptCompleted(
-                  targetVertex.getVertexId(), tce)
+                  targetVertex.getVertexId(), completionEvent)
               );
         }
       }
     }
   }
 
-  private static class TaskAttemptFetchFailureTransition implements
-      SingleArcTransition<VertexImpl, VertexEvent> {
-    @Override
-    public void transition(VertexImpl vertex, VertexEvent event) {
-      VertexEventTaskAttemptFetchFailure fetchfailureEvent =
-          (VertexEventTaskAttemptFetchFailure) event;
-      for (TezTaskAttemptID mapId : fetchfailureEvent.getSources()) {
-        Integer fetchFailures = vertex.fetchFailuresMapping.get(mapId);
-        fetchFailures = (fetchFailures == null) ? 1 : (fetchFailures+1);
-        vertex.fetchFailuresMapping.put(mapId, fetchFailures);
-
-        //get number of running reduces
-        int runningReduceTasks = 0;
-        for (TezTaskID taskId : vertex.tasks.keySet()) {
-          if (TaskState.RUNNING.equals(vertex.tasks.get(taskId).getState())) {
-            runningReduceTasks++;
-          }
-        }
-
-        float failureRate = runningReduceTasks == 0 ? 1.0f :
-          (float) fetchFailures / runningReduceTasks;
-        // declare faulty if fetch-failures >= max-allowed-failures
-        boolean isMapFaulty =
-            (failureRate >= MAX_ALLOWED_FETCH_FAILURES_FRACTION);
-        if (fetchFailures >= MAX_FETCH_FAILURES_NOTIFICATIONS && isMapFaulty) {
-          LOG.info("Too many fetch-failures for output of task attempt: " +
-              mapId + " ... raising fetch failure to source");
-          vertex.eventHandler.handle(new TaskAttemptEvent(mapId,
-              TaskAttemptEventType.TA_TOO_MANY_FETCH_FAILURES));
-          vertex.fetchFailuresMapping.remove(mapId);
-        }
-      }
-    }
-  }
-
   private static class TaskCompletedTransition implements
       MultipleArcTransition<VertexImpl, VertexEvent, VertexState> {
 
@@ -1413,12 +1313,39 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
       SingleArcTransition<VertexImpl, VertexEvent> {
     @Override
     public void transition(VertexImpl vertex, VertexEvent event) {
-      //succeeded map task is restarted back
+      //succeeded task is restarted back
       vertex.completedTaskCount--;
       vertex.succeededTaskCount--;
     }
   }
+  
+  private static class TaskRescheduledAfterVertexSuccessTransition implements
+    MultipleArcTransition<VertexImpl, VertexEvent, VertexState> {
 
+    @Override
+    public VertexState transition(VertexImpl vertex, VertexEvent event) {
+      if (vertex.committer instanceof NullVertexOutputCommitter) {
+        LOG.info(vertex.getVertexId() + " back to running due to rescheduling "
+            + ((VertexEventTaskReschedule)event).getTaskID());
+        (new TaskRescheduledTransition()).transition(vertex, event);
+        // inform the DAG that we are re-running
+        vertex.eventHandler.handle(new DAGEventVertexReRunning(vertex.getVertexId()));
+        return VertexState.RUNNING;
+      }
+      
+      LOG.info(vertex.getVertexId() + " failed due to post-commit rescheduling of "
+          + ((VertexEventTaskReschedule)event).getTaskID());
+      // terminate any running tasks
+      vertex.enactKill(VertexTerminationCause.OWN_TASK_FAILURE,
+          TaskTerminationCause.OWN_TASK_FAILURE);
+      // since the DAG thinks this vertex is completed it must be notified of 
+      // an error
+      vertex.eventHandler.handle(new DAGEvent(vertex.getDAGId(),
+          DAGEventType.INTERNAL_ERROR));
+      return VertexState.FAILED;
+    }
+  }
+  
   private void addDiagnostic(String diag) {
     diagnostics.add(diag);
   }

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/3749a18f/tez-dag/src/main/java/org/apache/tez/runtime/records/TezDependentTaskCompletionEvent.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/runtime/records/TezDependentTaskCompletionEvent.java b/tez-dag/src/main/java/org/apache/tez/runtime/records/TezDependentTaskCompletionEvent.java
deleted file mode 100644
index fd4c1ee..0000000
--- a/tez-dag/src/main/java/org/apache/tez/runtime/records/TezDependentTaskCompletionEvent.java
+++ /dev/null
@@ -1,228 +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.runtime.records;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.io.WritableUtils;
-import org.apache.tez.dag.records.TezTaskAttemptID;
-
-/**
- * This is used to track task completion events on 
- * job tracker. 
- */
-@InterfaceAudience.Public
-@InterfaceStability.Stable
-// TODO TEZAM3 This needs to be more generic. Maybe some kind of a serialized
-// blob - which can be interpretted by the Input plugin.
-public class TezDependentTaskCompletionEvent implements Writable {
-  @InterfaceAudience.Public
-  @InterfaceStability.Evolving
-  // TODO EVENTUALLY - Remove TIPFAILED state ?
-  static public enum Status {FAILED, KILLED, SUCCEEDED, OBSOLETE, TIPFAILED};
-    
-  private int eventId;
-  private int taskRunTime; // using int since runtime is the time difference
-  private TezTaskAttemptID taskAttemptId;
-  private long dataSize;
-  Status status;
-  byte[] userPayload;
-  // TODO TEZAM2 Get rid of the isMap field. Job specific type information can be determined from TaskAttemptId.getTaskType
-//  boolean isMap = false;
-  public static final TezDependentTaskCompletionEvent[] EMPTY_ARRAY = 
-    new TezDependentTaskCompletionEvent[0];
-
-  public TezDependentTaskCompletionEvent() {
-    taskAttemptId = new TezTaskAttemptID();
-  }
-  
-  /**
-   * Constructor. eventId should be created externally and incremented
-   * per event for each job. 
-   * @param eventId event id, event id should be unique and assigned in
-   *  incrementally, starting from 0. 
-   * @param taskAttemptId task id
-   * @param status task's status 
-   * @param taskTrackerHttp task tracker's host:port for http. 
-   */
-  public TezDependentTaskCompletionEvent(int eventId, 
-                             TezTaskAttemptID taskAttemptId,
-//                             boolean isMap,
-                             Status status, 
-                             int runTime,
-                             long dataSize){
-      
-    this.taskAttemptId = taskAttemptId;
-//    this.isMap = isMap;
-    this.eventId = eventId; 
-    this.status =status; 
-    this.taskRunTime = runTime;
-    this.dataSize = dataSize;
-  }
-  
-  public TezDependentTaskCompletionEvent clone() {
-    TezDependentTaskCompletionEvent clone = new TezDependentTaskCompletionEvent(
-        this.eventId, this.taskAttemptId, this.status, 
-        this.taskRunTime, this.dataSize);
-    
-    return clone;
-  }
-  
-  /**
-   * Returns event Id. 
-   * @return event id
-   */
-  public int getEventId() {
-    return eventId;
-  }
-
-  /**
-   * Returns task id. 
-   * @return task id
-   */
-  public TezTaskAttemptID getTaskAttemptID() {
-    return taskAttemptId;
-  }
-  
-  /**
-   * Returns enum Status.SUCESS or Status.FAILURE.
-   * @return task tracker status
-   */
-  public Status getStatus() {
-    return status;
-  }
-  
-  /**
-   * Returns time (in millisec) the task took to complete. 
-   */
-  public int getTaskRunTime() {
-    return taskRunTime;
-  }
-  
-  /**
-   * Return size of output produced by the task
-   */
-  public long getDataSize() {
-    return dataSize;
-  }
-  
-  /**
-   * @return user payload. Maybe null
-   */
-  public byte[] getUserPayload() {
-    return userPayload;
-  }
-
-  /**
-   * Set the task completion time
-   * @param taskCompletionTime time (in millisec) the task took to complete
-   */
-  protected void setTaskRunTime(int taskCompletionTime) {
-    this.taskRunTime = taskCompletionTime;
-  }
-
-  /**
-   * set event Id. should be assigned incrementally starting from 0. 
-   * @param eventId
-   */
-  public void setEventId(int eventId) {
-    this.eventId = eventId;
-  }
-
-  /**
-   * Sets task id. 
-   * @param taskId
-   */
-  public void setTaskAttemptID(TezTaskAttemptID taskId) {
-    this.taskAttemptId = taskId;
-  }
-  
-  /**
-   * Set task status. 
-   * @param status
-   */
-  public void setTaskStatus(Status status) {
-    this.status = status;
-  }
-  
-  /**
-   * Set the user payload
-   * @param userPayload
-   */
-  public void setUserPayload(byte[] userPayload) {
-    this.userPayload = userPayload;
-  }
-    
-  @Override
-  public String toString(){
-    StringBuffer buf = new StringBuffer(); 
-    buf.append("Task Id : "); 
-    buf.append(taskAttemptId); 
-    buf.append(", Status : ");  
-    buf.append(status.name());
-    return buf.toString();
-  }
-    
-  @Override
-  public boolean equals(Object o) {
-    // not counting userPayload as that is a piggyback mechanism
-    if(o == null)
-      return false;
-    if(o.getClass().equals(this.getClass())) {
-      TezDependentTaskCompletionEvent event = (TezDependentTaskCompletionEvent) o;
-      return this.eventId == event.getEventId()
-             && this.status.equals(event.getStatus())
-             && this.taskAttemptId.equals(event.getTaskAttemptID()) 
-             && this.taskRunTime == event.getTaskRunTime()
-             && this.dataSize == event.getDataSize();
-    }
-    return false;
-  }
-
-  @Override
-  public int hashCode() {
-    return toString().hashCode(); 
-  }
-
-  @Override
-  public void write(DataOutput out) throws IOException {
-    taskAttemptId.write(out);
-//    out.writeBoolean(isMap);
-    WritableUtils.writeEnum(out, status);
-    WritableUtils.writeVInt(out, taskRunTime);
-    WritableUtils.writeVInt(out, eventId);
-    WritableUtils.writeCompressedByteArray(out, userPayload);
-  }
-
-  @Override
-  public void readFields(DataInput in) throws IOException {
-    taskAttemptId.readFields(in);
-//    isMap = in.readBoolean();
-    status = WritableUtils.readEnum(in, Status.class);
-    taskRunTime = WritableUtils.readVInt(in);
-    eventId = WritableUtils.readVInt(in);
-    userPayload = WritableUtils.readCompressedByteArray(in);
-  }
-  
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/3749a18f/tez-dag/src/main/java/org/apache/tez/runtime/records/TezTaskDependencyCompletionEventsUpdate.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/runtime/records/TezTaskDependencyCompletionEventsUpdate.java b/tez-dag/src/main/java/org/apache/tez/runtime/records/TezTaskDependencyCompletionEventsUpdate.java
deleted file mode 100644
index ff4f267..0000000
--- a/tez-dag/src/main/java/org/apache/tez/runtime/records/TezTaskDependencyCompletionEventsUpdate.java
+++ /dev/null
@@ -1,64 +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.runtime.records;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-
-import org.apache.hadoop.io.Writable;
-
-
-public class TezTaskDependencyCompletionEventsUpdate implements Writable {
-  TezDependentTaskCompletionEvent[] events;
-  boolean reset;
-
-  public TezTaskDependencyCompletionEventsUpdate() { }
-
-  public TezTaskDependencyCompletionEventsUpdate(
-      TezDependentTaskCompletionEvent[] events, boolean reset) {
-    this.events = events;
-    this.reset = reset;
-  }
-
-  public boolean shouldReset() {
-    return reset;
-  }
-
-  public TezDependentTaskCompletionEvent[] getDependentTaskCompletionEvents() {
-    return events;
-  }
-  
-  public void write(DataOutput out) throws IOException {
-    out.writeBoolean(reset);
-    out.writeInt(events.length);
-    for (TezDependentTaskCompletionEvent event : events) {
-      event.write(out);
-    }
-  }
-
-  public void readFields(DataInput in) throws IOException {
-    reset = in.readBoolean();
-    events = new TezDependentTaskCompletionEvent[in.readInt()];
-    for (int i = 0; i < events.length; ++i) {
-      events[i] = new TezDependentTaskCompletionEvent();
-      events[i].readFields(in);
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/3749a18f/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 f2717be..434a4b5 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
@@ -71,6 +71,7 @@ import org.apache.tez.dag.app.dag.event.TaskAttemptEvent;
 import org.apache.tez.dag.app.dag.event.TaskAttemptEventContainerTerminated;
 import org.apache.tez.dag.app.dag.event.TaskAttemptEventContainerTerminating;
 import org.apache.tez.dag.app.dag.event.TaskAttemptEventKillRequest;
+import org.apache.tez.dag.app.dag.event.TaskAttemptEventOutputFailed;
 import org.apache.tez.dag.app.dag.event.TaskAttemptEventSchedule;
 import org.apache.tez.dag.app.dag.event.TaskAttemptEventStartedRemotely;
 import org.apache.tez.dag.app.dag.event.TaskAttemptEventType;
@@ -82,7 +83,10 @@ import org.apache.tez.dag.records.TezDAGID;
 import org.apache.tez.dag.records.TezTaskAttemptID;
 import org.apache.tez.dag.records.TezTaskID;
 import org.apache.tez.dag.records.TezVertexID;
+import org.apache.tez.runtime.api.events.InputReadErrorEvent;
+import org.apache.tez.runtime.api.impl.EventMetaData;
 import org.apache.tez.runtime.api.impl.TaskSpec;
+import org.apache.tez.runtime.api.impl.TezEvent;
 import org.junit.Test;
 import org.mockito.ArgumentCaptor;
 
@@ -583,7 +587,7 @@ public class TestTaskAttempt {
   @Test
   // Verifies that multiple TooManyFetchFailures are handled correctly by the
   // TaskAttempt.
-  public void testMultipleTooManyFetchFailures() throws Exception {
+  public void testMultipleOutputFailed() throws Exception {
     ApplicationId appId = ApplicationId.newInstance(1, 2);
     ApplicationAttemptId appAttemptId = ApplicationAttemptId.newInstance(
         appId, 0);
@@ -641,9 +645,14 @@ public class TestTaskAttempt {
     verify(eventHandler, times(expectedEventsTillSucceeded)).handle(arg.capture());
     verifyEventType(arg.getAllValues(), TaskEventTAUpdate.class, 2);
 
-    taImpl.handle(new TaskAttemptEvent(taskAttemptID,
-        TaskAttemptEventType.TA_TOO_MANY_FETCH_FAILURES));
-    int expectedEventsAfterFetchFailure = expectedEventsTillSucceeded + 3;
+    InputReadErrorEvent reEvent = new InputReadErrorEvent("", 0, 1);
+    EventMetaData mockMeta = mock(EventMetaData.class);
+    TezTaskAttemptID mockDestId = mock(TezTaskAttemptID.class);
+    when(mockMeta.getTaskAttemptID()).thenReturn(mockDestId);
+    TezEvent tzEvent = new TezEvent(reEvent, mockMeta);
+    taImpl.handle(new TaskAttemptEventOutputFailed(taskAttemptID, tzEvent, 1));
+    int expectedEventsAfterFetchFailure = expectedEventsTillSucceeded + 2;
+    arg.getAllValues().clear();
     verify(eventHandler, times(expectedEventsAfterFetchFailure)).handle(arg.capture());
     verifyEventType(
         arg.getAllValues().subList(expectedEventsTillSucceeded,
@@ -651,8 +660,7 @@ public class TestTaskAttempt {
 
     assertEquals("Task attempt is not in FAILED state", taImpl.getState(),
         TaskAttemptState.FAILED);
-    taImpl.handle(new TaskAttemptEvent(taskAttemptID,
-        TaskAttemptEventType.TA_TOO_MANY_FETCH_FAILURES));
+    taImpl.handle(new TaskAttemptEventOutputFailed(taskAttemptID, tzEvent, 1));
     assertEquals("Task attempt is not in FAILED state, still",
         taImpl.getState(), TaskAttemptState.FAILED);
     assertFalse(

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/3749a18f/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 b524f6a..2cbf1fe 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
@@ -63,6 +63,7 @@ import org.apache.tez.dag.app.TaskHeartbeatHandler;
 import org.apache.tez.dag.app.dag.DAG;
 import org.apache.tez.dag.app.dag.EdgeManager;
 import org.apache.tez.dag.app.dag.Task;
+import org.apache.tez.dag.app.dag.TaskAttemptStateInternal;
 import org.apache.tez.dag.app.dag.Vertex;
 import org.apache.tez.dag.app.dag.VertexState;
 import org.apache.tez.dag.app.dag.VertexTerminationCause;
@@ -82,8 +83,6 @@ 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.records.TezDependentTaskCompletionEvent;
-import org.apache.tez.runtime.records.TezDependentTaskCompletionEvent.Status;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
@@ -977,7 +976,6 @@ public class TestVertexImpl {
     TezTaskID t2_v4 = new TezTaskID(v4.getVertexId(), 1);
     TezTaskID t1_v5 = new TezTaskID(v5.getVertexId(), 0);
     TezTaskID t2_v5 = new TezTaskID(v5.getVertexId(), 1);
-    TezTaskID t1_v6 = new TezTaskID(v6.getVertexId(), 0);
 
     TezTaskAttemptID ta1_t1_v4 = new TezTaskAttemptID(t1_v4, 0);
     TezTaskAttemptID ta2_t1_v4 = new TezTaskAttemptID(t1_v4, 0);
@@ -985,33 +983,13 @@ public class TestVertexImpl {
     TezTaskAttemptID ta1_t1_v5 = new TezTaskAttemptID(t1_v5, 0);
     TezTaskAttemptID ta1_t2_v5 = new TezTaskAttemptID(t2_v5, 0);
     TezTaskAttemptID ta2_t2_v5 = new TezTaskAttemptID(t2_v5, 0);
-    TezTaskAttemptID ta1_t1_v6 = new TezTaskAttemptID(t1_v6, 0);
-
-    TezDependentTaskCompletionEvent cEvt1 =
-        new TezDependentTaskCompletionEvent(1, ta1_t1_v4,
-            Status.FAILED, 3, 0);
-    TezDependentTaskCompletionEvent cEvt2 =
-        new TezDependentTaskCompletionEvent(2, ta2_t1_v4,
-            Status.SUCCEEDED, 4, 1);
-    TezDependentTaskCompletionEvent cEvt3 =
-        new TezDependentTaskCompletionEvent(2, ta1_t2_v4,
-            Status.SUCCEEDED, 5, 2);
-    TezDependentTaskCompletionEvent cEvt4 =
-        new TezDependentTaskCompletionEvent(2, ta1_t1_v5,
-            Status.SUCCEEDED, 5, 3);
-    TezDependentTaskCompletionEvent cEvt5 =
-        new TezDependentTaskCompletionEvent(1, ta1_t2_v5,
-            Status.FAILED, 3, 4);
-    TezDependentTaskCompletionEvent cEvt6 =
-        new TezDependentTaskCompletionEvent(2, ta2_t2_v5,
-            Status.SUCCEEDED, 4, 5);
-
-    v4.handle(new VertexEventTaskAttemptCompleted(cEvt1));
-    v4.handle(new VertexEventTaskAttemptCompleted(cEvt2));
-    v4.handle(new VertexEventTaskAttemptCompleted(cEvt3));
-    v5.handle(new VertexEventTaskAttemptCompleted(cEvt4));
-    v5.handle(new VertexEventTaskAttemptCompleted(cEvt5));
-    v5.handle(new VertexEventTaskAttemptCompleted(cEvt6));
+
+    v4.handle(new VertexEventTaskAttemptCompleted(ta1_t1_v4, TaskAttemptStateInternal.FAILED));
+    v4.handle(new VertexEventTaskAttemptCompleted(ta2_t1_v4, TaskAttemptStateInternal.SUCCEEDED));
+    v4.handle(new VertexEventTaskAttemptCompleted(ta1_t2_v4, TaskAttemptStateInternal.SUCCEEDED));
+    v5.handle(new VertexEventTaskAttemptCompleted(ta1_t1_v5, TaskAttemptStateInternal.SUCCEEDED));
+    v5.handle(new VertexEventTaskAttemptCompleted(ta1_t2_v5, TaskAttemptStateInternal.FAILED));
+    v5.handle(new VertexEventTaskAttemptCompleted(ta2_t2_v5, TaskAttemptStateInternal.SUCCEEDED));
 
     v4.handle(new VertexEventTaskCompleted(t1_v4, TaskState.SUCCEEDED));
     v4.handle(new VertexEventTaskCompleted(t2_v4, TaskState.SUCCEEDED));
@@ -1023,9 +1001,7 @@ public class TestVertexImpl {
     Assert.assertEquals(VertexState.SUCCEEDED, v5.getState());
 
     Assert.assertEquals(VertexState.RUNNING, v6.getState());
-    Assert.assertEquals(4, v6.successSourceAttemptCompletionEventNoMap.size());
-    Assert.assertEquals(6,
-        v6.getTaskAttemptCompletionEvents(ta1_t1_v6, 0, 100).length);
+    Assert.assertEquals(4, v6.numSuccessSourceAttemptCompletions);
 
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/3749a18f/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexScheduler.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexScheduler.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexScheduler.java
index 81715bd..b2e13e2 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexScheduler.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexScheduler.java
@@ -41,8 +41,8 @@ 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.mapreduce.hadoop.MRHelpers;
-import org.apache.tez.runtime.records.TezDependentTaskCompletionEvent;
 import org.junit.Assert;
+import org.junit.Ignore;
 import org.junit.Test;
 import org.mockito.invocation.InvocationOnMock;
 import org.mockito.stubbing.Answer;
@@ -53,6 +53,7 @@ public class TestVertexScheduler {
 
   @SuppressWarnings({ "unchecked", "rawtypes" })
   @Test(timeout = 5000)
+  @Ignore // TODO TEZ-481
   public void testShuffleVertexManagerAutoParallelism() throws IOException {
     Configuration conf = new Configuration();
     conf.setBoolean(
@@ -98,9 +99,6 @@ public class TestVertexScheduler {
     when(mockManagedVertex.getInputVertices()).thenReturn(mockInputVertices);
     
     
-    TezDependentTaskCompletionEvent mockEvent = 
-        mock(TezDependentTaskCompletionEvent.class);
-    
     mockInputVertices.put(mockSrcVertex1, new Edge(eProp1, mockEventHandler));
     mockInputVertices.put(mockSrcVertex2, new Edge(eProp2, mockEventHandler));
     mockInputVertices.put(mockSrcVertex3, new Edge(eProp3, mockEventHandler));
@@ -165,12 +163,12 @@ public class TestVertexScheduler {
         new TezTaskAttemptID(new TezTaskID(mockSrcVertexId3, 0), 0);
 
     // parallelism not change due to large data size
-    when(mockEvent.getDataSize()).thenReturn(5000L);
+    //when(mockEvent.getDataSize()).thenReturn(5000L);
     scheduler = createScheduler(conf, mockManagedVertex, 0.1f, 0.1f);
     scheduler.onVertexStarted();
     Assert.assertTrue(scheduler.pendingTasks.size() == 4); // no tasks scheduled
     Assert.assertTrue(scheduler.numSourceTasks == 4);
-    scheduler.onSourceTaskCompleted(mockSrcAttemptId11, mockEvent);
+    scheduler.onSourceTaskCompleted(mockSrcAttemptId11);
     // managedVertex tasks reduced
     verify(mockManagedVertex, times(0)).setParallelism(anyInt(), anyMap());
     Assert.assertEquals(0, scheduler.pendingTasks.size()); // all tasks scheduled
@@ -179,7 +177,7 @@ public class TestVertexScheduler {
     Assert.assertEquals(5000L, scheduler.completedSourceTasksOutputSize);
     
     // parallelism changed due to small data size
-    when(mockEvent.getDataSize()).thenReturn(500L);
+    //when(mockEvent.getDataSize()).thenReturn(500L);
     scheduledTasks.clear();
     Configuration procConf = new Configuration();
     ProcessorDescriptor procDesc = new ProcessorDescriptor("REDUCE");
@@ -191,23 +189,23 @@ public class TestVertexScheduler {
     Assert.assertEquals(4, scheduler.pendingTasks.size()); // no tasks scheduled
     Assert.assertEquals(4, scheduler.numSourceTasks);
     // task completion from non-bipartite stage does nothing
-    scheduler.onSourceTaskCompleted(mockSrcAttemptId31, mockEvent);
+    scheduler.onSourceTaskCompleted(mockSrcAttemptId31);
     Assert.assertEquals(4, scheduler.pendingTasks.size()); // no tasks scheduled
     Assert.assertEquals(4, scheduler.numSourceTasks);
     Assert.assertEquals(0, scheduler.numSourceTasksCompleted);
-    scheduler.onSourceTaskCompleted(mockSrcAttemptId11, mockEvent);
+    scheduler.onSourceTaskCompleted(mockSrcAttemptId11);
     Assert.assertEquals(4, scheduler.pendingTasks.size());
     Assert.assertEquals(0, scheduledTasks.size()); // no tasks scheduled
     Assert.assertEquals(1, scheduler.numSourceTasksCompleted);
     Assert.assertEquals(500L, scheduler.completedSourceTasksOutputSize);
     // ignore duplicate completion
-    scheduler.onSourceTaskCompleted(mockSrcAttemptId11, mockEvent);
+    scheduler.onSourceTaskCompleted(mockSrcAttemptId11);
     Assert.assertEquals(4, scheduler.pendingTasks.size());
     Assert.assertEquals(0, scheduledTasks.size()); // no tasks scheduled
     Assert.assertEquals(1, scheduler.numSourceTasksCompleted);
     Assert.assertEquals(500L, scheduler.completedSourceTasksOutputSize);
     
-    scheduler.onSourceTaskCompleted(mockSrcAttemptId12, mockEvent);
+    scheduler.onSourceTaskCompleted(mockSrcAttemptId12);
     // managedVertex tasks reduced
     verify(mockManagedVertex).setParallelism(eq(2), anyMap());
     Assert.assertEquals(2, newEdgeManagers.size());
@@ -220,7 +218,7 @@ public class TestVertexScheduler {
     Assert.assertEquals(1000L, scheduler.completedSourceTasksOutputSize);
     
     // more completions dont cause recalculation of parallelism
-    scheduler.onSourceTaskCompleted(mockSrcAttemptId21, mockEvent);
+    scheduler.onSourceTaskCompleted(mockSrcAttemptId21);
     verify(mockManagedVertex).setParallelism(eq(2), anyMap());
   }
   
@@ -266,9 +264,6 @@ public class TestVertexScheduler {
     when(mockManagedVertex.getVertexId()).thenReturn(mockManagedVertexId);
     when(mockManagedVertex.getInputVertices()).thenReturn(mockInputVertices);
     
-    TezDependentTaskCompletionEvent mockEvent = 
-        mock(TezDependentTaskCompletionEvent.class);
-
     // fail if there is no bipartite src vertex
     mockInputVertices.put(mockSrcVertex3, new Edge(eProp3, mockEventHandler));
     try {
@@ -362,11 +357,11 @@ public class TestVertexScheduler {
     Assert.assertTrue(scheduler.pendingTasks.size() == 3); // no tasks scheduled
     Assert.assertTrue(scheduler.numSourceTasks == 4);
     // task completion from non-bipartite stage does nothing
-    scheduler.onSourceTaskCompleted(mockSrcAttemptId31, mockEvent);
+    scheduler.onSourceTaskCompleted(mockSrcAttemptId31);
     Assert.assertTrue(scheduler.pendingTasks.size() == 3); // no tasks scheduled
     Assert.assertTrue(scheduler.numSourceTasks == 4);
     Assert.assertTrue(scheduler.numSourceTasksCompleted == 0);
-    scheduler.onSourceTaskCompleted(mockSrcAttemptId11, mockEvent);
+    scheduler.onSourceTaskCompleted(mockSrcAttemptId11);
     Assert.assertTrue(scheduler.pendingTasks.isEmpty());
     Assert.assertTrue(scheduledTasks.size() == 3); // all tasks scheduled
     Assert.assertTrue(scheduler.numSourceTasksCompleted == 1);
@@ -377,20 +372,20 @@ public class TestVertexScheduler {
     Assert.assertTrue(scheduler.pendingTasks.size() == 3); // no tasks scheduled
     Assert.assertTrue(scheduler.numSourceTasks == 4);
     // task completion from non-bipartite stage does nothing
-    scheduler.onSourceTaskCompleted(mockSrcAttemptId31, mockEvent);
+    scheduler.onSourceTaskCompleted(mockSrcAttemptId31);
     Assert.assertTrue(scheduler.pendingTasks.size() == 3); // no tasks scheduled
     Assert.assertTrue(scheduler.numSourceTasks == 4);
     Assert.assertTrue(scheduler.numSourceTasksCompleted == 0);
-    scheduler.onSourceTaskCompleted(mockSrcAttemptId11, mockEvent);
+    scheduler.onSourceTaskCompleted(mockSrcAttemptId11);
     Assert.assertTrue(scheduler.pendingTasks.size() == 3);
     Assert.assertTrue(scheduler.numSourceTasksCompleted == 1);
-    scheduler.onSourceTaskCompleted(mockSrcAttemptId12, mockEvent);
+    scheduler.onSourceTaskCompleted(mockSrcAttemptId12);
     Assert.assertTrue(scheduler.pendingTasks.size() == 3);
     Assert.assertTrue(scheduler.numSourceTasksCompleted == 2);
-    scheduler.onSourceTaskCompleted(mockSrcAttemptId21, mockEvent);
+    scheduler.onSourceTaskCompleted(mockSrcAttemptId21);
     Assert.assertTrue(scheduler.pendingTasks.size() == 3);
     Assert.assertTrue(scheduler.numSourceTasksCompleted == 3);
-    scheduler.onSourceTaskCompleted(mockSrcAttemptId22, mockEvent);
+    scheduler.onSourceTaskCompleted(mockSrcAttemptId22);
     Assert.assertTrue(scheduler.pendingTasks.isEmpty());
     Assert.assertTrue(scheduledTasks.size() == 3); // all tasks scheduled
     Assert.assertTrue(scheduler.numSourceTasksCompleted == 4);
@@ -401,20 +396,20 @@ public class TestVertexScheduler {
     Assert.assertTrue(scheduler.pendingTasks.size() == 3); // no tasks scheduled
     Assert.assertTrue(scheduler.numSourceTasks == 4);
     // task completion from non-bipartite stage does nothing
-    scheduler.onSourceTaskCompleted(mockSrcAttemptId31, mockEvent);
+    scheduler.onSourceTaskCompleted(mockSrcAttemptId31);
     Assert.assertTrue(scheduler.pendingTasks.size() == 3); // no tasks scheduled
     Assert.assertTrue(scheduler.numSourceTasks == 4);
     Assert.assertTrue(scheduler.numSourceTasksCompleted == 0);
-    scheduler.onSourceTaskCompleted(mockSrcAttemptId11, mockEvent);
+    scheduler.onSourceTaskCompleted(mockSrcAttemptId11);
     Assert.assertTrue(scheduler.pendingTasks.size() == 3);
     Assert.assertTrue(scheduler.numSourceTasksCompleted == 1);
-    scheduler.onSourceTaskCompleted(mockSrcAttemptId12, mockEvent);
+    scheduler.onSourceTaskCompleted(mockSrcAttemptId12);
     Assert.assertTrue(scheduler.pendingTasks.size() == 3);
     Assert.assertTrue(scheduler.numSourceTasksCompleted == 2);
-    scheduler.onSourceTaskCompleted(mockSrcAttemptId21, mockEvent);
+    scheduler.onSourceTaskCompleted(mockSrcAttemptId21);
     Assert.assertTrue(scheduler.pendingTasks.size() == 3);
     Assert.assertTrue(scheduler.numSourceTasksCompleted == 3);
-    scheduler.onSourceTaskCompleted(mockSrcAttemptId22, mockEvent);
+    scheduler.onSourceTaskCompleted(mockSrcAttemptId22);
     Assert.assertTrue(scheduler.pendingTasks.isEmpty());
     Assert.assertTrue(scheduledTasks.size() == 3); // all tasks scheduled
     Assert.assertTrue(scheduler.numSourceTasksCompleted == 4);
@@ -424,22 +419,22 @@ public class TestVertexScheduler {
     scheduler.onVertexStarted();
     Assert.assertTrue(scheduler.pendingTasks.size() == 3); // no tasks scheduled
     Assert.assertTrue(scheduler.numSourceTasks == 4);
-    scheduler.onSourceTaskCompleted(mockSrcAttemptId11, mockEvent);
-    scheduler.onSourceTaskCompleted(mockSrcAttemptId12, mockEvent);
+    scheduler.onSourceTaskCompleted(mockSrcAttemptId11);
+    scheduler.onSourceTaskCompleted(mockSrcAttemptId12);
     Assert.assertTrue(scheduler.pendingTasks.size() == 2);
     Assert.assertTrue(scheduledTasks.size() == 1); // 1 task scheduled
     Assert.assertTrue(scheduler.numSourceTasksCompleted == 2);
     // completion of same task again should not get counted
-    scheduler.onSourceTaskCompleted(mockSrcAttemptId12, mockEvent);
+    scheduler.onSourceTaskCompleted(mockSrcAttemptId12);
     Assert.assertTrue(scheduler.pendingTasks.size() == 2);
     Assert.assertTrue(scheduledTasks.size() == 1); // 1 task scheduled
     Assert.assertTrue(scheduler.numSourceTasksCompleted == 2);
-    scheduler.onSourceTaskCompleted(mockSrcAttemptId21, mockEvent);
+    scheduler.onSourceTaskCompleted(mockSrcAttemptId21);
     Assert.assertTrue(scheduler.pendingTasks.size() == 0);
     Assert.assertTrue(scheduledTasks.size() == 2); // 2 tasks scheduled
     Assert.assertTrue(scheduler.numSourceTasksCompleted == 3);
     scheduledTasks.clear();
-    scheduler.onSourceTaskCompleted(mockSrcAttemptId22, mockEvent); // we are done. no action
+    scheduler.onSourceTaskCompleted(mockSrcAttemptId22); // we are done. no action
     Assert.assertTrue(scheduler.pendingTasks.size() == 0);
     Assert.assertTrue(scheduledTasks.size() == 0); // no task scheduled
     Assert.assertTrue(scheduler.numSourceTasksCompleted == 4);
@@ -449,16 +444,16 @@ public class TestVertexScheduler {
     scheduler.onVertexStarted();
     Assert.assertTrue(scheduler.pendingTasks.size() == 3); // no tasks scheduled
     Assert.assertTrue(scheduler.numSourceTasks == 4);
-    scheduler.onSourceTaskCompleted(mockSrcAttemptId11, mockEvent);
-    scheduler.onSourceTaskCompleted(mockSrcAttemptId12, mockEvent);
+    scheduler.onSourceTaskCompleted(mockSrcAttemptId11);
+    scheduler.onSourceTaskCompleted(mockSrcAttemptId12);
     Assert.assertTrue(scheduler.pendingTasks.size() == 2);
     Assert.assertTrue(scheduledTasks.size() == 1); // 1 task scheduled
     Assert.assertTrue(scheduler.numSourceTasksCompleted == 2);
-    scheduler.onSourceTaskCompleted(mockSrcAttemptId21, mockEvent);
+    scheduler.onSourceTaskCompleted(mockSrcAttemptId21);
     Assert.assertTrue(scheduler.pendingTasks.size() == 1);
     Assert.assertTrue(scheduledTasks.size() == 1); // 1 task scheduled
     Assert.assertTrue(scheduler.numSourceTasksCompleted == 3);
-    scheduler.onSourceTaskCompleted(mockSrcAttemptId22, mockEvent);
+    scheduler.onSourceTaskCompleted(mockSrcAttemptId22);
     Assert.assertTrue(scheduler.pendingTasks.size() == 0);
     Assert.assertTrue(scheduledTasks.size() == 1); // no task scheduled
     Assert.assertTrue(scheduler.numSourceTasksCompleted == 4);


[03/50] [abbrv] git commit: TEZ-433. Change Combiner to work with new APIs (part of TEZ-398). (sseth)

Posted by ss...@apache.org.
TEZ-433. Change Combiner to work with new APIs (part of TEZ-398).
(sseth)


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

Branch: refs/heads/master
Commit: eb0f6ffe5250b497f35909f8559682cbaa3621ae
Parents: 85a9d46
Author: Siddharth Seth <ss...@apache.org>
Authored: Fri Sep 20 15:43:56 2013 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Fri Sep 20 15:43:56 2013 -0700

----------------------------------------------------------------------
 .../org/apache/tez/common/TezJobConfig.java     |   5 +
 .../apache/tez/engine/common/ConfigUtils.java   |   4 +
 .../tez/engine/common/TezEngineUtils.java       |  39 +++
 .../tez/engine/common/ValuesIterator.java       |   2 +
 .../tez/engine/common/combine/Combiner.java     |  43 ++++
 .../common/shuffle/impl/MergeManager.java       |  41 +---
 .../tez/engine/common/shuffle/impl/Shuffle.java |  10 +-
 .../engine/common/sort/impl/ExternalSorter.java |  45 +---
 .../common/sort/impl/PipelinedSorter.java       |  15 +-
 .../common/sort/impl/dflt/DefaultSorter.java    |   4 +-
 .../engine/common/task/impl/ValuesIterator.java |   9 +-
 .../engine/lib/input/ShuffledMergedInput.java   |   1 -
 .../apache/tez/mapreduce/hadoop/MRHelpers.java  |  13 +
 .../hadoop/MultiStageMRConfToTezTranslator.java |  19 +-
 .../tez/mapreduce/newcombine/MRCombiner.java    | 242 +++++++++++++++++++
 .../mapreduce/newpartition/MRPartitioner.java   |   3 +-
 .../mapreduce/newprocessor/MRTaskReporter.java  |   7 +
 17 files changed, 411 insertions(+), 91 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/eb0f6ffe/tez-common/src/main/java/org/apache/tez/common/TezJobConfig.java
----------------------------------------------------------------------
diff --git a/tez-common/src/main/java/org/apache/tez/common/TezJobConfig.java b/tez-common/src/main/java/org/apache/tez/common/TezJobConfig.java
index 12c2b4b..7c4540c 100644
--- a/tez-common/src/main/java/org/apache/tez/common/TezJobConfig.java
+++ b/tez-common/src/main/java/org/apache/tez/common/TezJobConfig.java
@@ -128,6 +128,11 @@ public class TezJobConfig {
    */
   public static final String TEZ_ENGINE_PARTITIONER_CLASS = "tez.engine.partitioner.class";
   
+  /**
+   * Specifies a combiner class (primarily for Shuffle)
+   */
+  public static final String TEZ_ENGINE_COMBINER_CLASS = "tez.engine.combiner.class";
+  
   public static final String TEZ_ENGINE_NUM_EXPECTED_PARTITIONS = "tez.engine.num.expected.partitions";
   
   /**

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/eb0f6ffe/tez-engine/src/main/java/org/apache/tez/engine/common/ConfigUtils.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/ConfigUtils.java b/tez-engine/src/main/java/org/apache/tez/engine/common/ConfigUtils.java
index a92cf1b..f73adfd 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/ConfigUtils.java
+++ b/tez-engine/src/main/java/org/apache/tez/engine/common/ConfigUtils.java
@@ -140,5 +140,9 @@ public class ConfigUtils {
 
     return ReflectionUtils.newInstance(theClass, conf);
   }
+  
+  public static boolean useNewApi(Configuration conf) {
+    return conf.getBoolean("mapred.mapper.new-api", false);
+  }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/eb0f6ffe/tez-engine/src/main/java/org/apache/tez/engine/common/TezEngineUtils.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/TezEngineUtils.java b/tez-engine/src/main/java/org/apache/tez/engine/common/TezEngineUtils.java
index f352e08..3920ce6 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/TezEngineUtils.java
+++ b/tez-engine/src/main/java/org/apache/tez/engine/common/TezEngineUtils.java
@@ -29,9 +29,11 @@ import org.apache.tez.common.Constants;
 import org.apache.tez.common.TezJobConfig;
 import org.apache.tez.dag.api.TezUncheckedException;
 import org.apache.tez.engine.api.Partitioner;
+import org.apache.tez.engine.common.combine.Combiner;
 import org.apache.tez.engine.common.task.local.newoutput.TezTaskOutput;
 import org.apache.tez.engine.common.task.local.newoutput.TezTaskOutputFiles;
 import org.apache.tez.engine.newapi.TezOutputContext;
+import org.apache.tez.engine.newapi.TezTaskContext;
 
 public class TezEngineUtils {
 
@@ -55,6 +57,43 @@ public class TezEngineUtils {
   }
 
   @SuppressWarnings("unchecked")
+  public static Combiner instantiateCombiner(Configuration conf, TezTaskContext taskContext) throws IOException {
+    Class<? extends Combiner> clazz;
+    String className = conf.get(TezJobConfig.TEZ_ENGINE_COMBINER_CLASS);
+    if (className == null) {
+      LOG.info("No combiner specified via " + TezJobConfig.TEZ_ENGINE_COMBINER_CLASS + ". Combiner will not be used");
+      return null;
+    }
+    LOG.info("Using Combiner class: " + className);
+    try {
+      clazz = (Class<? extends Combiner>) conf.getClassByName(className);
+    } catch (ClassNotFoundException e) {
+      throw new IOException("Unable to load combiner class: " + className);
+    }
+    
+    Combiner combiner = null;
+    
+      Constructor<? extends Combiner> ctor;
+      try {
+        ctor = clazz.getConstructor(TezTaskContext.class);
+        combiner = ctor.newInstance(taskContext);
+      } catch (SecurityException e) {
+        throw new IOException(e);
+      } catch (NoSuchMethodException e) {
+        throw new IOException(e);
+      } catch (IllegalArgumentException e) {
+        throw new IOException(e);
+      } catch (InstantiationException e) {
+        throw new IOException(e);
+      } catch (IllegalAccessException e) {
+        throw new IOException(e);
+      } catch (InvocationTargetException e) {
+        throw new IOException(e);
+      }
+      return combiner;
+  }
+  
+  @SuppressWarnings("unchecked")
   public static Partitioner instantiatePartitioner(Configuration conf)
       throws IOException {
     Class<? extends Partitioner> clazz;

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/eb0f6ffe/tez-engine/src/main/java/org/apache/tez/engine/common/ValuesIterator.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/ValuesIterator.java b/tez-engine/src/main/java/org/apache/tez/engine/common/ValuesIterator.java
index a33d00b..b7867aa 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/ValuesIterator.java
+++ b/tez-engine/src/main/java/org/apache/tez/engine/common/ValuesIterator.java
@@ -101,6 +101,8 @@ public class ValuesIterator<KEY,VALUE> {
     return key; 
   }
   
+  // TODO NEWTEZ Maybe add another method which returns an iterator instead of iterable
+  
   public Iterable<VALUE> getValues() {
     return new Iterable<VALUE>() {
 

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/eb0f6ffe/tez-engine/src/main/java/org/apache/tez/engine/common/combine/Combiner.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/combine/Combiner.java b/tez-engine/src/main/java/org/apache/tez/engine/common/combine/Combiner.java
new file mode 100644
index 0000000..6f86d61
--- /dev/null
+++ b/tez-engine/src/main/java/org/apache/tez/engine/common/combine/Combiner.java
@@ -0,0 +1,43 @@
+/**
+ * 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.engine.common.combine;
+
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience.LimitedPrivate;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.tez.common.TezJobConfig;
+import org.apache.tez.common.TezTaskContext;
+import org.apache.tez.engine.common.sort.impl.IFile.Writer;
+import org.apache.tez.engine.common.sort.impl.TezRawKeyValueIterator;
+
+/**
+ *<b>Combiner Initialization</b></p> The Combiner class is picked up
+ * using the TEZ_ENGINE_COMBINER_CLASS attribute in {@link TezJobConfig}
+ * 
+ * 
+ * Partitioners need to provide a single argument ({@link TezTaskContext})
+ * constructor.
+ */
+@Unstable
+@LimitedPrivate("mapreduce")
+public interface Combiner {
+  public void combine(TezRawKeyValueIterator rawIter, Writer writer)
+      throws InterruptedException, IOException;
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/eb0f6ffe/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/MergeManager.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/MergeManager.java b/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/MergeManager.java
index 093a293..ad9bb5f 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/MergeManager.java
+++ b/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/MergeManager.java
@@ -48,6 +48,7 @@ import org.apache.tez.common.TezJobConfig;
 import org.apache.tez.common.counters.TezCounter;
 import org.apache.tez.engine.common.ConfigUtils;
 import org.apache.tez.engine.common.InputAttemptIdentifier;
+import org.apache.tez.engine.common.combine.Combiner;
 import org.apache.tez.engine.common.sort.impl.IFile;
 import org.apache.tez.engine.common.sort.impl.TezMerger;
 import org.apache.tez.engine.common.sort.impl.TezRawKeyValueIterator;
@@ -55,7 +56,6 @@ import org.apache.tez.engine.common.sort.impl.IFile.Writer;
 import org.apache.tez.engine.common.sort.impl.TezMerger.Segment;
 import org.apache.tez.engine.common.task.local.newoutput.TezTaskOutputFiles;
 import org.apache.tez.engine.hadoop.compat.NullProgressable;
-import org.apache.tez.engine.newapi.Processor;
 import org.apache.tez.engine.newapi.TezInputContext;
 
 @InterfaceAudience.Private
@@ -72,7 +72,7 @@ public class MergeManager {
   
   private final  TezTaskOutputFiles mapOutputFile;
   private final Progressable nullProgressable = new NullProgressable();
-  private final Processor combineProcessor = null; // TODO NEWTEZ Fix CombineProcessor  
+  private final Combiner combiner;  
   
   Set<MapOutput> inMemoryMergedMapOutputs = 
     new TreeSet<MapOutput>(new MapOutput.MapOutputComparator());
@@ -98,12 +98,6 @@ public class MergeManager {
   private final ExceptionReporter exceptionReporter;
   
   private final TezInputContext inputContext;
-  
-  /**
-   * Combiner processor to run during in-memory merge, if defined.
-   */
-  // TODO NEWTEZ Fix Combiner
-  //private final Processor combineProcessor;
 
   private final TezCounter spilledRecordsCounter;
 
@@ -119,18 +113,18 @@ public class MergeManager {
                       FileSystem localFS,
                       LocalDirAllocator localDirAllocator,  
                       TezInputContext inputContext,
-                      Processor combineProcessor,
+                      Combiner combiner,
                       TezCounter spilledRecordsCounter,
                       TezCounter reduceCombineInputCounter,
                       TezCounter mergedMapOutputsCounter,
                       ExceptionReporter exceptionReporter) {
-    // TODO NEWTEZ Change to include Combiner
     this.inputContext = inputContext;
     this.conf = conf;
     this.localDirAllocator = localDirAllocator;
     this.exceptionReporter = exceptionReporter;
     
-    //this.combineProcessor = combineProcessor;
+    this.combiner = combiner;
+
     this.reduceCombineInputCounter = reduceCombineInputCounter;
     this.spilledRecordsCounter = spilledRecordsCounter;
     this.mergedMapOutputsCounter = mergedMapOutputsCounter;
@@ -370,27 +364,8 @@ public class MergeManager {
   }
    
   void runCombineProcessor(TezRawKeyValueIterator kvIter, Writer writer)
-  throws IOException, InterruptedException {
-
-    // TODO NEWTEZ Fix CombineProcessor
-    
-//    CombineInput combineIn = new CombineInput(kvIter);
-//    combineIn.initialize(conf, reporter);
-//    
-//    CombineOutput combineOut = new CombineOutput(writer);
-//    combineOut.initialize(conf, reporter);
-//
-//    try {
-//      combineProcessor.process(new Input[] {combineIn},
-//          new Output[] {combineOut});
-//    } catch (IOException ioe) {
-//      try {
-//        combineProcessor.close();
-//      } catch (IOException ignoredException) {}
-//
-//      throw ioe;
-//    }
-  
+      throws IOException, InterruptedException {
+    combiner.combine(kvIter, writer);
   }
 
   private class IntermediateMemoryToMemoryMerger 
@@ -500,7 +475,7 @@ public class MergeManager {
             (RawComparator)ConfigUtils.getIntermediateInputKeyComparator(conf),
             nullProgressable, spilledRecordsCounter, null, null);
 
-        if (null == combineProcessor) {
+        if (null == combiner) {
           TezMerger.writeFile(rIter, writer, nullProgressable, conf);
         } else {
           runCombineProcessor(rIter, writer);

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/eb0f6ffe/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/Shuffle.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/Shuffle.java b/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/Shuffle.java
index 620c620..f605b7c 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/Shuffle.java
+++ b/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/Shuffle.java
@@ -38,6 +38,8 @@ import org.apache.tez.common.TezJobConfig;
 import org.apache.tez.common.counters.TaskCounter;
 import org.apache.tez.common.counters.TezCounter;
 import org.apache.tez.dag.api.TezUncheckedException;
+import org.apache.tez.engine.common.TezEngineUtils;
+import org.apache.tez.engine.common.combine.Combiner;
 import org.apache.tez.engine.common.shuffle.server.ShuffleHandler;
 import org.apache.tez.engine.common.sort.impl.TezRawKeyValueIterator;
 import org.apache.tez.engine.newapi.Event;
@@ -67,11 +69,10 @@ public class Shuffle implements ExceptionReporter {
   private final SecretKey jobTokenSecret;
   private AtomicInteger reduceRange = new AtomicInteger(
       TezJobConfig.TEZ_ENGINE_SHUFFLE_PARTITION_RANGE_DEFAULT);
-  
+
   private FutureTask<TezRawKeyValueIterator> runShuffleFuture;
 
   public Shuffle(TezInputContext inputContext, Configuration conf, int numInputs) throws IOException {
-    // TODO NEWTEZ Handle Combiner
     this.inputContext = inputContext;
     this.conf = conf;
     this.metrics = new ShuffleClientMetrics(inputContext.getDAGName(),
@@ -84,6 +85,8 @@ public class Shuffle implements ExceptionReporter {
         .getJobTokenSecretFromTokenBytes(inputContext
             .getServiceConsumerMetaData(ShuffleHandler.MAPREDUCE_SHUFFLE_SERVICEID));
     
+    Combiner combiner = TezEngineUtils.instantiateCombiner(conf, inputContext);
+    
     FileSystem localFS = FileSystem.getLocal(this.conf);
     LocalDirAllocator localDirAllocator = 
         new LocalDirAllocator(TezJobConfig.LOCAL_DIRS);
@@ -123,7 +126,7 @@ public class Shuffle implements ExceptionReporter {
           localFS,
           localDirAllocator,
           inputContext,
-          null, // TODO NEWTEZ Fix Combiner
+          combiner,
           spilledRecordsCounter,
           reduceCombineInputCounter,
           mergedMapOutputsCounter,
@@ -272,5 +275,4 @@ public class Shuffle implements ExceptionReporter {
       throw e; 
     }
   }
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/eb0f6ffe/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/ExternalSorter.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/ExternalSorter.java b/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/ExternalSorter.java
index 8b20192..1b5e015 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/ExternalSorter.java
+++ b/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/ExternalSorter.java
@@ -43,15 +43,14 @@ import org.apache.tez.common.TezJobConfig;
 import org.apache.tez.common.counters.TaskCounter;
 import org.apache.tez.common.counters.TezCounter;
 import org.apache.tez.engine.api.Partitioner;
-import org.apache.tez.engine.api.Processor;
 import org.apache.tez.engine.common.ConfigUtils;
 import org.apache.tez.engine.common.TezEngineUtils;
+import org.apache.tez.engine.common.combine.Combiner;
 import org.apache.tez.engine.common.shuffle.impl.ShuffleHeader;
 import org.apache.tez.engine.common.sort.impl.IFile.Writer;
 import org.apache.tez.engine.common.task.local.newoutput.TezTaskOutput;
 import org.apache.tez.engine.hadoop.compat.NullProgressable;
 import org.apache.tez.engine.newapi.TezOutputContext;
-import org.apache.tez.engine.records.OutputContext;
 
 @SuppressWarnings({"unchecked", "rawtypes"})
 public abstract class ExternalSorter {
@@ -66,7 +65,7 @@ public abstract class ExternalSorter {
 
   protected Progressable nullProgressable = new NullProgressable();
   protected TezOutputContext outputContext;
-  protected Processor combineProcessor;
+  protected Combiner combiner;
   protected Partitioner partitioner;
   protected Configuration conf;
   protected FileSystem rfs;
@@ -84,9 +83,6 @@ public abstract class ExternalSorter {
   // Compression for map-outputs
   protected CompressionCodec codec;
 
-  // TODO NEWTEZ Setup CombineProcessor
-  // TODO NEWTEZ Setup Partitioner in SimpleOutput
-
   // Counters
   // TODO TEZ Rename all counter variables [Mapping of counter to MR for compatibility in the MR layer]
   protected TezCounter mapOutputByteCounter;
@@ -139,12 +135,7 @@ public abstract class ExternalSorter {
     LOG.info("Instantiating Partitioner: [" + conf.get(TezJobConfig.TEZ_ENGINE_PARTITIONER_CLASS) + "]");
     this.conf.setInt(TezJobConfig.TEZ_ENGINE_NUM_EXPECTED_PARTITIONS, this.partitions);
     this.partitioner = TezEngineUtils.instantiatePartitioner(this.conf);
-  }
-
-  // TODO NEWTEZ Add an interface (! Processor) for CombineProcessor, which MR tasks can initialize and set.
-  // Alternately add a config key with a classname, which is easy to initialize.
-  public void setCombiner(Processor combineProcessor) {
-    this.combineProcessor = combineProcessor;
+    this.combiner = TezEngineUtils.instantiateCombiner(this.conf, outputContext);
   }
 
   /**
@@ -165,27 +156,11 @@ public abstract class ExternalSorter {
 
   protected void runCombineProcessor(TezRawKeyValueIterator kvIter,
       Writer writer) throws IOException {
-
-    // TODO NEWTEZ Fix Combiner.
-//    CombineInput combineIn = new CombineInput(kvIter);
-//    combineIn.initialize(job, runningTaskContext.getTaskReporter());
-//
-//    CombineOutput combineOut = new CombineOutput(writer);
-//    combineOut.initialize(job, runningTaskContext.getTaskReporter());
-//
-//    try {
-//      combineProcessor.process(new Input[] {combineIn},
-//          new Output[] {combineOut});
-//    } catch (IOException ioe) {
-//      try {
-//        combineProcessor.close();
-//      } catch (IOException ignored) {}
-//
-//      // Do not close output here as the sorter should close the combine output
-//
-//      throw ioe;
-//    }
-
+    try {
+      combiner.combine(kvIter, writer);
+    } catch (InterruptedException e) {
+      throw new IOException(e);
+    }
   }
 
   /**
@@ -216,8 +191,4 @@ public abstract class ExternalSorter {
   public ShuffleHeader getShuffleHeader(int reduce) {
     throw new UnsupportedOperationException("getShuffleHeader isn't supported!");
   }
-
-  public OutputContext getOutputContext() {
-    return null;
-  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/eb0f6ffe/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/PipelinedSorter.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/PipelinedSorter.java b/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/PipelinedSorter.java
index bafbd4d..952568e 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/PipelinedSorter.java
+++ b/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/PipelinedSorter.java
@@ -45,10 +45,9 @@ import org.apache.hadoop.util.IndexedSorter;
 import org.apache.hadoop.util.Progress;
 import org.apache.tez.common.TezJobConfig;
 import org.apache.tez.engine.common.ConfigUtils;
-import org.apache.tez.engine.newapi.TezOutputContext;
-import org.apache.tez.engine.records.OutputContext;
 import org.apache.tez.engine.common.sort.impl.IFile.Writer;
 import org.apache.tez.engine.common.sort.impl.TezMerger.Segment;
+import org.apache.tez.engine.newapi.TezOutputContext;
 
 @SuppressWarnings({"unchecked", "rawtypes"})
 public class PipelinedSorter extends ExternalSorter {
@@ -270,7 +269,7 @@ public class PipelinedSorter extends ExternalSorter {
           new Writer(conf, out, keyClass, valClass, codec,
               spilledRecordsCounter);
         writer.setRLE(merger.needsRLE());
-        if (combineProcessor == null) {
+        if (combiner == null) {
           while(kvIter.next()) {
             writer.append(kvIter.getKey(), kvIter.getValue());
           }
@@ -380,10 +379,10 @@ public class PipelinedSorter extends ExternalSorter {
           new Writer(conf, finalOut, keyClass, valClass, codec,
                            spilledRecordsCounter);
       writer.setRLE(merger.needsRLE());
-      if (combineProcessor == null || numSpills < minSpillsForCombine) {
+      if (combiner == null || numSpills < minSpillsForCombine) {
         TezMerger.writeFile(kvIter, writer, nullProgressable, conf);
       } else {
-    	runCombineProcessor(kvIter, writer);
+        runCombineProcessor(kvIter, writer);
       }
 
       //close
@@ -930,10 +929,4 @@ public class PipelinedSorter extends ExternalSorter {
     }
 
   }
-
-  @Override
-  public OutputContext getOutputContext() {
-    return null;
-  }
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/eb0f6ffe/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/dflt/DefaultSorter.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/dflt/DefaultSorter.java b/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/dflt/DefaultSorter.java
index b1e17e7..1ad31f7 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/dflt/DefaultSorter.java
+++ b/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/dflt/DefaultSorter.java
@@ -752,7 +752,7 @@ public class DefaultSorter extends ExternalSorter implements IndexedSortable {
           long segmentStart = out.getPos();
           writer = new Writer(conf, out, keyClass, valClass, codec,
                                     spilledRecordsCounter);
-          if (combineProcessor == null) {
+          if (combiner == null) {
             // spill directly
             DataInputBuffer key = new DataInputBuffer();
             while (spindex < mend &&
@@ -1082,7 +1082,7 @@ public class DefaultSorter extends ExternalSorter implements IndexedSortable {
         Writer writer =
             new Writer(conf, finalOut, keyClass, valClass, codec,
                 spilledRecordsCounter);
-        if (combineProcessor == null || numSpills < minSpillsForCombine) {
+        if (combiner == null || numSpills < minSpillsForCombine) {
           TezMerger.writeFile(kvIter, writer,
               nullProgressable, conf);
         } else {

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/eb0f6ffe/tez-engine/src/main/java/org/apache/tez/engine/common/task/impl/ValuesIterator.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/task/impl/ValuesIterator.java b/tez-engine/src/main/java/org/apache/tez/engine/common/task/impl/ValuesIterator.java
index 126c5b2..841e54d 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/task/impl/ValuesIterator.java
+++ b/tez-engine/src/main/java/org/apache/tez/engine/common/task/impl/ValuesIterator.java
@@ -30,7 +30,14 @@ import org.apache.hadoop.io.serializer.SerializationFactory;
 import org.apache.hadoop.util.Progressable;
 import org.apache.tez.engine.common.sort.impl.TezRawKeyValueIterator;
 
-/** Iterates values while keys match in sorted input. */
+
+/**
+ * Iterates values while keys match in sorted input.
+ *
+ * Usage: Call moveToNext to move to the next k, v pair. This returns true if another exists,
+ * followed by getKey() and getValues() to get the current key and list of values.
+ * 
+ */
 public class ValuesIterator<KEY,VALUE> implements Iterator<VALUE> {
   protected TezRawKeyValueIterator in; //input iterator
   private KEY key;               // current key

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/eb0f6ffe/tez-engine/src/main/java/org/apache/tez/engine/lib/input/ShuffledMergedInput.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/lib/input/ShuffledMergedInput.java b/tez-engine/src/main/java/org/apache/tez/engine/lib/input/ShuffledMergedInput.java
index 0732e20..91bb6d5 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/lib/input/ShuffledMergedInput.java
+++ b/tez-engine/src/main/java/org/apache/tez/engine/lib/input/ShuffledMergedInput.java
@@ -165,7 +165,6 @@ public class ShuffledMergedInput implements LogicalInput {
 
   }
 
-
   // This functionality is currently broken. If there's inputs which need to be
   // written to disk, there's a possibility that inputs from the different
   // sources could clobber each others' output. Also the current structures do

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/eb0f6ffe/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/MRHelpers.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/MRHelpers.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/MRHelpers.java
index 2f4a62a..7a9b7e0 100644
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/MRHelpers.java
+++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/MRHelpers.java
@@ -59,6 +59,7 @@ import org.apache.tez.common.TezJobConfig;
 import org.apache.tez.common.TezUtils;
 import org.apache.tez.dag.api.TezConfiguration;
 import org.apache.tez.dag.api.VertexLocationHint.TaskLocationHint;
+import org.apache.tez.mapreduce.newcombine.MRCombiner;
 import org.apache.tez.mapreduce.newpartition.MRPartitioner;
 
 
@@ -367,6 +368,18 @@ public class MRHelpers {
 
     // TODO eventually ACLs
     conf.set(TezJobConfig.TEZ_ENGINE_PARTITIONER_CLASS, MRPartitioner.class.getName());
+    
+    boolean useNewApi = conf.getBoolean("mapred.mapper.new-api", false);
+    if (useNewApi) {
+      if (conf.get(MRJobConfig.COMBINE_CLASS_ATTR) != null) {
+        conf.set(TezJobConfig.TEZ_ENGINE_COMBINER_CLASS, MRCombiner.class.getName());
+      }
+    } else {
+      if (conf.get("mapred.combiner.class") != null) {
+        conf.set(TezJobConfig.TEZ_ENGINE_COMBINER_CLASS, MRCombiner.class.getName());
+      }
+    }
+    
     setWorkingDirectory(conf);
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/eb0f6ffe/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/MultiStageMRConfToTezTranslator.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/MultiStageMRConfToTezTranslator.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/MultiStageMRConfToTezTranslator.java
index 6b68e95..ad231b3 100644
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/MultiStageMRConfToTezTranslator.java
+++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/MultiStageMRConfToTezTranslator.java
@@ -18,7 +18,6 @@
 
 package org.apache.tez.mapreduce.hadoop;
 
-import java.util.Iterator;
 import java.util.Map;
 import java.util.Map.Entry;
 
@@ -31,6 +30,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.tez.common.TezJobConfig;
 import org.apache.tez.mapreduce.hadoop.DeprecatedKeys.MultiStageKeys;
+import org.apache.tez.mapreduce.newcombine.MRCombiner;
 import org.apache.tez.mapreduce.newpartition.MRPartitioner;
 
 import com.google.common.base.Preconditions;
@@ -220,22 +220,39 @@ public class MultiStageMRConfToTezTranslator {
     // Assuming no 0 map jobs, and the first stage is always a map.
     int numStages = numIntermediateStages + (hasFinalReduceStage ? 2 : 1);
 
+    // Setup Tez partitioner class
     conf.set(TezJobConfig.TEZ_ENGINE_PARTITIONER_CLASS,
         MRPartitioner.class.getName());
+    
+    // Setup Tez Combiner class if required.
+    // This would already have been set since the call is via JobClient
+    boolean useNewApi = conf.getBoolean("mapred.mapper.new-api", false);
+    if (useNewApi) {
+      if (conf.get(MRJobConfig.COMBINE_CLASS_ATTR) != null) {
+        conf.set(TezJobConfig.TEZ_ENGINE_COMBINER_CLASS, MRCombiner.class.getName());
+      }
+    } else {
+      if (conf.get("mapred.combiner.class") != null) {
+        conf.set(TezJobConfig.TEZ_ENGINE_COMBINER_CLASS, MRCombiner.class.getName());
+      }
+    }
 
     Configuration confs[] = new Configuration[numStages];
     Configuration nonItermediateConf = MultiStageMRConfigUtil.extractStageConf(
         conf, "");
+    confs[0].setBoolean(MRConfig.IS_MAP_PROCESSOR, true);
     if (numStages == 1) {
       confs[0] = nonItermediateConf;
     } else {
       confs[0] = nonItermediateConf;
       confs[numStages - 1] = new Configuration(nonItermediateConf);
+      confs[numStages -1].setBoolean(MRConfig.IS_MAP_PROCESSOR, false);
     }
     if (numStages > 2) {
       for (int i = 1; i < numStages - 1; i++) {
         confs[i] = MultiStageMRConfigUtil.extractStageConf(conf,
             MultiStageMRConfigUtil.getPropertyNameForIntermediateStage(i, ""));
+        confs[i].setBoolean(MRConfig.IS_MAP_PROCESSOR, false);
       }
     }
     return confs;

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/eb0f6ffe/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/newcombine/MRCombiner.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/newcombine/MRCombiner.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/newcombine/MRCombiner.java
new file mode 100644
index 0000000..788019a
--- /dev/null
+++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/newcombine/MRCombiner.java
@@ -0,0 +1,242 @@
+/**
+ * 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.newcombine;
+
+import java.io.IOException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.DataInputBuffer;
+import org.apache.hadoop.io.RawComparator;
+import org.apache.hadoop.mapred.OutputCollector;
+import org.apache.hadoop.mapred.RawKeyValueIterator;
+import org.apache.hadoop.mapred.Reducer;
+import org.apache.hadoop.mapreduce.Counter;
+import org.apache.hadoop.mapreduce.RecordWriter;
+import org.apache.hadoop.mapreduce.ReduceContext;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.TaskAttemptID;
+import org.apache.hadoop.mapreduce.TaskCounter;
+import org.apache.hadoop.mapreduce.TaskID;
+import org.apache.hadoop.mapreduce.TaskType;
+import org.apache.hadoop.mapreduce.lib.reduce.WrappedReducer;
+import org.apache.hadoop.mapreduce.task.ReduceContextImpl;
+import org.apache.hadoop.util.Progress;
+import org.apache.hadoop.util.ReflectionUtils;
+import org.apache.tez.common.TezUtils;
+import org.apache.tez.common.counters.TezCounter;
+import org.apache.tez.engine.common.ConfigUtils;
+import org.apache.tez.engine.common.ValuesIterator;
+import org.apache.tez.engine.common.combine.Combiner;
+import org.apache.tez.engine.common.sort.impl.IFile.Writer;
+import org.apache.tez.engine.common.sort.impl.TezRawKeyValueIterator;
+import org.apache.tez.engine.newapi.TezInputContext;
+import org.apache.tez.engine.newapi.TezOutputContext;
+import org.apache.tez.engine.newapi.impl.TezTaskContextImpl;
+import org.apache.tez.mapreduce.hadoop.MRConfig;
+import org.apache.tez.mapreduce.hadoop.MRJobConfig;
+import org.apache.tez.mapreduce.hadoop.mapred.MRCounters;
+import org.apache.tez.mapreduce.newprocessor.MRTaskReporter;
+
+@SuppressWarnings({"rawtypes", "unchecked"})
+public class MRCombiner implements Combiner {
+
+  private static Log LOG = LogFactory.getLog(MRCombiner.class);
+  
+  private final Configuration conf;
+  private final Class<?> keyClass;
+  private final Class<?> valClass;
+  private final RawComparator<?> comparator;
+  private final boolean useNewApi;
+  
+  private final TezCounter combineInputKeyCounter;
+  private final TezCounter combineInputValueCounter;
+  
+  private final MRTaskReporter reporter;
+  private final TaskAttemptID mrTaskAttemptID;
+
+  public MRCombiner(TezTaskContextImpl taskContext) throws IOException {
+    this.conf = TezUtils.createConfFromUserPayload(taskContext.getUserPayload());
+
+    assert(taskContext instanceof TezInputContext || taskContext instanceof TezOutputContext);
+    if (taskContext instanceof TezOutputContext) {
+      this.keyClass = ConfigUtils.getIntermediateOutputKeyClass(conf);
+      this.valClass = ConfigUtils.getIntermediateOutputKeyClass(conf);
+      this.comparator = ConfigUtils.getIntermediateOutputKeyComparator(conf);
+      this.reporter = new MRTaskReporter((TezOutputContext)taskContext);
+    } else {
+      this.keyClass = ConfigUtils.getIntermediateInputKeyClass(conf);
+      this.valClass = ConfigUtils.getIntermediateInputValueClass(conf);
+      this.comparator = ConfigUtils.getIntermediateInputKeyComparator(conf);
+      this.reporter = new MRTaskReporter((TezInputContext)taskContext);
+    }
+
+    this.useNewApi = ConfigUtils.useNewApi(conf);
+    
+    combineInputKeyCounter = taskContext.getCounters().findCounter(TaskCounter.COMBINE_INPUT_RECORDS);
+    combineInputValueCounter = taskContext.getCounters().findCounter(TaskCounter.COMBINE_OUTPUT_RECORDS);
+    
+    boolean isMap = conf.getBoolean(MRConfig.IS_MAP_PROCESSOR,false);
+    this.mrTaskAttemptID = new TaskAttemptID(
+        new TaskID(String.valueOf(taskContext.getApplicationId()
+            .getClusterTimestamp()), taskContext.getApplicationId().getId(),
+            isMap ? TaskType.MAP : TaskType.REDUCE,
+            taskContext.getTaskIndex()), taskContext.getTaskAttemptNumber());
+    
+    LOG.info("Using combineKeyClass: " + keyClass + ", combineValueClass: " + valClass + ", combineComparator: " +comparator + ", useNewApi: " + useNewApi);
+  }
+
+  @Override
+  public void combine(TezRawKeyValueIterator rawIter, Writer writer)
+      throws InterruptedException, IOException {
+    if (useNewApi) {
+      runNewCombiner(rawIter, writer);
+    } else {
+      runOldCombiner(rawIter, writer);
+    }
+    
+  }
+
+  ///////////////// Methods for old API //////////////////////
+  
+  private void runOldCombiner(final TezRawKeyValueIterator rawIter, final Writer writer) throws IOException {
+    Class<? extends Reducer> reducerClazz = (Class<? extends Reducer>) conf.getClass("mapred.combiner.class", null, Reducer.class);
+    
+    Reducer combiner = ReflectionUtils.newInstance(reducerClazz, conf);
+    
+    OutputCollector collector = new OutputCollector() {
+      @Override
+      public void collect(Object key, Object value) throws IOException {
+        writer.append(key, value);
+      }
+    };
+    
+    CombinerValuesIterator values = new CombinerValuesIterator(rawIter, keyClass, valClass, comparator);
+    
+    while (values.moveToNext()) {
+      combiner.reduce(values.getKey(), values.getValues().iterator(), collector, reporter);
+    }
+  }
+  
+  private final class CombinerValuesIterator<KEY,VALUE> extends ValuesIterator<KEY, VALUE> {
+    public CombinerValuesIterator(TezRawKeyValueIterator rawIter,
+        Class<KEY> keyClass, Class<VALUE> valClass,
+        RawComparator<KEY> comparator) throws IOException {
+      super(rawIter, comparator, keyClass, valClass, conf,
+          combineInputKeyCounter, combineInputValueCounter);
+    }
+  }
+  
+  ///////////////// End of methods for old API //////////////////////
+  
+  ///////////////// Methods for new API //////////////////////
+  
+  private void runNewCombiner(final TezRawKeyValueIterator rawIter, final Writer writer) throws InterruptedException, IOException {
+    
+    RecordWriter recordWriter = new RecordWriter() {
+
+      @Override
+      public void write(Object key, Object value) throws IOException,
+          InterruptedException {
+        writer.append(key, value);
+      }
+
+      @Override
+      public void close(TaskAttemptContext context) throws IOException,
+          InterruptedException {
+        // Will be closed by whoever invokes the combiner.
+      }
+    };
+    
+    Class<? extends org.apache.hadoop.mapreduce.Reducer> reducerClazz = (Class<? extends org.apache.hadoop.mapreduce.Reducer>) conf
+        .getClass(MRJobConfig.COMBINE_CLASS_ATTR, null,
+            org.apache.hadoop.mapreduce.Reducer.class);
+    org.apache.hadoop.mapreduce.Reducer reducer = ReflectionUtils.newInstance(reducerClazz, conf);
+    
+    org.apache.hadoop.mapreduce.Reducer.Context reducerContext =
+        createReduceContext(
+            conf,
+            mrTaskAttemptID,
+            rawIter,
+            new MRCounters.MRCounter(combineInputKeyCounter),
+            new MRCounters.MRCounter(combineInputValueCounter),
+            recordWriter,
+            reporter,
+            (RawComparator)comparator,
+            keyClass,
+            valClass);
+    
+    reducer.run(reducerContext);
+    recordWriter.close(reducerContext);
+  }
+
+  private static <KEYIN, VALUEIN, KEYOUT, VALUEOUT> org.apache.hadoop.mapreduce.Reducer<KEYIN, VALUEIN, KEYOUT, VALUEOUT>.Context createReduceContext(
+      Configuration conf,
+      TaskAttemptID mrTaskAttemptID,
+      final TezRawKeyValueIterator rawIter,
+      Counter combineInputKeyCounter,
+      Counter combineInputValueCounter,
+      RecordWriter<KEYOUT, VALUEOUT> recordWriter,
+      MRTaskReporter reporter,
+      RawComparator<KEYIN> comparator,
+      Class<KEYIN> keyClass,
+      Class<VALUEIN> valClass) throws InterruptedException, IOException {
+
+    RawKeyValueIterator r = new RawKeyValueIterator() {
+
+      @Override
+      public boolean next() throws IOException {
+        return rawIter.next();
+      }
+
+      @Override
+      public DataInputBuffer getValue() throws IOException {
+        return rawIter.getValue();
+      }
+
+      @Override
+      public Progress getProgress() {
+        return rawIter.getProgress();
+      }
+
+      @Override
+      public DataInputBuffer getKey() throws IOException {
+        return rawIter.getKey();
+      }
+
+      @Override
+      public void close() throws IOException {
+        rawIter.close();
+      }
+    };
+
+    ReduceContext<KEYIN, VALUEIN, KEYOUT, VALUEOUT> rContext = new ReduceContextImpl<KEYIN, VALUEIN, KEYOUT, VALUEOUT>(
+        conf, mrTaskAttemptID, r, combineInputKeyCounter,
+        combineInputValueCounter, recordWriter, null, reporter, comparator,
+        keyClass, valClass);
+
+    org.apache.hadoop.mapreduce.Reducer<KEYIN, VALUEIN, KEYOUT, VALUEOUT>.Context reducerContext = new WrappedReducer<KEYIN, VALUEIN, KEYOUT, VALUEOUT>()
+        .getReducerContext(rContext);
+    return reducerContext;
+  }
+
+  
+ 
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/eb0f6ffe/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/newpartition/MRPartitioner.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/newpartition/MRPartitioner.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/newpartition/MRPartitioner.java
index 4a967ad..dcea35c 100644
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/newpartition/MRPartitioner.java
+++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/newpartition/MRPartitioner.java
@@ -24,6 +24,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.tez.common.TezJobConfig;
+import org.apache.tez.engine.common.ConfigUtils;
 import org.apache.tez.mapreduce.hadoop.MRJobConfig;
 
 @SuppressWarnings({ "rawtypes", "unchecked" })
@@ -38,7 +39,7 @@ public class MRPartitioner implements org.apache.tez.engine.api.Partitioner {
   private org.apache.hadoop.mapred.Partitioner oldPartitioner;
 
   public MRPartitioner(Configuration conf) {
-    this.useNewApi = conf.getBoolean("mapred.mapper.new-api", false);
+    this.useNewApi = ConfigUtils.useNewApi(conf);
     this.partitions = conf.getInt(TezJobConfig.TEZ_ENGINE_NUM_EXPECTED_PARTITIONS, 1);
 
     if (useNewApi) {

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/eb0f6ffe/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/newprocessor/MRTaskReporter.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/newprocessor/MRTaskReporter.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/newprocessor/MRTaskReporter.java
index d01e562..c7c9567 100644
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/newprocessor/MRTaskReporter.java
+++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/newprocessor/MRTaskReporter.java
@@ -24,6 +24,7 @@ import org.apache.hadoop.mapred.Counters;
 import org.apache.hadoop.mapred.InputSplit;
 import org.apache.hadoop.mapred.Reporter;
 import org.apache.tez.common.counters.TezCounter;
+import org.apache.tez.engine.newapi.TezInputContext;
 import org.apache.tez.engine.newapi.TezOutputContext;
 import org.apache.tez.engine.newapi.TezProcessorContext;
 import org.apache.tez.engine.newapi.TezTaskContext;
@@ -53,6 +54,12 @@ public class MRTaskReporter
     this.reporter = new MRReporter(context);
     this.isProcessorContext = false;
   }
+  
+  public MRTaskReporter(TezInputContext context) {
+    this.context= context;
+    this.reporter = new MRReporter(context);
+    this.isProcessorContext = false;
+  }
 
   public void setProgress(float progress) {
     if (isProcessorContext) {


[29/50] [abbrv] Rename tez-engine-api to tez-runtime-api and tez-engine is split into 2: - tez-engine-library for user-visible Input/Output/Processor implementations - tez-engine-internals for framework internals

Posted by ss...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/dflt/SortBufferInputStream.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/dflt/SortBufferInputStream.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/dflt/SortBufferInputStream.java
new file mode 100644
index 0000000..b7b1e82
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/dflt/SortBufferInputStream.java
@@ -0,0 +1,271 @@
+/**
+* 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.library.common.sort.impl.dflt;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.IntBuffer;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.io.BoundedByteArrayOutputStream;
+import org.apache.hadoop.io.DataInputBuffer;
+import org.apache.hadoop.io.WritableUtils;
+import org.apache.tez.runtime.library.common.shuffle.impl.InMemoryWriter;
+import org.apache.tez.runtime.library.common.sort.impl.dflt.DefaultSorter.InMemValBytes;
+
+  public class SortBufferInputStream extends InputStream {
+
+  private static final Log LOG = LogFactory.getLog(SortBufferInputStream.class);
+  
+  private final InMemoryShuffleSorter sorter;
+  private InMemoryWriter sortOutput;
+  
+  private int mend;
+  private int recIndex;
+  private final byte[] kvbuffer;       
+  private final IntBuffer kvmeta;
+  private final int partitionBytes;
+  private final int partition;
+  
+  byte[] dualBuf = new byte[8192];
+  DualBufferOutputStream out;
+  private int readBytes = 0;
+  
+  public SortBufferInputStream(
+      InMemoryShuffleSorter sorter, int partition) {
+    this.sorter = sorter;
+    this.partitionBytes = 
+        (int)sorter.getShuffleHeader(partition).getCompressedLength();
+    this.partition = partition;
+    this.mend = sorter.getMetaEnd();
+    this.recIndex = sorter.getSpillIndex(partition);
+    this.kvbuffer = sorter.kvbuffer;
+    this.kvmeta = sorter.kvmeta;
+    out = new DualBufferOutputStream(null, 0, 0, dualBuf);
+    sortOutput = new InMemoryWriter(out);
+  }
+  
+  byte[] one = new byte[1];
+  
+  @Override
+  public int read() throws IOException {
+    int b = read(one, 0, 1);
+    return (b == -1) ? b : one[0]; 
+  }
+
+  @Override
+  public int read(byte[] b) throws IOException {
+    return read(b, 0, b.length);
+  }
+
+  @Override
+  public int read(byte[] b, int off, int len) throws IOException {
+    if (available() == 0) {
+      return -1;
+    }
+    
+    int currentOffset = off;
+    int currentLength = len;
+    int currentReadBytes = 0;
+    
+    // Check if there is residual data in the dualBuf
+    int residualLen = out.getCurrent();
+    if (residualLen > 0) {
+      int readable = Math.min(currentLength, residualLen);
+      System.arraycopy(dualBuf, 0, b, currentOffset, readable);
+      currentOffset += readable;
+      currentReadBytes += readable;
+      out.setCurrentPointer(-readable);
+      
+      // buffer has less capacity
+      currentLength -= readable;
+      
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("XXX read_residual:" +
+            " readable=" + readable +
+            " readBytes=" + readBytes);
+      }
+    }
+    
+    // Now, use the provided buffer
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("XXX read: out.reset" +
+          " b=" + b + 
+          " currentOffset=" + currentOffset + 
+          " currentLength=" + currentLength +
+          " recIndex=" + recIndex);
+    }
+    out.reset(b, currentOffset, currentLength);
+    
+    // Read from sort-buffer into the provided buffer, space permitting
+    DataInputBuffer key = new DataInputBuffer();
+    final InMemValBytes value = sorter.createInMemValBytes();
+    
+    int kvPartition = 0;
+    int numRec = 0;
+    for (;
+         currentLength > 0 && recIndex < mend && 
+             (kvPartition = getKVPartition(recIndex)) == partition;
+        ++recIndex) {
+      
+      final int kvoff = sorter.offsetFor(recIndex);
+      
+      int keyLen = 
+          (kvmeta.get(kvoff + InMemoryShuffleSorter.VALSTART) - 
+              kvmeta.get(kvoff + InMemoryShuffleSorter.KEYSTART));
+      key.reset(
+          kvbuffer, 
+          kvmeta.get(kvoff + InMemoryShuffleSorter.KEYSTART),
+          keyLen
+          );
+      
+      int valLen = sorter.getVBytesForOffset(kvoff, value);
+
+      int recLen = 
+          (keyLen + WritableUtils.getVIntSize(keyLen)) + 
+          (valLen + WritableUtils.getVIntSize(valLen));
+      
+      currentReadBytes += recLen;
+      currentOffset += recLen;
+      currentLength -= recLen;
+
+      // Write out key/value into the in-mem ifile
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("XXX read: sortOutput.append" +
+            " #rec=" + ++numRec +
+            " recIndex=" + recIndex + " kvoff=" + kvoff + 
+            " keyLen=" + keyLen + " valLen=" + valLen + " recLen=" + recLen +
+            " readBytes=" + readBytes +
+            " currentReadBytes="  + currentReadBytes +
+            " currentLength=" + currentLength);
+      }
+      sortOutput.append(key, value);
+    }
+
+    // If we are at the end of the segment, close the ifile
+    if (currentLength > 0 && 
+        (recIndex == mend || kvPartition != partition)) {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("XXX About to call close:" +
+            " currentLength=" + currentLength + 
+            " recIndex=" + recIndex + " mend=" + mend + 
+            " kvPartition=" + kvPartition + " partitino=" + partition);
+      }
+      sortOutput.close();
+      currentReadBytes += 
+          (InMemoryShuffleSorter.IFILE_EOF_LENGTH + 
+              InMemoryShuffleSorter.IFILE_CHECKSUM_LENGTH);
+    } else {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("XXX Hmm..." +
+            " currentLength=" + currentLength + 
+            " recIndex=" + recIndex + " mend=" + mend + 
+            " kvPartition=" + kvPartition + " partitino=" + partition);
+      }
+    }
+    
+    int retVal = Math.min(currentReadBytes, len);
+    readBytes += retVal;
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("XXX read: done" +
+          " retVal=" + retVal + 
+          " currentReadBytes=" + currentReadBytes +
+          " len=" + len + 
+          " readBytes=" + readBytes +
+          " partitionBytes=" + partitionBytes +
+          " residualBytes=" + out.getCurrent());
+    }
+    return retVal;
+  }
+
+  private int getKVPartition(int recIndex) {
+    return kvmeta.get(
+        sorter.offsetFor(recIndex) + InMemoryShuffleSorter.PARTITION);
+  }
+  
+  @Override
+  public int available() throws IOException {
+    return (partitionBytes - readBytes);
+  }
+
+  @Override
+  public void close() throws IOException {
+    super.close();
+  }
+
+  @Override
+  public boolean markSupported() {
+    return false;
+  }
+  
+  static class DualBufferOutputStream extends BoundedByteArrayOutputStream {
+
+    byte[] dualBuf;
+    int currentPointer = 0;
+    byte[] one = new byte[1];
+    
+    public DualBufferOutputStream(
+        byte[] buf, int offset, int length, 
+        byte[] altBuf) {
+      super(buf, offset, length);
+      this.dualBuf = altBuf;
+    }
+    
+    public void reset(byte[] b, int off, int len) {
+      super.resetBuffer(b, off, len);
+    }
+
+    @Override
+    public void write(int b) throws IOException {
+      one[0] = (byte)b;
+      write(one, 0, 1);
+    }
+
+    @Override
+    public void write(byte[] b) throws IOException {
+      write(b, 0, b.length);
+    }
+
+    @Override
+    public void write(byte[] b, int off, int len) throws IOException {
+      int available = super.available();
+      if (available >= len) {
+        super.write(b, off, len);
+      } else {
+        super.write(b, off, available);
+        System.arraycopy(b, off+available, dualBuf, currentPointer, len-available);
+        currentPointer += (len - available);
+      }
+    }
+    
+    int getCurrent() {
+      return currentPointer;
+    }
+    
+    void setCurrentPointer(int delta) {
+      if ((currentPointer + delta) > dualBuf.length) {
+        throw new IndexOutOfBoundsException("Trying to set dualBuf 'current'" +
+        		" marker to " + (currentPointer+delta) + " when " +
+        		" dualBuf.length is " + dualBuf.length);
+      }
+      currentPointer = (currentPointer + delta) % dualBuf.length;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/task/impl/ValuesIterator.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/task/impl/ValuesIterator.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/task/impl/ValuesIterator.java
new file mode 100644
index 0000000..88cb750
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/task/impl/ValuesIterator.java
@@ -0,0 +1,149 @@
+/**
+* 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.library.common.task.impl;
+
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.NoSuchElementException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.DataInputBuffer;
+import org.apache.hadoop.io.RawComparator;
+import org.apache.hadoop.io.serializer.Deserializer;
+import org.apache.hadoop.io.serializer.SerializationFactory;
+import org.apache.hadoop.util.Progressable;
+import org.apache.tez.runtime.library.common.sort.impl.TezRawKeyValueIterator;
+
+
+/**
+ * Iterates values while keys match in sorted input.
+ *
+ * Usage: Call moveToNext to move to the next k, v pair. This returns true if another exists,
+ * followed by getKey() and getValues() to get the current key and list of values.
+ * 
+ */
+public class ValuesIterator<KEY,VALUE> implements Iterator<VALUE> {
+  protected TezRawKeyValueIterator in; //input iterator
+  private KEY key;               // current key
+  private KEY nextKey;
+  private VALUE value;             // current value
+  private boolean hasNext;                      // more w/ this key
+  private boolean more;                         // more in file
+  private RawComparator<KEY> comparator;
+  protected Progressable reporter;
+  private Deserializer<KEY> keyDeserializer;
+  private Deserializer<VALUE> valDeserializer;
+  private DataInputBuffer keyIn = new DataInputBuffer();
+  private DataInputBuffer valueIn = new DataInputBuffer();
+  
+  public ValuesIterator (TezRawKeyValueIterator in, 
+                         RawComparator<KEY> comparator, 
+                         Class<KEY> keyClass,
+                         Class<VALUE> valClass, Configuration conf, 
+                         Progressable reporter)
+    throws IOException {
+    this.in = in;
+    this.comparator = comparator;
+    this.reporter = reporter;
+    SerializationFactory serializationFactory = new SerializationFactory(conf);
+    this.keyDeserializer = serializationFactory.getDeserializer(keyClass);
+    this.keyDeserializer.open(keyIn);
+    this.valDeserializer = serializationFactory.getDeserializer(valClass);
+    this.valDeserializer.open(this.valueIn);
+    readNextKey();
+    key = nextKey;
+    nextKey = null; // force new instance creation
+    hasNext = more;
+  }
+
+  TezRawKeyValueIterator getRawIterator() { return in; }
+  
+  /// Iterator methods
+
+  public boolean hasNext() { return hasNext; }
+
+  private int ctr = 0;
+  public VALUE next() {
+    if (!hasNext) {
+      throw new NoSuchElementException("iterate past last value");
+    }
+    try {
+      readNextValue();
+      readNextKey();
+    } catch (IOException ie) {
+      throw new RuntimeException("problem advancing post rec#"+ctr, ie);
+    }
+    reporter.progress();
+    return value;
+  }
+
+  public void remove() { throw new RuntimeException("not implemented"); }
+
+  /// Auxiliary methods
+
+  /** Start processing next unique key. */
+  public void nextKey() throws IOException {
+    // read until we find a new key
+    while (hasNext) { 
+      readNextKey();
+    }
+    ++ctr;
+    
+    // move the next key to the current one
+    KEY tmpKey = key;
+    key = nextKey;
+    nextKey = tmpKey;
+    hasNext = more;
+  }
+
+  /** True iff more keys remain. */
+  public boolean more() { 
+    return more; 
+  }
+
+  /** The current key. */
+  public KEY getKey() { 
+    return key; 
+  }
+
+  /** 
+   * read the next key 
+   */
+  private void readNextKey() throws IOException {
+    more = in.next();
+    if (more) {
+      DataInputBuffer nextKeyBytes = in.getKey();
+      keyIn.reset(nextKeyBytes.getData(), nextKeyBytes.getPosition(), nextKeyBytes.getLength());
+      nextKey = keyDeserializer.deserialize(nextKey);
+      hasNext = key != null && (comparator.compare(key, nextKey) == 0);
+    } else {
+      hasNext = false;
+    }
+  }
+
+  /**
+   * Read the next value
+   * @throws IOException
+   */
+  private void readNextValue() throws IOException {
+    DataInputBuffer nextValueBytes = in.getValue();
+    valueIn.reset(nextValueBytes.getData(), nextValueBytes.getPosition(), nextValueBytes.getLength());
+    value = valDeserializer.deserialize(value);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/task/local/output/TezLocalTaskOutputFiles.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/task/local/output/TezLocalTaskOutputFiles.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/task/local/output/TezLocalTaskOutputFiles.java
new file mode 100644
index 0000000..30d28f0
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/task/local/output/TezLocalTaskOutputFiles.java
@@ -0,0 +1,249 @@
+/**
+ * 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.library.common.task.local.output;
+
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+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.tez.common.TezJobConfig;
+import org.apache.tez.runtime.library.common.Constants;
+import org.apache.tez.runtime.library.common.InputAttemptIdentifier;
+
+/**
+ * Manipulate the working area for the transient store for maps and reduces.
+ *
+ * This class is used by map and reduce tasks to identify the directories that
+ * they need to write to/read from for intermediate files. The callers of
+ * these methods are from the Child running the Task.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class TezLocalTaskOutputFiles extends TezTaskOutput {
+
+  public TezLocalTaskOutputFiles(Configuration conf, String uniqueId) {
+    super(conf, uniqueId);
+  }
+
+  private LocalDirAllocator lDirAlloc =
+    new LocalDirAllocator(TezJobConfig.LOCAL_DIRS);
+
+
+  /**
+   * Return the path to local map output file created earlier
+   *
+   * @return path
+   * @throws IOException
+   */
+  @Override
+  public Path getOutputFile()
+      throws IOException {
+    return lDirAlloc.getLocalPathToRead(Constants.TEZ_RUNTIME_TASK_OUTPUT_DIR + Path.SEPARATOR
+        + Constants.TEZ_RUNTIME_TASK_OUTPUT_FILENAME_STRING, conf);
+  }
+
+  /**
+   * Create a local map output file name.
+   *
+   * @param size the size of the file
+   * @return path
+   * @throws IOException
+   */
+  @Override
+  public Path getOutputFileForWrite(long size)
+      throws IOException {
+    return lDirAlloc.getLocalPathForWrite(Constants.TEZ_RUNTIME_TASK_OUTPUT_DIR + Path.SEPARATOR
+        + Constants.TEZ_RUNTIME_TASK_OUTPUT_FILENAME_STRING, size, conf);
+  }
+  
+  /**
+   * Create a local map output file name. This should *only* be used if the size
+   * of the file is not known. Otherwise use the equivalent which accepts a size
+   * parameter.
+   * 
+   * @return path
+   * @throws IOException
+   */
+  @Override
+  public Path getOutputFileForWrite() throws IOException {
+    return lDirAlloc.getLocalPathForWrite(Constants.TEZ_RUNTIME_TASK_OUTPUT_DIR
+        + Path.SEPARATOR + Constants.TEZ_RUNTIME_TASK_OUTPUT_FILENAME_STRING,
+        conf);
+  }
+
+  /**
+   * Create a local map output file name on the same volume.
+   */
+  @Override
+  public Path getOutputFileForWriteInVolume(Path existing) {
+    return new Path(existing.getParent(), Constants.TEZ_RUNTIME_TASK_OUTPUT_FILENAME_STRING);
+  }
+
+  /**
+   * Return the path to a local map output index file created earlier
+   *
+   * @return path
+   * @throws IOException
+   */
+  @Override
+  public Path getOutputIndexFile()
+      throws IOException {
+    return lDirAlloc.getLocalPathToRead(Constants.TEZ_RUNTIME_TASK_OUTPUT_DIR + Path.SEPARATOR
+        + Constants.TEZ_RUNTIME_TASK_OUTPUT_FILENAME_STRING + Constants.TEZ_RUNTIME_TASK_OUTPUT_INDEX_SUFFIX_STRING,
+        conf);
+  }
+
+  /**
+   * Create a local map output index file name.
+   *
+   * @param size the size of the file
+   * @return path
+   * @throws IOException
+   */
+  @Override
+  public Path getOutputIndexFileForWrite(long size)
+      throws IOException {
+    return lDirAlloc.getLocalPathForWrite(Constants.TEZ_RUNTIME_TASK_OUTPUT_DIR + Path.SEPARATOR
+        + Constants.TEZ_RUNTIME_TASK_OUTPUT_FILENAME_STRING + Constants.TEZ_RUNTIME_TASK_OUTPUT_INDEX_SUFFIX_STRING,
+        size, conf);
+  }
+
+  /**
+   * Create a local map output index file name on the same volume.
+   */
+  @Override
+  public Path getOutputIndexFileForWriteInVolume(Path existing) {
+    return new Path(existing.getParent(),
+        Constants.TEZ_RUNTIME_TASK_OUTPUT_FILENAME_STRING + Constants.TEZ_RUNTIME_TASK_OUTPUT_INDEX_SUFFIX_STRING);
+  }
+
+  /**
+   * Return a local map spill file created earlier.
+   *
+   * @param spillNumber the number
+   * @return path
+   * @throws IOException
+   */
+  @Override
+  public Path getSpillFile(int spillNumber)
+      throws IOException {
+    return lDirAlloc.getLocalPathToRead(Constants.TEZ_RUNTIME_TASK_OUTPUT_DIR + "/spill"
+        + spillNumber + ".out", conf);
+  }
+
+  /**
+   * Create a local map spill file name.
+   *
+   * @param spillNumber the number
+   * @param size the size of the file
+   * @return path
+   * @throws IOException
+   */
+  @Override
+  public Path getSpillFileForWrite(int spillNumber, long size)
+      throws IOException {
+    return lDirAlloc.getLocalPathForWrite(Constants.TEZ_RUNTIME_TASK_OUTPUT_DIR + "/spill"
+        + spillNumber + ".out", size, conf);
+  }
+
+  /**
+   * Return a local map spill index file created earlier
+   *
+   * @param spillNumber the number
+   * @return path
+   * @throws IOException
+   */
+  @Override
+  public Path getSpillIndexFile(int spillNumber)
+      throws IOException {
+    return lDirAlloc.getLocalPathToRead(Constants.TEZ_RUNTIME_TASK_OUTPUT_DIR + "/spill"
+        + spillNumber + ".out.index", conf);
+  }
+
+  /**
+   * Create a local map spill index file name.
+   *
+   * @param spillNumber the number
+   * @param size the size of the file
+   * @return path
+   * @throws IOException
+   */
+  @Override
+  public Path getSpillIndexFileForWrite(int spillNumber, long size)
+      throws IOException {
+    return lDirAlloc.getLocalPathForWrite(Constants.TEZ_RUNTIME_TASK_OUTPUT_DIR + "/spill"
+        + spillNumber + ".out.index", size, conf);
+  }
+
+  /**
+   * Return a local reduce input file created earlier
+   *
+   * @param mapId a map task id
+   * @return path
+   * @throws IOException
+   */
+  @Override
+  public Path getInputFile(InputAttemptIdentifier mapId)
+      throws IOException {
+    return lDirAlloc.getLocalPathToRead(String.format(
+        Constants.TEZ_RUNTIME_TASK_INPUT_FILE_FORMAT_STRING, 
+        Constants.TEZ_RUNTIME_TASK_OUTPUT_DIR, Integer.valueOf(mapId.getInputIdentifier().getSrcTaskIndex())), conf);
+  }
+
+  /**
+   * Create a local reduce input file name.
+   *
+   * @param mapId a map task id
+   * @param size the size of the file
+   * @return path
+   * @throws IOException
+   */
+  @Override
+  public Path getInputFileForWrite(int taskId,
+                                   long size)
+      throws IOException {
+    return lDirAlloc.getLocalPathForWrite(String.format(
+        Constants.TEZ_RUNTIME_TASK_INPUT_FILE_FORMAT_STRING, Constants.TEZ_RUNTIME_TASK_OUTPUT_DIR, taskId),
+        size, conf);
+  }
+
+  /** Removes all of the files related to a task. */
+  @Override
+  public void removeAll()
+      throws IOException {
+    deleteLocalFiles(Constants.TEZ_RUNTIME_TASK_OUTPUT_DIR);
+  }
+
+  private String[] getLocalDirs() throws IOException {
+    return conf.getStrings(TezJobConfig.LOCAL_DIRS);
+  }
+
+  @SuppressWarnings("deprecation")
+  private void deleteLocalFiles(String subdir) throws IOException {
+    String[] localDirs = getLocalDirs();
+    for (int i = 0; i < localDirs.length; i++) {
+      FileSystem.getLocal(conf).delete(new Path(localDirs[i], subdir));
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/task/local/output/TezTaskOutput.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/task/local/output/TezTaskOutput.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/task/local/output/TezTaskOutput.java
new file mode 100644
index 0000000..d3e7d27
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/task/local/output/TezTaskOutput.java
@@ -0,0 +1,165 @@
+/**
+ * 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.library.common.task.local.output;
+
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.tez.runtime.library.common.InputAttemptIdentifier;
+
+/**
+ * Manipulate the working area for the transient store for maps and reduces.
+ *
+ * This class is used by map and reduce tasks to identify the directories that
+ * they need to write to/read from for intermediate files. The callers of
+ * these methods are from child space and see mapreduce.cluster.local.dir as
+ * taskTracker/jobCache/jobId/attemptId
+ * This class should not be used from TaskTracker space.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public abstract class TezTaskOutput {
+
+  protected Configuration conf;
+  protected String uniqueId;
+
+  public TezTaskOutput(Configuration conf, String uniqueId) {
+    this.conf = conf;
+    this.uniqueId = uniqueId;
+  }
+
+  /**
+   * Return the path to local map output file created earlier
+   *
+   * @return path
+   * @throws IOException
+   */
+  public abstract Path getOutputFile() throws IOException;
+
+  /**
+   * Create a local map output file name.
+   *
+   * @param size the size of the file
+   * @return path
+   * @throws IOException
+   */
+  public abstract Path getOutputFileForWrite(long size) throws IOException;
+
+  /**
+   * Create a local output file name. This method is meant to be used *only* if
+   * the size of the file is not know up front.
+   * 
+   * @return path
+   * @throws IOException
+   */
+  public abstract Path getOutputFileForWrite() throws IOException;
+  
+  /**
+   * Create a local map output file name on the same volume.
+   */
+  public abstract Path getOutputFileForWriteInVolume(Path existing);
+
+  /**
+   * Return the path to a local map output index file created earlier
+   *
+   * @return path
+   * @throws IOException
+   */
+  public abstract Path getOutputIndexFile() throws IOException;
+
+  /**
+   * Create a local map output index file name.
+   *
+   * @param size the size of the file
+   * @return path
+   * @throws IOException
+   */
+  public abstract Path getOutputIndexFileForWrite(long size) throws IOException;
+
+  /**
+   * Create a local map output index file name on the same volume.
+   */
+  public abstract Path getOutputIndexFileForWriteInVolume(Path existing);
+
+  /**
+   * Return a local map spill file created earlier.
+   *
+   * @param spillNumber the number
+   * @return path
+   * @throws IOException
+   */
+  public abstract Path getSpillFile(int spillNumber) throws IOException;
+
+  /**
+   * Create a local map spill file name.
+   *
+   * @param spillNumber the number
+   * @param size the size of the file
+   * @return path
+   * @throws IOException
+   */
+  public abstract Path getSpillFileForWrite(int spillNumber, long size)
+      throws IOException;
+
+  /**
+   * Return a local map spill index file created earlier
+   *
+   * @param spillNumber the number
+   * @return path
+   * @throws IOException
+   */
+  public abstract Path getSpillIndexFile(int spillNumber) throws IOException;
+
+  /**
+   * Create a local map spill index file name.
+   *
+   * @param spillNumber the number
+   * @param size the size of the file
+   * @return path
+   * @throws IOException
+   */
+  public abstract Path getSpillIndexFileForWrite(int spillNumber, long size)
+      throws IOException;
+
+  /**
+   * Return a local reduce input file created earlier
+   *
+   * @param attemptIdentifier The identifier for the source task
+   * @return path
+   * @throws IOException
+   */
+  public abstract Path getInputFile(InputAttemptIdentifier attemptIdentifier) throws IOException;
+
+  /**
+   * Create a local reduce input file name.
+   *
+   * @param taskIdentifier The identifier for the source task
+   * @param size the size of the file
+   * @return path
+   * @throws IOException
+   */
+  public abstract Path getInputFileForWrite(
+      int taskIdentifier, long size) throws IOException;
+
+  /** Removes all of the files related to a task. */
+  public abstract void removeAll() throws IOException;
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/task/local/output/TezTaskOutputFiles.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/task/local/output/TezTaskOutputFiles.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/task/local/output/TezTaskOutputFiles.java
new file mode 100644
index 0000000..2c18b4e
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/task/local/output/TezTaskOutputFiles.java
@@ -0,0 +1,246 @@
+/**
+ * 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.library.common.task.local.output;
+
+import java.io.IOException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.LocalDirAllocator;
+import org.apache.hadoop.fs.Path;
+import org.apache.tez.common.TezJobConfig;
+import org.apache.tez.runtime.library.common.Constants;
+import org.apache.tez.runtime.library.common.InputAttemptIdentifier;
+
+/**
+ * Manipulate the working area for the transient store for maps and reduces.
+ *
+ * This class is used by map and reduce tasks to identify the directories that
+ * they need to write to/read from for intermediate files. The callers of
+ * these methods are from child space and see mapreduce.cluster.local.dir as
+ * taskTracker/jobCache/jobId/attemptId
+ * This class should not be used from TaskTracker space.
+ */
+
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class TezTaskOutputFiles extends TezTaskOutput {
+  
+  public TezTaskOutputFiles(Configuration conf, String uniqueId) {
+    super(conf, uniqueId);
+  }
+
+  private static final Log LOG = LogFactory.getLog(TezTaskOutputFiles.class);
+
+  private static final String SPILL_FILE_PATTERN = "%s_spill_%d.out";
+  private static final String SPILL_INDEX_FILE_PATTERN = SPILL_FILE_PATTERN
+      + ".index";
+
+  
+
+  // assume configured to $localdir/usercache/$user/appcache/$appId
+  private LocalDirAllocator lDirAlloc =
+    new LocalDirAllocator(TezJobConfig.LOCAL_DIRS);
+  
+
+  private Path getAttemptOutputDir() {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("getAttemptOutputDir: "
+          + Constants.TEZ_RUNTIME_TASK_OUTPUT_DIR + "/"
+          + uniqueId);
+    }
+    return new Path(Constants.TEZ_RUNTIME_TASK_OUTPUT_DIR, uniqueId);
+  }
+
+  /**
+   * Return the path to local map output file created earlier
+   *
+   * @return path
+   * @throws IOException
+   */
+  public Path getOutputFile() throws IOException {
+    Path attemptOutput =
+      new Path(getAttemptOutputDir(), Constants.TEZ_RUNTIME_TASK_OUTPUT_FILENAME_STRING);
+    return lDirAlloc.getLocalPathToRead(attemptOutput.toString(), conf);
+  }
+
+  /**
+   * Create a local map output file name.
+   *
+   * @param size the size of the file
+   * @return path
+   * @throws IOException
+   */
+  public Path getOutputFileForWrite(long size) throws IOException {
+    Path attemptOutput =
+      new Path(getAttemptOutputDir(), Constants.TEZ_RUNTIME_TASK_OUTPUT_FILENAME_STRING);
+    return lDirAlloc.getLocalPathForWrite(attemptOutput.toString(), size, conf);
+  }
+
+  /**
+   * Create a local map output file name. This should *only* be used if the size
+   * of the file is not known. Otherwise use the equivalent which accepts a size
+   * parameter.
+   * 
+   * @return path
+   * @throws IOException
+   */
+  public Path getOutputFileForWrite() throws IOException {
+    Path attemptOutput =
+      new Path(getAttemptOutputDir(), Constants.TEZ_RUNTIME_TASK_OUTPUT_FILENAME_STRING);
+    return lDirAlloc.getLocalPathForWrite(attemptOutput.toString(), conf);
+  }
+
+  /**
+   * Create a local map output file name on the same volume.
+   */
+  public Path getOutputFileForWriteInVolume(Path existing) {
+    Path outputDir = new Path(existing.getParent(), Constants.TEZ_RUNTIME_TASK_OUTPUT_DIR);
+    Path attemptOutputDir = new Path(outputDir, uniqueId);
+    return new Path(attemptOutputDir, Constants.TEZ_RUNTIME_TASK_OUTPUT_FILENAME_STRING);
+  }
+
+  /**
+   * Return the path to a local map output index file created earlier
+   *
+   * @return path
+   * @throws IOException
+   */
+  public Path getOutputIndexFile() throws IOException {
+    Path attemptIndexOutput =
+      new Path(getAttemptOutputDir(), Constants.TEZ_RUNTIME_TASK_OUTPUT_FILENAME_STRING +
+                                      Constants.TEZ_RUNTIME_TASK_OUTPUT_INDEX_SUFFIX_STRING);
+    return lDirAlloc.getLocalPathToRead(attemptIndexOutput.toString(), conf);
+  }
+
+  /**
+   * Create a local map output index file name.
+   *
+   * @param size the size of the file
+   * @return path
+   * @throws IOException
+   */
+  public Path getOutputIndexFileForWrite(long size) throws IOException {
+    Path attemptIndexOutput =
+      new Path(getAttemptOutputDir(), Constants.TEZ_RUNTIME_TASK_OUTPUT_FILENAME_STRING +
+                                      Constants.TEZ_RUNTIME_TASK_OUTPUT_INDEX_SUFFIX_STRING);
+    return lDirAlloc.getLocalPathForWrite(attemptIndexOutput.toString(),
+        size, conf);
+  }
+
+  /**
+   * Create a local map output index file name on the same volume.
+   */
+  public Path getOutputIndexFileForWriteInVolume(Path existing) {
+    Path outputDir = new Path(existing.getParent(), Constants.TEZ_RUNTIME_TASK_OUTPUT_DIR);
+    Path attemptOutputDir = new Path(outputDir, uniqueId);
+    return new Path(attemptOutputDir, Constants.TEZ_RUNTIME_TASK_OUTPUT_FILENAME_STRING +
+                                      Constants.TEZ_RUNTIME_TASK_OUTPUT_INDEX_SUFFIX_STRING);
+  }
+
+  /**
+   * Return a local map spill file created earlier.
+   *
+   * @param spillNumber the number
+   * @return path
+   * @throws IOException
+   */
+  public Path getSpillFile(int spillNumber) throws IOException {
+    return lDirAlloc.getLocalPathToRead(
+        String.format(SPILL_FILE_PATTERN,
+            uniqueId, spillNumber), conf);
+  }
+
+  /**
+   * Create a local map spill file name.
+   *
+   * @param spillNumber the number
+   * @param size the size of the file
+   * @return path
+   * @throws IOException
+   */
+  public Path getSpillFileForWrite(int spillNumber, long size)
+      throws IOException {
+    return lDirAlloc.getLocalPathForWrite(
+        String.format(String.format(SPILL_FILE_PATTERN,
+            uniqueId, spillNumber)), size, conf);
+  }
+
+  /**
+   * Return a local map spill index file created earlier
+   *
+   * @param spillNumber the number
+   * @return path
+   * @throws IOException
+   */
+  public Path getSpillIndexFile(int spillNumber) throws IOException {
+    return lDirAlloc.getLocalPathToRead(
+        String.format(SPILL_INDEX_FILE_PATTERN,
+            uniqueId, spillNumber), conf);
+  }
+
+  /**
+   * Create a local map spill index file name.
+   *
+   * @param spillNumber the number
+   * @param size the size of the file
+   * @return path
+   * @throws IOException
+   */
+  public Path getSpillIndexFileForWrite(int spillNumber, long size)
+      throws IOException {
+    return lDirAlloc.getLocalPathForWrite(
+        String.format(SPILL_INDEX_FILE_PATTERN,
+            uniqueId, spillNumber), size, conf);
+  }
+
+  /**
+   * Return a local reduce input file created earlier
+   *
+   * @param attemptIdentifier an identifier for a task. The attempt information is ignored.
+   * @return path
+   * @throws IOException
+   */
+  public Path getInputFile(InputAttemptIdentifier attemptIdentifier) throws IOException {
+    throw new UnsupportedOperationException("Incompatible with LocalRunner");
+  }
+
+  /**
+   * Create a local reduce input file name.
+   *
+   * @param attemptIdentifier an identifier for a task. The attempt information is ignored.
+   * @param size the size of the file
+   * @return path
+   * @throws IOException
+   */
+  public Path getInputFileForWrite(int srcTaskId,
+      long size) throws IOException {
+    return lDirAlloc.getLocalPathForWrite(String.format(
+        uniqueId, getAttemptOutputDir().toString(), srcTaskId),
+        size, conf);
+  }
+
+  /** Removes all of the files related to a task. */
+  public void removeAll() throws IOException {
+    throw new UnsupportedOperationException("Incompatible with LocalRunner");
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/hadoop/compat/NullProgressable.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/hadoop/compat/NullProgressable.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/hadoop/compat/NullProgressable.java
new file mode 100644
index 0000000..eb8d176
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/hadoop/compat/NullProgressable.java
@@ -0,0 +1,33 @@
+/**
+* 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.library.hadoop.compat;
+
+import org.apache.hadoop.util.Progressable;
+
+public class NullProgressable implements Progressable {
+
+  public NullProgressable() {
+    // TODO Auto-generated constructor stub
+  }
+
+  @Override
+  public void progress() {
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/LocalMergedInput.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/LocalMergedInput.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/LocalMergedInput.java
new file mode 100644
index 0000000..3aec247
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/LocalMergedInput.java
@@ -0,0 +1,52 @@
+/**
+ * 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.library.input;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.tez.common.TezUtils;
+import org.apache.tez.runtime.api.Event;
+import org.apache.tez.runtime.api.LogicalInput;
+import org.apache.tez.runtime.api.TezInputContext;
+import org.apache.tez.runtime.library.common.localshuffle.LocalShuffle;
+
+/**
+ * <code>LocalMergedInput</code> in an {@link LogicalInput} which shuffles intermediate
+ * sorted data, merges them and provides key/<values> to the consumer. 
+ */
+public class LocalMergedInput extends ShuffledMergedInputLegacy {
+
+  @Override
+  public List<Event> initialize(TezInputContext inputContext) throws IOException {
+    this.inputContext = inputContext;
+    this.conf = TezUtils.createConfFromUserPayload(inputContext.getUserPayload());
+
+    LocalShuffle localShuffle = new LocalShuffle(inputContext, conf, numInputs);
+    rawIter = localShuffle.run();
+    createValuesIterator();
+    return Collections.emptyList();
+  }
+
+  @Override
+  public List<Event> close() throws IOException {
+    rawIter.close();
+    return Collections.emptyList();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/ShuffledMergedInput.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/ShuffledMergedInput.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/ShuffledMergedInput.java
new file mode 100644
index 0000000..771ac1b
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/ShuffledMergedInput.java
@@ -0,0 +1,179 @@
+/**
+ * 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.library.input;
+
+import java.io.IOException;
+import java.util.Collections;
+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.io.RawComparator;
+import org.apache.tez.common.TezJobConfig;
+import org.apache.tez.common.TezUtils;
+import org.apache.tez.common.counters.TaskCounter;
+import org.apache.tez.common.counters.TezCounter;
+import org.apache.tez.runtime.api.Event;
+import org.apache.tez.runtime.api.LogicalInput;
+import org.apache.tez.runtime.api.TezInputContext;
+import org.apache.tez.runtime.library.api.KVReader;
+import org.apache.tez.runtime.library.common.ConfigUtils;
+import org.apache.tez.runtime.library.common.ValuesIterator;
+import org.apache.tez.runtime.library.common.shuffle.impl.Shuffle;
+import org.apache.tez.runtime.library.common.sort.impl.TezRawKeyValueIterator;
+
+/**
+ * <code>ShuffleMergedInput</code> in a {@link LogicalInput} which shuffles
+ * intermediate sorted data, merges them and provides key/<values> to the
+ * consumer.
+ *
+ * The Copy and Merge will be triggered by the initialization - which is handled
+ * by the Tez framework. Input is not consumable until the Copy and Merge are
+ * complete. Methods are provided to check for this, as well as to wait for
+ * completion. Attempting to get a reader on a non-complete input will block.
+ *
+ */
+public class ShuffledMergedInput implements LogicalInput {
+
+  static final Log LOG = LogFactory.getLog(ShuffledMergedInput.class);
+
+  protected TezInputContext inputContext;
+  protected TezRawKeyValueIterator rawIter = null;
+  protected Configuration conf;
+  protected int numInputs = 0;
+  protected Shuffle shuffle;
+  @SuppressWarnings("rawtypes")
+  protected ValuesIterator vIter;
+
+  private TezCounter inputKeyCounter;
+  private TezCounter inputValueCounter;
+
+  @Override
+  public List<Event> initialize(TezInputContext inputContext) throws IOException {
+    this.inputContext = inputContext;
+    this.conf = TezUtils.createConfFromUserPayload(inputContext.getUserPayload());
+
+    this.inputKeyCounter = inputContext.getCounters().findCounter(TaskCounter.REDUCE_INPUT_GROUPS);
+    this.inputValueCounter = inputContext.getCounters().findCounter(TaskCounter.REDUCE_INPUT_RECORDS);
+    this.conf.setStrings(TezJobConfig.LOCAL_DIRS,
+        inputContext.getWorkDirs());
+
+    // Start the shuffle - copy and merge.
+    shuffle = new Shuffle(inputContext, this.conf, numInputs);
+    shuffle.run();
+
+    return Collections.emptyList();
+  }
+
+  /**
+   * Check if the input is ready for consumption
+   *
+   * @return true if the input is ready for consumption, or if an error occurred
+   *         processing fetching the input. false if the shuffle and merge are
+   *         still in progress
+   */
+  public boolean isInputReady() {
+    return shuffle.isInputReady();
+  }
+
+  /**
+   * Waits for the input to become ready for consumption
+   * @throws IOException
+   * @throws InterruptedException
+   */
+  public void waitForInputReady() throws IOException, InterruptedException {
+    rawIter = shuffle.waitForInput();
+    createValuesIterator();
+  }
+
+  @Override
+  public List<Event> close() throws IOException {
+    rawIter.close();
+    return Collections.emptyList();
+  }
+
+  /**
+   * Get a KVReader for the Input.</p> This method will block until the input is
+   * ready - i.e. the copy and merge stages are complete. Users can use the
+   * isInputReady method to check if the input is ready, which gives an
+   * indication of whether this method will block or not.
+   *
+   * NOTE: All values for the current K-V pair must be read prior to invoking
+   * moveToNext. Once moveToNext() is called, the valueIterator from the
+   * previous K-V pair will throw an Exception
+   *
+   * @return a KVReader over the sorted input.
+   */
+  @Override
+  public KVReader getReader() throws IOException {
+    if (rawIter == null) {
+      try {
+        waitForInputReady();
+      } catch (InterruptedException e) {
+        Thread.currentThread().interrupt();
+        throw new IOException("Interrupted while waiting for input ready", e);
+      }
+    }
+    return new KVReader() {
+
+      @Override
+      public boolean next() throws IOException {
+        return vIter.moveToNext();
+      }
+
+      @SuppressWarnings("unchecked")
+      @Override
+      public KVRecord getCurrentKV() {
+        return new KVRecord(vIter.getKey(), vIter.getValues());
+      }
+    };
+  }
+
+  @Override
+  public void handleEvents(List<Event> inputEvents) {
+    shuffle.handleEvents(inputEvents);
+  }
+
+  @Override
+  public void setNumPhysicalInputs(int numInputs) {
+    this.numInputs = numInputs;
+  }
+
+  @SuppressWarnings({ "rawtypes", "unchecked" })
+  protected void createValuesIterator()
+      throws IOException {
+    vIter = new ValuesIterator(rawIter,
+        (RawComparator) ConfigUtils.getIntermediateInputKeyComparator(conf),
+        ConfigUtils.getIntermediateInputKeyClass(conf),
+        ConfigUtils.getIntermediateInputValueClass(conf), conf, inputKeyCounter, inputValueCounter);
+
+  }
+
+  // This functionality is currently broken. If there's inputs which need to be
+  // written to disk, there's a possibility that inputs from the different
+  // sources could clobber each others' output. Also the current structures do
+  // not have adequate information to de-dupe these (vertex name)
+//  public void mergeWith(ShuffledMergedInput other) {
+//    this.numInputs += other.getNumPhysicalInputs();
+//  }
+//
+//  public int getNumPhysicalInputs() {
+//    return this.numInputs;
+//  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/ShuffledMergedInputLegacy.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/ShuffledMergedInputLegacy.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/ShuffledMergedInputLegacy.java
new file mode 100644
index 0000000..97e19d8
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/ShuffledMergedInputLegacy.java
@@ -0,0 +1,30 @@
+/**
+ * <code>ShuffleMergedInput</code> in a {@link LogicalInput} which shuffles
+ * intermediate sorted data, merges them and provides key/<values> to the
+ * consumer.
+ * 
+ * The Copy and Merge will be triggered by the initialization - which is handled
+ * by the Tez framework. Input is not consumable until the Copy and Merge are
+ * complete. Methods are provided to check for this, as well as to wait for
+ * completion. Attempting to get a reader on a non-complete input will block.
+ * 
+ */
+
+package org.apache.tez.runtime.library.input;
+
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience.LimitedPrivate;
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.tez.runtime.library.common.sort.impl.TezRawKeyValueIterator;
+
+@LimitedPrivate("mapreduce")
+public class ShuffledMergedInputLegacy extends ShuffledMergedInput {
+
+  @Private
+  public TezRawKeyValueIterator getIterator() throws IOException, InterruptedException {
+    // wait for input so that iterator is available
+    waitForInputReady();
+    return rawIter;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/ShuffledUnorderedKVInput.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/ShuffledUnorderedKVInput.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/ShuffledUnorderedKVInput.java
new file mode 100644
index 0000000..42b2e00
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/ShuffledUnorderedKVInput.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.runtime.library.input;
+
+import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.tez.common.TezJobConfig;
+import org.apache.tez.common.TezUtils;
+import org.apache.tez.runtime.api.Event;
+import org.apache.tez.runtime.api.LogicalInput;
+import org.apache.tez.runtime.api.Reader;
+import org.apache.tez.runtime.api.TezInputContext;
+import org.apache.tez.runtime.library.broadcast.input.BroadcastShuffleManager;
+
+import com.google.common.base.Preconditions;
+
+public class ShuffledUnorderedKVInput implements LogicalInput {
+
+  private Configuration conf;
+  private int numInputs = -1;
+  private BroadcastShuffleManager shuffleManager;
+  
+  
+  
+  public ShuffledUnorderedKVInput() {
+  }
+
+  @Override
+  public List<Event> initialize(TezInputContext inputContext) throws Exception {
+    Preconditions.checkArgument(numInputs != -1, "Number of Inputs has not been set");
+    this.conf = TezUtils.createConfFromUserPayload(inputContext.getUserPayload());
+    this.conf.setStrings(TezJobConfig.LOCAL_DIRS, inputContext.getWorkDirs());
+    
+    this.shuffleManager = new BroadcastShuffleManager(inputContext, conf, numInputs);
+    return null;
+  }
+
+  @Override
+  public Reader getReader() throws Exception {
+    // TODO Auto-generated method stub
+    return null;
+  }
+
+  @Override
+  public void handleEvents(List<Event> inputEvents) {
+    shuffleManager.handleEvents(inputEvents);
+  }
+
+  @Override
+  public List<Event> close() throws Exception {
+    this.shuffleManager.shutdown();
+    return null;
+  }
+
+  @Override
+  public void setNumPhysicalInputs(int numInputs) {
+    this.numInputs = numInputs;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/InMemorySortedOutput.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/InMemorySortedOutput.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/InMemorySortedOutput.java
new file mode 100644
index 0000000..2ec6b2a
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/InMemorySortedOutput.java
@@ -0,0 +1,81 @@
+/**
+ * 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.library.output;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.tez.common.TezUtils;
+import org.apache.tez.runtime.api.Event;
+import org.apache.tez.runtime.api.LogicalOutput;
+import org.apache.tez.runtime.api.Output;
+import org.apache.tez.runtime.api.TezOutputContext;
+import org.apache.tez.runtime.api.Writer;
+import org.apache.tez.runtime.library.api.KVWriter;
+import org.apache.tez.runtime.library.common.sort.impl.dflt.InMemoryShuffleSorter;
+
+/**
+ * {@link InMemorySortedOutput} is an {@link Output} which sorts key/value pairs 
+ * written to it and persists it to a file.
+ */
+public class InMemorySortedOutput implements LogicalOutput {
+  
+  protected InMemoryShuffleSorter sorter;
+  protected int numTasks;
+  protected TezOutputContext outputContext;
+  
+
+  @Override
+  public List<Event> initialize(TezOutputContext outputContext)
+      throws IOException {
+    this.outputContext = outputContext;
+    this.sorter = new InMemoryShuffleSorter();
+    sorter.initialize(outputContext, TezUtils.createConfFromUserPayload(outputContext.getUserPayload()), numTasks);
+    return Collections.emptyList();
+  }
+
+  @Override
+  public Writer getWriter() throws IOException {
+    return new KVWriter() {
+      
+      @Override
+      public void write(Object key, Object value) throws IOException {
+        sorter.write(key, value);
+      }
+    };
+  }
+
+  @Override
+  public void handleEvents(List<Event> outputEvents) {
+    // No events expected.
+  }
+
+  @Override
+  public void setNumPhysicalOutputs(int numOutputs) {
+    this.numTasks = numOutputs;
+  }
+  
+  @Override
+  public List<Event> close() throws IOException {
+    sorter.flush();
+    sorter.close();
+    // TODO NEWTEZ Event generation
+    return null;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/LocalOnFileSorterOutput.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/LocalOnFileSorterOutput.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/LocalOnFileSorterOutput.java
new file mode 100644
index 0000000..a19d5e1
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/LocalOnFileSorterOutput.java
@@ -0,0 +1,63 @@
+/**
+* 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.library.output;
+
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.tez.runtime.api.Event;
+import org.apache.tez.runtime.library.common.task.local.output.TezTaskOutput;
+
+public class LocalOnFileSorterOutput extends OnFileSortedOutput {
+
+  private static final Log LOG = LogFactory.getLog(LocalOnFileSorterOutput.class);
+
+  
+
+  @Override
+  public List<Event> close() throws IOException {
+    LOG.debug("Closing LocalOnFileSorterOutput");
+    super.close();
+
+    TezTaskOutput mapOutputFile = sorter.getMapOutput();
+    FileSystem localFs = FileSystem.getLocal(conf);
+
+    Path src = mapOutputFile.getOutputFile();
+    Path dst =
+        mapOutputFile.getInputFileForWrite(
+            outputContext.getTaskIndex(),
+            localFs.getFileStatus(src).getLen());
+
+    LOG.info("Renaming src = " + src + ", dst = " + dst);
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Renaming src = " + src + ", dst = " + dst);
+    }
+    localFs.rename(src, dst);
+    return null;
+  }
+  
+  @Override
+  protected List<Event> generateDataMovementEventsOnClose() throws IOException {
+    return null;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/OnFileSortedOutput.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/OnFileSortedOutput.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/OnFileSortedOutput.java
new file mode 100644
index 0000000..42e1eeb
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/OnFileSortedOutput.java
@@ -0,0 +1,123 @@
+/**
+ * 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.library.output;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.api.ApplicationConstants;
+import org.apache.tez.common.TezJobConfig;
+import org.apache.tez.common.TezUtils;
+import org.apache.tez.runtime.api.Event;
+import org.apache.tez.runtime.api.LogicalOutput;
+import org.apache.tez.runtime.api.TezOutputContext;
+import org.apache.tez.runtime.api.events.DataMovementEvent;
+import org.apache.tez.runtime.library.api.KVWriter;
+import org.apache.tez.runtime.library.common.sort.impl.ExternalSorter;
+import org.apache.tez.runtime.library.common.sort.impl.dflt.DefaultSorter;
+import org.apache.tez.runtime.library.shuffle.common.ShuffleUtils;
+import org.apache.tez.runtime.library.shuffle.impl.ShuffleUserPayloads.DataMovementEventPayloadProto;
+
+import com.google.common.collect.Lists;
+
+/**
+ * <code>OnFileSortedOutput</code> is an {@link LogicalOutput} which sorts key/value pairs 
+ * written to it and persists it to a file.
+ */
+public class OnFileSortedOutput implements LogicalOutput {
+  
+  protected ExternalSorter sorter;
+  protected Configuration conf;
+  protected int numOutputs;
+  protected TezOutputContext outputContext;
+  private long startTime;
+  private long endTime;
+  
+  
+  @Override
+  public List<Event> initialize(TezOutputContext outputContext)
+      throws IOException {
+    this.startTime = System.nanoTime();
+    this.outputContext = outputContext;
+    sorter = new DefaultSorter();
+    this.conf = TezUtils.createConfFromUserPayload(outputContext.getUserPayload());
+    // Initializing this parametr in this conf since it is used in multiple
+    // places (wherever LocalDirAllocator is used) - TezTaskOutputFiles,
+    // TezMerger, etc.
+    this.conf.setStrings(TezJobConfig.LOCAL_DIRS, outputContext.getWorkDirs());
+    sorter.initialize(outputContext, conf, numOutputs);
+    return Collections.emptyList();
+  }
+
+  @Override
+  public KVWriter getWriter() throws IOException {
+    return new KVWriter() {
+      @Override
+      public void write(Object key, Object value) throws IOException {
+        sorter.write(key, value);
+      }
+    };
+  }
+
+  @Override
+  public void handleEvents(List<Event> outputEvents) {
+    // Not expecting any events.
+  }
+
+  @Override
+  public void setNumPhysicalOutputs(int numOutputs) {
+    this.numOutputs = numOutputs;
+  }
+
+  @Override
+  public List<Event> close() throws IOException {
+    sorter.flush();
+    sorter.close();
+    this.endTime = System.nanoTime();
+
+   return generateDataMovementEventsOnClose();
+  }
+  
+  protected List<Event> generateDataMovementEventsOnClose() throws IOException {
+    String host = System.getenv(ApplicationConstants.Environment.NM_HOST
+        .toString());
+    ByteBuffer shuffleMetadata = outputContext
+        .getServiceProviderMetaData(ShuffleUtils.SHUFFLE_HANDLER_SERVICE_ID);
+    int shufflePort = ShuffleUtils.deserializeShuffleProviderMetaData(shuffleMetadata);
+
+    DataMovementEventPayloadProto.Builder payloadBuilder = DataMovementEventPayloadProto
+        .newBuilder();
+    payloadBuilder.setHost(host);
+    payloadBuilder.setPort(shufflePort);
+    payloadBuilder.setPathComponent(outputContext.getUniqueIdentifier());
+    payloadBuilder.setRunDuration((int) ((endTime - startTime) / 1000));
+    DataMovementEventPayloadProto payloadProto = payloadBuilder.build();
+    byte[] payloadBytes = payloadProto.toByteArray();
+
+    List<Event> events = Lists.newArrayListWithCapacity(numOutputs);
+
+    for (int i = 0; i < numOutputs; i++) {
+      DataMovementEvent event = new DataMovementEvent(i, payloadBytes);
+      events.add(event);
+    }
+    return events;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/OnFileUnorderedKVOutput.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/OnFileUnorderedKVOutput.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/OnFileUnorderedKVOutput.java
new file mode 100644
index 0000000..dd18149
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/OnFileUnorderedKVOutput.java
@@ -0,0 +1,98 @@
+/**
+* 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.library.output;
+
+import java.nio.ByteBuffer;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.hadoop.yarn.api.ApplicationConstants;
+import org.apache.tez.dag.api.TezUncheckedException;
+import org.apache.tez.runtime.api.Event;
+import org.apache.tez.runtime.api.LogicalOutput;
+import org.apache.tez.runtime.api.TezOutputContext;
+import org.apache.tez.runtime.api.events.DataMovementEvent;
+import org.apache.tez.runtime.library.api.KVWriter;
+import org.apache.tez.runtime.library.broadcast.output.FileBasedKVWriter;
+import org.apache.tez.runtime.library.shuffle.common.ShuffleUtils;
+import org.apache.tez.runtime.library.shuffle.impl.ShuffleUserPayloads.DataMovementEventPayloadProto;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+
+public class OnFileUnorderedKVOutput implements LogicalOutput {
+
+  private TezOutputContext outputContext;
+  private FileBasedKVWriter kvWriter;
+
+  public OnFileUnorderedKVOutput() {
+  }
+
+  @Override
+  public List<Event> initialize(TezOutputContext outputContext)
+      throws Exception {
+    this.outputContext = outputContext;
+    this.kvWriter = new FileBasedKVWriter(outputContext);
+    return Collections.emptyList();
+  }
+
+  @Override
+  public KVWriter getWriter() throws Exception {
+    return kvWriter;
+  }
+
+  @Override
+  public void handleEvents(List<Event> outputEvents) {
+    throw new TezUncheckedException("Not expecting any events");
+  }
+
+  @Override
+  public List<Event> close() throws Exception {
+    boolean outputGenerated = this.kvWriter.close();
+    DataMovementEventPayloadProto.Builder payloadBuilder = DataMovementEventPayloadProto
+        .newBuilder();
+
+    String host = System.getenv(ApplicationConstants.Environment.NM_HOST
+        .toString());
+    ByteBuffer shuffleMetadata = outputContext
+        .getServiceProviderMetaData(ShuffleUtils.SHUFFLE_HANDLER_SERVICE_ID);
+    int shufflePort = ShuffleUtils
+        .deserializeShuffleProviderMetaData(shuffleMetadata);
+    payloadBuilder.setOutputGenerated(outputGenerated);
+    if (outputGenerated) {
+      payloadBuilder.setHost(host);
+      payloadBuilder.setPort(shufflePort);
+      payloadBuilder.setPathComponent(outputContext.getUniqueIdentifier());
+    }
+    DataMovementEventPayloadProto payloadProto = payloadBuilder.build();
+
+    DataMovementEvent dmEvent = new DataMovementEvent(0,
+        payloadProto.toByteArray());
+    List<Event> events = Lists.newArrayListWithCapacity(1);
+    events.add(dmEvent);
+    return events;
+  }
+
+  @Override
+  public void setNumPhysicalOutputs(int numOutputs) {
+    Preconditions.checkArgument(numOutputs == 1,
+        "Number of outputs can only be 1 for " + this.getClass().getName());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/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
new file mode 100644
index 0000000..a98ce63
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/DiskFetchedInput.java
@@ -0,0 +1,111 @@
+/**
+ * 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.library.shuffle.common;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+
+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.tez.runtime.library.common.InputAttemptIdentifier;
+import org.apache.tez.runtime.library.common.task.local.output.TezTaskOutputFiles;
+
+import com.google.common.base.Preconditions;
+
+public class DiskFetchedInput extends FetchedInput {
+
+  private static final Log LOG = LogFactory.getLog(DiskFetchedInput.class);
+  
+  private final FileSystem localFS;
+  private final Path tmpOutputPath;
+  private final Path outputPath;
+
+  public DiskFetchedInput(long size,
+      InputAttemptIdentifier inputAttemptIdentifier,
+      FetchedInputCallback callbackHandler, Configuration conf,
+      LocalDirAllocator localDirAllocator, TezTaskOutputFiles filenameAllocator)
+      throws IOException {
+    super(Type.DISK, size, inputAttemptIdentifier, callbackHandler);
+
+    this.localFS = FileSystem.getLocal(conf);
+    this.outputPath = filenameAllocator.getInputFileForWrite(
+        this.inputAttemptIdentifier.getInputIdentifier().getSrcTaskIndex(), size);
+    this.tmpOutputPath = outputPath.suffix(String.valueOf(id));
+  }
+
+  @Override
+  public OutputStream getOutputStream() throws IOException {
+    return localFS.create(tmpOutputPath);
+  }
+
+  @Override
+  public InputStream getInputStream() throws IOException {
+    return localFS.open(outputPath);
+  }
+
+  @Override
+  public void commit() throws IOException {
+    if (state == State.PENDING) {
+      state = State.COMMITTED;
+      localFS.rename(tmpOutputPath, outputPath);
+      notifyFetchComplete();
+    }
+  }
+
+  @Override
+  public void abort() throws IOException {
+    if (state == State.PENDING) {
+      state = State.ABORTED;
+      // TODO NEWTEZ Maybe defer this to container cleanup
+      localFS.delete(tmpOutputPath, false);
+      notifyFetchFailure();
+    }
+  }
+  
+  @Override
+  public void free() {
+    Preconditions.checkState(
+        state == State.COMMITTED || state == State.ABORTED,
+        "FetchedInput can only be freed after it is committed or aborted");
+    if (state == State.COMMITTED) {
+      state = State.FREED;
+      try {
+        // TODO NEWTEZ Maybe defer this to container cleanup
+        localFS.delete(outputPath, false);
+      } catch (IOException e) {
+        // Ignoring the exception, will eventually be cleaned by container
+        // cleanup.
+        LOG.warn("Failed to remvoe file : " + outputPath.toString());
+      }
+      notifyFreedResource();
+    }
+  }
+
+  @Override
+  public String toString() {
+    return "DiskFetchedInput [outputPath=" + outputPath
+        + ", inputAttemptIdentifier=" + inputAttemptIdentifier + ", size="
+        + size + ", type=" + type + ", id=" + id + ", state=" + state + "]";
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/FetchResult.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/FetchResult.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/FetchResult.java
new file mode 100644
index 0000000..df38b07
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/FetchResult.java
@@ -0,0 +1,70 @@
+/**
+ * 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.library.shuffle.common;
+/**
+ * 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.
+ */
+
+import org.apache.tez.runtime.library.common.InputAttemptIdentifier;
+
+public class FetchResult {
+
+  private final String host;
+  private final int port;
+  private final int partition;
+  private final Iterable<InputAttemptIdentifier> pendingInputs;
+
+  public FetchResult(String host, int port, int partition,
+      Iterable<InputAttemptIdentifier> pendingInputs) {
+    this.host = host;
+    this.port = port;
+    this.partition = partition;
+    this.pendingInputs = pendingInputs;
+  }
+
+  public String getHost() {
+    return host;
+  }
+
+  public int getPort() {
+    return port;
+  }
+
+  public int getPartition() {
+    return partition;
+  }
+
+  public Iterable<InputAttemptIdentifier> getPendingInputs() {
+    return pendingInputs;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/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
new file mode 100644
index 0000000..8f3c407
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/FetchedInput.java
@@ -0,0 +1,144 @@
+/**
+ * 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.library.shuffle.common;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.tez.runtime.library.common.InputAttemptIdentifier;
+
+@Private
+public abstract class FetchedInput {
+  
+  public static enum Type {
+    WAIT, // TODO NEWTEZ Implement this, only if required.
+    MEMORY,
+    DISK,
+  }
+  
+  protected static enum State {
+    PENDING, COMMITTED, ABORTED, FREED
+  }
+
+  private static AtomicInteger ID_GEN = new AtomicInteger(0);
+
+  protected InputAttemptIdentifier inputAttemptIdentifier;
+  protected final long size;
+  protected final Type type;
+  protected final FetchedInputCallback callback;
+  protected final int id;
+  protected State state;
+
+  public FetchedInput(Type type, long size,
+      InputAttemptIdentifier inputAttemptIdentifier,
+      FetchedInputCallback callbackHandler) {
+    this.type = type;
+    this.size = size;
+    this.inputAttemptIdentifier = inputAttemptIdentifier;
+    this.callback = callbackHandler;
+    this.id = ID_GEN.getAndIncrement();
+    this.state = State.PENDING;
+  }
+
+  public Type getType() {
+    return this.type;
+  }
+
+  public long getSize() {
+    return this.size;
+  }
+
+  public InputAttemptIdentifier getInputAttemptIdentifier() {
+    return this.inputAttemptIdentifier;
+  }
+
+  /**
+   * Inform the Allocator about a committed resource.
+   * This should be called by commit
+   */
+  public void notifyFetchComplete() {
+    this.callback.fetchComplete(this);
+  }
+  
+  /**
+   * Inform the Allocator about a failed resource.
+   * This should be called by abort
+   */
+  public void notifyFetchFailure() {
+    this.callback.fetchFailed(this);
+  }
+  
+  /**
+   * Inform the Allocator about a completed resource being released.
+   * This should be called by free
+   */
+  public void notifyFreedResource() {
+    this.callback.freeResources(this);
+  }
+  
+  /**
+   * Returns the output stream to be used to write fetched data. Users are
+   * expected to close the OutputStream when they're done
+   */
+  public abstract OutputStream getOutputStream() throws IOException;
+
+  /**
+   * Return an input stream to be used to read the previously fetched data.
+   * Users are expected to close the InputStream when they're done
+   */
+  public abstract InputStream getInputStream() throws IOException;
+
+  /**
+   * Commit the output. Should be idempotent
+   */
+  public abstract void commit() throws IOException;
+
+  /**
+   * Abort the output. Should be idempotent
+   */
+  public abstract void abort() throws IOException;
+
+  /**
+   * Called when this input has been consumed, so that resources can be
+   * reclaimed.
+   */
+  public abstract void free();
+  
+  @Override
+  public int hashCode() {
+    return id;
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (this == obj)
+      return true;
+    if (obj == null)
+      return false;
+    if (getClass() != obj.getClass())
+      return false;
+    FetchedInput other = (FetchedInput) obj;
+    if (id != other.id)
+      return false;
+    return true;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/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
new file mode 100644
index 0000000..1d60b68
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/FetchedInputAllocator.java
@@ -0,0 +1,31 @@
+/**
+ * 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.library.shuffle.common;
+
+import java.io.IOException;
+
+import org.apache.tez.runtime.library.common.InputAttemptIdentifier;
+
+public interface FetchedInputAllocator {
+
+  public FetchedInput allocate(long size,
+      InputAttemptIdentifier inputAttemptIdentifier) throws IOException;
+  
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/FetchedInputCallback.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/FetchedInputCallback.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/FetchedInputCallback.java
new file mode 100644
index 0000000..a22ce45
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/FetchedInputCallback.java
@@ -0,0 +1,29 @@
+/**
+ * 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.library.shuffle.common;
+
+public interface FetchedInputCallback {
+  
+  public void fetchComplete(FetchedInput fetchedInput);
+  
+  public void fetchFailed(FetchedInput fetchedInput);
+  
+  public void freeResources(FetchedInput fetchedInput);
+  
+}