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 2015/05/06 09:41:03 UTC

[01/50] [abbrv] tez git commit: TEZ-2363: Fix off-by-one error in REDUCE_INPUT_RECORDS counter (gopalv)

Repository: tez
Updated Branches:
  refs/heads/TEZ-2003 437b57152 -> 9338abfe9 (forced update)


TEZ-2363: Fix off-by-one error in REDUCE_INPUT_RECORDS counter (gopalv)


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

Branch: refs/heads/TEZ-2003
Commit: e36f962e78f301974cd8eed2380a6a5bd26a49ae
Parents: 4ae87f0
Author: Gopal V <go...@apache.org>
Authored: Thu Apr 30 20:40:49 2015 +0530
Committer: Gopal V <go...@apache.org>
Committed: Thu Apr 30 20:40:49 2015 +0530

----------------------------------------------------------------------
 .../runtime/library/common/ValuesIterator.java  | 19 ++---
 .../library/common/TestValuesIterator.java      | 73 +++++++++++++++++++-
 2 files changed, 81 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/e36f962e/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
index 0f1bc5b..a1f52e7 100644
--- 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
@@ -95,14 +95,13 @@ public class ValuesIterator<KEY,VALUE> {
       readNextKey();
       key = nextKey;
       nextKey = null;
-      hasMoreValues = more;
       isFirstRecord = false;
     } else {
       nextKey();
     }
     return more;
   }
-  
+
   /** The current key. */
   public KEY getKey() { 
     return key; 
@@ -162,11 +161,7 @@ public class ValuesIterator<KEY,VALUE> {
     while (hasMoreValues) { 
       readNextKey();
     }
-    if (more) {
-      inputKeyCounter.increment(1);
-      ++keyCtr;
-    }
-    
+
     // move the next key to the current one
     KEY tmpKey = key;
     key = nextKey;
@@ -185,8 +180,14 @@ public class ValuesIterator<KEY,VALUE> {
         keyIn.reset(nextKeyBytes.getData(), nextKeyBytes.getPosition(),
             nextKeyBytes.getLength() - nextKeyBytes.getPosition());
         nextKey = keyDeserializer.deserialize(nextKey);
-        // TODO Is a counter increment required here ?
-        hasMoreValues = key != null && (comparator.compare(key, nextKey) == 0);
+        // hasMoreValues = is it first key or is key the same?
+        hasMoreValues = (key == null) || (comparator.compare(key, nextKey) == 0);
+        if (key == null || false == hasMoreValues) {
+          // invariant: more=true & there are no more values in an existing key group
+          // so this indicates start of new key group
+          inputKeyCounter.increment(1);
+          ++keyCtr;
+        }
       } else {
         hasMoreValues = in.isSameKey();
       }

http://git-wip-us.apache.org/repos/asf/tez/blob/e36f962e/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/TestValuesIterator.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/TestValuesIterator.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/TestValuesIterator.java
index c483a81..e1718c8 100644
--- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/TestValuesIterator.java
+++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/TestValuesIterator.java
@@ -64,6 +64,7 @@ import java.util.TreeMap;
 
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.assertEquals;
 import static org.mockito.Mockito.doReturn;
 import static org.mockito.Mockito.mock;
 
@@ -179,6 +180,32 @@ public class TestValuesIterator {
   }
 
   @Test(timeout = 20000)
+  public void testCountedIteratorWithInmemoryReader() throws IOException {
+    verifyCountedIteratorReader(true);
+  }
+
+  @Test(timeout = 20000)
+  public void testCountedIteratorWithIFileReader() throws IOException {
+    verifyCountedIteratorReader(false);
+  }
+
+  private void verifyCountedIteratorReader(boolean inMemory) throws IOException {
+    TezCounter keyCounter = new GenericCounter("inputKeyCounter", "y3");
+    TezCounter tupleCounter = new GenericCounter("inputValuesCounter", "y4");
+    ValuesIterator iterator = createCountedIterator(inMemory, keyCounter,
+        tupleCounter);
+    List<Integer> sequence = verifyIteratorData(iterator);
+    if (expectedTestResult) {
+      assertEquals((long) sequence.size(), keyCounter.getValue());
+      long rows = 0;
+      for (Integer i : sequence) {
+        rows += i.longValue();
+      }
+      assertEquals(rows, tupleCounter.getValue());
+    }
+  }
+
+  @Test(timeout = 20000)
   public void testIteratorWithIFileReaderEmptyPartitions() throws IOException {
     ValuesIterator iterator = createEmptyIterator(false);
     assert(iterator.moveToNext() == false);
@@ -212,13 +239,19 @@ public class TestValuesIterator {
 
   /**
    * Tests whether data in valuesIterator matches with sorted input data set.
-   *
+   * 
+   * Returns a list of value counts for each key.
+   * 
    * @param valuesIterator
+   * @return List
    * @throws IOException
    */
-  private void verifyIteratorData(ValuesIterator valuesIterator) throws IOException {
+  private List<Integer> verifyIteratorData(
+      ValuesIterator valuesIterator) throws IOException {
     boolean result = true;
 
+    ArrayList<Integer> sequence = new ArrayList<Integer>();
+
     //sort original data based on comparator
     ListMultimap<Writable, Writable> sortedMap =
         new ImmutableListMultimap.Builder<Writable, Writable>()
@@ -240,6 +273,7 @@ public class TestValuesIterator {
         break;
       }
 
+      int valueCount = 0;
       //Verify values
       Iterator<Writable> vItr = valuesIterator.getValues().iterator();
       for (Writable val : sortedMap.get(oriKey)) {
@@ -250,13 +284,19 @@ public class TestValuesIterator {
           result = false;
           break;
         }
+
+        valueCount++;
       }
+      sequence.add(valueCount);
+      assertTrue("At least 1 value per key", valueCount > 0);
     }
     if (expectedTestResult) {
       assertTrue(result);
     } else {
       assertFalse(result);
     }
+
+    return sequence;
   }
 
   /**
@@ -287,6 +327,35 @@ public class TestValuesIterator {
         (TezCounter) new GenericCounter("inputValueCounter", "y4"));
   }
 
+  /**
+   * Create sample data (in memory), with an attached counter  and return ValuesIterator
+   *
+   * @param inMemory
+   * @param keyCounter
+   * @param tupleCounter
+   * @return ValuesIterator
+   * @throws IOException
+   */
+  private ValuesIterator createCountedIterator(boolean inMemory, TezCounter keyCounter, TezCounter tupleCounter) throws IOException {
+    if (!inMemory) {
+      streamPaths = createFiles();
+      //Merge all files to get KeyValueIterator
+      rawKeyValueIterator =
+          TezMerger.merge(conf, fs, keyClass, valClass, null,
+              false, -1, 1024, streamPaths, false, mergeFactor, tmpDir, comparator,
+              new ProgressReporter(), null, null, null, null);
+    } else {
+      List<TezMerger.Segment> segments = createInMemStreams();
+      rawKeyValueIterator =
+          TezMerger.merge(conf, fs, keyClass, valClass, segments, mergeFactor, tmpDir,
+              comparator, new ProgressReporter(), new GenericCounter("readsCounter", "y"),
+              new GenericCounter("writesCounter", "y1"),
+              new GenericCounter("bytesReadCounter", "y2"), new Progress());
+    }
+    return new ValuesIterator(rawKeyValueIterator, comparator,
+        keyClass, valClass, conf, keyCounter, tupleCounter);
+  }
+
   @Parameterized.Parameters(name = "test[{0}, {1}, {2}, {3} {4} {5} {6}]")
   public static Collection<Object[]> getParameters() {
     Collection<Object[]> parameters = new ArrayList<Object[]>();


[42/50] [abbrv] tez git commit: TEZ-2302. Allow TaskCommunicators to subscribe for Vertex updates. (sseth)

Posted by ss...@apache.org.
TEZ-2302. Allow TaskCommunicators to subscribe for Vertex updates. (sseth)


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

Branch: refs/heads/TEZ-2003
Commit: 3e03847cadd5135f05992a86a202e2bff41fe76e
Parents: 2f8a273
Author: Siddharth Seth <ss...@apache.org>
Authored: Thu Apr 9 13:33:48 2015 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Wed May 6 00:18:08 2015 -0700

----------------------------------------------------------------------
 TEZ-2003-CHANGES.txt                            |   1 +
 .../apache/tez/dag/api/TaskCommunicator.java    |  20 +++
 .../tez/dag/api/TaskCommunicatorContext.java    |  14 ++-
 .../dag/app/TaskAttemptListenerImpTezDag.java   |  52 +++-----
 .../dag/app/TaskCommunicatorContextImpl.java    | 124 +++++++++++++++++++
 .../tez/dag/app/TezTaskCommunicatorImpl.java    |   6 +
 .../java/org/apache/tez/dag/app/dag/DAG.java    |   2 +
 .../apache/tez/dag/app/dag/impl/DAGImpl.java    |   5 +
 8 files changed, 188 insertions(+), 36 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/3e03847c/TEZ-2003-CHANGES.txt
----------------------------------------------------------------------
diff --git a/TEZ-2003-CHANGES.txt b/TEZ-2003-CHANGES.txt
index 9d6b220..ca5225e 100644
--- a/TEZ-2003-CHANGES.txt
+++ b/TEZ-2003-CHANGES.txt
@@ -15,5 +15,6 @@ ALL CHANGES:
   TEZ-2283. Fixes after rebase 04/07.
   TEZ-2284. Separate TaskReporter into an interface.
   TEZ-2285. Allow TaskCommunicators to indicate task/container liveness.
+  TEZ-2302. Allow TaskCommunicators to subscribe for Vertex updates.
 
 INCOMPATIBLE CHANGES:

http://git-wip-us.apache.org/repos/asf/tez/blob/3e03847c/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicator.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicator.java b/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicator.java
index 945091e..a2cd858 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicator.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicator.java
@@ -22,6 +22,7 @@ import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.service.AbstractService;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.LocalResource;
+import org.apache.tez.dag.api.event.VertexStateUpdate;
 import org.apache.tez.dag.records.TezTaskAttemptID;
 import org.apache.tez.runtime.api.impl.TaskSpec;
 
@@ -54,4 +55,23 @@ public abstract class TaskCommunicator extends AbstractService {
   public abstract InetSocketAddress getAddress();
 
   // TODO Eventually. Add methods here to support preemption of tasks.
+
+  /**
+   * Receive notifications on vertex state changes.
+   * <p/>
+   * State changes will be received based on the registration via {@link
+   * org.apache.tez.runtime.api.InputInitializerContext#registerForVertexStateUpdates(String,
+   * java.util.Set)}. Notifications will be received for all registered state changes, and not just
+   * for the latest state update. They will be in order in which the state change occurred. </p>
+   *
+   * Extensive processing should not be performed via this method call. Instead this should just be
+   * used as a notification mechanism.
+   * <br>This method may be invoked concurrently with other invocations into the TaskCommunicator and
+   * multi-threading/concurrency implications must be considered.
+   * @param stateUpdate an event indicating the name of the vertex, and it's updated state.
+   *                    Additional information may be available for specific events, Look at the
+   *                    type hierarchy for {@link org.apache.tez.dag.api.event.VertexStateUpdate}
+   * @throws Exception
+   */
+  public abstract void onVertexStateUpdated(VertexStateUpdate stateUpdate) throws Exception;
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/3e03847c/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicatorContext.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicatorContext.java b/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicatorContext.java
index 0c3bac3..19caed9 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicatorContext.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicatorContext.java
@@ -16,10 +16,12 @@ package org.apache.tez.dag.api;
 
 import javax.annotation.Nullable;
 import java.io.IOException;
+import java.util.Set;
 
 import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.tez.dag.api.event.VertexState;
 import org.apache.tez.dag.records.TezTaskAttemptID;
 
 
@@ -48,7 +50,7 @@ public interface TaskCommunicatorContext {
   void containerAlive(ContainerId containerId);
 
   // TODO TEZ-2003 Move to vertex, taskIndex, version. Rename to taskAttempt*
-  void taskStartedRemotely(TezTaskAttemptID taskAttemptID, ContainerId containerId);
+  void taskStartedRemotely(TezTaskAttemptID taskAttemptId, ContainerId containerId);
 
   // TODO TEZ-2003 Move to vertex, taskIndex, version. Rename to taskAttempt*
   void taskKilled(TezTaskAttemptID taskAttemptId, TaskAttemptEndReason taskAttemptEndReason, @Nullable String diagnostics);
@@ -56,6 +58,16 @@ public interface TaskCommunicatorContext {
   // TODO TEZ-2003 Move to vertex, taskIndex, version. Rename to taskAttempt*
   void taskFailed(TezTaskAttemptID taskAttemptId, TaskAttemptEndReason taskAttemptEndReason, @Nullable String diagnostics);
 
+  /**
+   * Register to get notifications on updates to the specified vertex. Notifications will be sent
+   * via {@link org.apache.tez.runtime.api.InputInitializer#onVertexStateUpdated(org.apache.tez.dag.api.event.VertexStateUpdate)} </p>
+   *
+   * This method can only be invoked once. Duplicate invocations will result in an error.
+   *
+   * @param vertexName the vertex name for which notifications are required.
+   * @param stateSet   the set of states for which notifications are required. null implies all
+   */
+  void registerForVertexStateUpdates(String vertexName, @Nullable Set<VertexState> stateSet);
   // TODO TEZ-2003 API. Should a method exist for task succeeded.
 
   // TODO Eventually Add methods to report availability stats to the scheduler.

http://git-wip-us.apache.org/repos/asf/tez/blob/3e03847c/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 a6ccbfa..a7bbba9 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
@@ -17,6 +17,7 @@
 
 package org.apache.tez.dag.app;
 
+import javax.annotation.Nullable;
 import java.io.IOException;
 import java.lang.reflect.Constructor;
 import java.lang.reflect.InvocationTargetException;
@@ -26,6 +27,7 @@ import java.net.UnknownHostException;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
+import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 
@@ -35,13 +37,13 @@ import org.apache.tez.dag.app.dag.event.TaskAttemptEventStatusUpdate;
 import org.apache.tez.dag.app.dag.event.TaskAttemptEventType;
 import org.apache.tez.runtime.api.events.TaskStatusUpdateEvent;
 import org.apache.tez.runtime.api.impl.EventType;
+import com.google.common.base.Preconditions;
+import org.apache.tez.dag.api.event.VertexState;
+import org.apache.tez.dag.api.event.VertexStateUpdate;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import com.google.common.annotations.VisibleForTesting;
 import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.service.AbstractService;
-import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.tez.common.ReflectionUtils;
 import org.apache.tez.common.TezUtilsInternal;
@@ -67,14 +69,12 @@ import org.apache.tez.dag.records.TezTaskAttemptID;
 import org.apache.tez.dag.records.TezVertexID;
 import org.apache.tez.runtime.api.impl.TezEvent;
 import org.apache.tez.common.security.JobTokenSecretManager;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 
 @SuppressWarnings("unchecked")
 @InterfaceAudience.Private
 public class TaskAttemptListenerImpTezDag extends AbstractService implements
-    TaskAttemptListener, TaskCommunicatorContext {
+    TaskAttemptListener {
 
   private static final Logger LOG = LoggerFactory
       .getLogger(TaskAttemptListenerImpTezDag.class);
@@ -124,7 +124,7 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
     }
     this.taskCommunicators = new TaskCommunicator[taskCommunicatorClassIdentifiers.length];
     for (int i = 0 ; i < taskCommunicatorClassIdentifiers.length ; i++) {
-      taskCommunicators[i] = createTaskCommunicator(taskCommunicatorClassIdentifiers[i]);
+      taskCommunicators[i] = createTaskCommunicator(taskCommunicatorClassIdentifiers[i], i);
     }
     // TODO TEZ-2118 Start using taskCommunicator indices properly
   }
@@ -145,13 +145,13 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
     }
   }
 
-  private TaskCommunicator createTaskCommunicator(String taskCommClassIdentifier) {
+  private TaskCommunicator createTaskCommunicator(String taskCommClassIdentifier, int taskCommIndex) {
     if (taskCommClassIdentifier.equals(TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT)) {
       LOG.info("Using Default Task Communicator");
-      return new TezTaskCommunicatorImpl(this);
+      return new TezTaskCommunicatorImpl(new TaskCommunicatorContextImpl(context, this, taskCommIndex));
     } else if (taskCommClassIdentifier.equals(TezConstants.TEZ_AM_SERVICE_PLUGINS_LOCAL_MODE_NAME_DEFAULT)) {
       LOG.info("Using Default Local Task Communicator");
-      return new TezLocalTaskCommunicatorImpl(this);
+      return new TezLocalTaskCommunicatorImpl(new TaskCommunicatorContextImpl(context, this, taskCommIndex));
     } else {
       LOG.info("Using TaskCommunicator: " + taskCommClassIdentifier);
       Class<? extends TaskCommunicator> taskCommClazz = (Class<? extends TaskCommunicator>) ReflectionUtils
@@ -159,7 +159,7 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
       try {
         Constructor<? extends TaskCommunicator> ctor = taskCommClazz.getConstructor(TaskCommunicatorContext.class);
         ctor.setAccessible(true);
-        return ctor.newInstance(this);
+        return ctor.newInstance(new TaskCommunicatorContextImpl(context, this, taskCommIndex));
       } catch (NoSuchMethodException e) {
         throw new TezUncheckedException(e);
       } catch (InvocationTargetException e) {
@@ -171,18 +171,6 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
       }
     }
   }
-
-  @Override
-  public ApplicationAttemptId getApplicationAttemptId() {
-    return context.getApplicationAttemptId();
-  }
-
-  @Override
-  public Credentials getCredentials() {
-    return context.getAppCredentials();
-  }
-
-  @Override
   public TaskHeartbeatResponse heartbeat(TaskHeartbeatRequest request)
       throws IOException, TezException {
     ContainerId containerId = ConverterUtils.toContainerId(request
@@ -252,30 +240,20 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
     }
     return new TaskHeartbeatResponse(false, outEvents);
   }
-
-  @Override
-  public boolean isKnownContainer(ContainerId containerId) {
-    return context.getAllContainers().get(containerId) != null;
-  }
-
-  @Override
   public void taskAlive(TezTaskAttemptID taskAttemptId) {
     taskHeartbeatHandler.pinged(taskAttemptId);
   }
 
-  @Override
   public void containerAlive(ContainerId containerId) {
     pingContainerHeartbeatHandler(containerId);
   }
 
-  @Override
   public void taskStartedRemotely(TezTaskAttemptID taskAttemptID, ContainerId containerId) {
     context.getEventHandler()
         .handle(new TaskAttemptEventStartedRemotely(taskAttemptID, containerId, null));
     pingContainerHeartbeatHandler(containerId);
   }
 
-  @Override
   public void taskKilled(TezTaskAttemptID taskAttemptId, TaskAttemptEndReason taskAttemptEndReason,
                          String diagnostics) {
     // Regular flow via TaskAttempt will take care of un-registering from the heartbeat handler,
@@ -288,7 +266,6 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
         taskAttemptEndReason)));
   }
 
-  @Override
   public void taskFailed(TezTaskAttemptID taskAttemptId, TaskAttemptEndReason taskAttemptEndReason,
                          String diagnostics) {
     // Regular flow via TaskAttempt will take care of un-registering from the heartbeat handler,
@@ -301,6 +278,11 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
         taskAttemptEndReason)));
   }
 
+  public void vertexStateUpdateNotificationReceived(VertexStateUpdate event, int taskCommIndex) throws
+      Exception {
+    taskCommunicators[taskCommIndex].onVertexStateUpdated(event);
+  }
+
 
   /**
    * Child checking whether it can commit.
@@ -310,7 +292,7 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
    * {@link Task#canCommit(TezTaskAttemptID)} This is * a legacy from the
    * centralized commit protocol handling by the JobTracker.
    */
-  @Override
+//  @Override
   public boolean canCommit(TezTaskAttemptID taskAttemptId) throws IOException {
     LOG.info("Commit go/no-go request from " + taskAttemptId.toString());
     // An attempt is asking if it can commit its output. This can be decided

http://git-wip-us.apache.org/repos/asf/tez/blob/3e03847c/tez-dag/src/main/java/org/apache/tez/dag/app/TaskCommunicatorContextImpl.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/TaskCommunicatorContextImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/TaskCommunicatorContextImpl.java
new file mode 100644
index 0000000..3714c3c
--- /dev/null
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/TaskCommunicatorContextImpl.java
@@ -0,0 +1,124 @@
+/*
+ * 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.
+ */
+
+package org.apache.tez.dag.app;
+
+import javax.annotation.Nullable;
+import java.io.IOException;
+import java.util.Set;
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.security.Credentials;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.tez.dag.api.TaskAttemptEndReason;
+import org.apache.tez.dag.api.TaskCommunicatorContext;
+import org.apache.tez.dag.api.TaskHeartbeatRequest;
+import org.apache.tez.dag.api.TaskHeartbeatResponse;
+import org.apache.tez.dag.api.TezException;
+import org.apache.tez.dag.api.TezUncheckedException;
+import org.apache.tez.dag.api.event.VertexState;
+import org.apache.tez.dag.api.event.VertexStateUpdate;
+import org.apache.tez.dag.app.dag.VertexStateUpdateListener;
+import org.apache.tez.dag.records.TezTaskAttemptID;
+
+@InterfaceAudience.Private
+public class TaskCommunicatorContextImpl implements TaskCommunicatorContext, VertexStateUpdateListener {
+
+
+  private final AppContext context;
+  private final TaskAttemptListenerImpTezDag taskAttemptListener;
+  private final int taskCommunicatorIndex;
+
+  public TaskCommunicatorContextImpl(AppContext appContext,
+                                     TaskAttemptListenerImpTezDag taskAttemptListener,
+                                     int taskCommunicatorIndex) {
+    this.context = appContext;
+    this.taskAttemptListener = taskAttemptListener;
+    this.taskCommunicatorIndex = taskCommunicatorIndex;
+  }
+
+  @Override
+  public ApplicationAttemptId getApplicationAttemptId() {
+    return context.getApplicationAttemptId();
+  }
+
+  @Override
+  public Credentials getCredentials() {
+    return context.getAppCredentials();
+  }
+
+  @Override
+  public boolean canCommit(TezTaskAttemptID taskAttemptId) throws IOException {
+    return taskAttemptListener.canCommit(taskAttemptId);
+  }
+
+  @Override
+  public TaskHeartbeatResponse heartbeat(TaskHeartbeatRequest request) throws IOException,
+      TezException {
+    return taskAttemptListener.heartbeat(request);
+  }
+
+  @Override
+  public boolean isKnownContainer(ContainerId containerId) {
+    return context.getAllContainers().get(containerId) != null;
+  }
+
+  @Override
+  public void taskAlive(TezTaskAttemptID taskAttemptId) {
+    taskAttemptListener.taskAlive(taskAttemptId);
+  }
+
+  @Override
+  public void containerAlive(ContainerId containerId) {
+    taskAttemptListener.containerAlive(containerId);
+  }
+
+  @Override
+  public void taskStartedRemotely(TezTaskAttemptID taskAttemptId, ContainerId containerId) {
+    taskAttemptListener.taskStartedRemotely(taskAttemptId, containerId);
+  }
+
+  @Override
+  public void taskKilled(TezTaskAttemptID taskAttemptId, TaskAttemptEndReason taskAttemptEndReason,
+                         @Nullable String diagnostics) {
+    taskAttemptListener.taskKilled(taskAttemptId, taskAttemptEndReason, diagnostics);
+  }
+
+  @Override
+  public void taskFailed(TezTaskAttemptID taskAttemptId, TaskAttemptEndReason taskAttemptEndReason,
+                         @Nullable String diagnostics) {
+    taskAttemptListener.taskFailed(taskAttemptId, taskAttemptEndReason, diagnostics);
+
+  }
+
+  @Override
+  public void registerForVertexStateUpdates(String vertexName,
+                                            @Nullable Set<VertexState> stateSet) {
+    Preconditions.checkNotNull(vertexName, "VertexName cannot be null: " + vertexName);
+    context.getCurrentDAG().getStateChangeNotifier().registerForVertexUpdates(vertexName, stateSet, this);
+  }
+
+
+  @Override
+  public void onStateUpdated(VertexStateUpdate event) {
+    try {
+      taskAttemptListener.vertexStateUpdateNotificationReceived(event, taskCommunicatorIndex);
+    } catch (Exception e) {
+      // TODO TEZ-2003 This needs to be propagated to the DAG as a user error.
+      throw new TezUncheckedException(e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/tez/blob/3e03847c/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java
index ef4f764..1417a3b 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java
@@ -48,6 +48,7 @@ import org.apache.tez.dag.api.TaskHeartbeatResponse;
 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.event.VertexStateUpdate;
 import org.apache.tez.dag.app.security.authorize.TezAMPolicyProvider;
 import org.apache.tez.dag.records.TezTaskAttemptID;
 import org.apache.tez.runtime.api.impl.TaskSpec;
@@ -252,6 +253,11 @@ public class TezTaskCommunicatorImpl extends TaskCommunicator {
     return address;
   }
 
+  @Override
+  public void onVertexStateUpdated(VertexStateUpdate stateUpdate) throws Exception {
+    // Empty. Not registering, or expecting any updates.
+  }
+
   protected String getTokenIdentifier() {
     return tokenIdentifier;
   }

http://git-wip-us.apache.org/repos/asf/tez/blob/3e03847c/tez-dag/src/main/java/org/apache/tez/dag/app/dag/DAG.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/DAG.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/DAG.java
index 4c3426a..6d6872b 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/DAG.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/DAG.java
@@ -94,4 +94,6 @@ public interface DAG {
 
   Map<String, TezVertexID> getVertexNameIDMapping();
 
+  StateChangeNotifier getStateChangeNotifier();
+
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/3e03847c/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 f769565..998108b 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
@@ -685,6 +685,11 @@ public class DAGImpl implements org.apache.tez.dag.app.dag.DAG,
   }
 
   @Override
+  public StateChangeNotifier getStateChangeNotifier() {
+    return entityUpdateTracker;
+  }
+
+  @Override
   public TezCounters getAllCounters() {
 
     readLock.lock();


[25/50] [abbrv] tez git commit: TEZ-2422. Update version on master to 0.8.0. (sseth)

Posted by ss...@apache.org.
TEZ-2422. Update version on master to 0.8.0. (sseth)


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

Branch: refs/heads/TEZ-2003
Commit: 5679b28386a957879eb0443557b0a48c7c95c4d4
Parents: 406bba9
Author: Siddharth Seth <ss...@apache.org>
Authored: Tue May 5 18:25:22 2015 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Tue May 5 18:25:22 2015 -0700

----------------------------------------------------------------------
 CHANGES.txt                                             | 6 ++++++
 docs/pom.xml                                            | 2 +-
 pom.xml                                                 | 2 +-
 tez-api/pom.xml                                         | 2 +-
 tez-common/pom.xml                                      | 2 +-
 tez-dag/pom.xml                                         | 2 +-
 tez-dist/pom.xml                                        | 2 +-
 tez-examples/pom.xml                                    | 2 +-
 tez-mapreduce/pom.xml                                   | 2 +-
 tez-plugins/pom.xml                                     | 2 +-
 tez-plugins/tez-mbeans-resource-calculator/pom.xml      | 2 +-
 tez-plugins/tez-yarn-timeline-history-with-acls/pom.xml | 2 +-
 tez-plugins/tez-yarn-timeline-history/pom.xml           | 2 +-
 tez-runtime-internals/pom.xml                           | 2 +-
 tez-runtime-library/pom.xml                             | 2 +-
 tez-tests/pom.xml                                       | 2 +-
 tez-tools/tez-tfile-parser/pom.xml                      | 4 ++--
 tez-ui/pom.xml                                          | 2 +-
 18 files changed, 24 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/5679b283/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 8633419..2446f6a 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -2,6 +2,12 @@ Apache Tez Change Log
 =====================
 
 
+Release 0.8.0: Unreleased
+
+INCOMPATIBLE CHANGES
+
+ALL CHANGES:
+
 Release 0.7.0: Unreleased
 
 INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/tez/blob/5679b283/docs/pom.xml
----------------------------------------------------------------------
diff --git a/docs/pom.xml b/docs/pom.xml
index 91e827e..a475c58 100644
--- a/docs/pom.xml
+++ b/docs/pom.xml
@@ -27,7 +27,7 @@
     <parent>
       <groupId>org.apache.tez</groupId>
       <artifactId>tez</artifactId>
-      <version>0.7.0-SNAPSHOT</version>
+      <version>0.8.0-SNAPSHOT</version>
     </parent>
     <artifactId>tez-docs</artifactId>
     <packaging>pom</packaging>

http://git-wip-us.apache.org/repos/asf/tez/blob/5679b283/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 1e29c4d..419138b 100644
--- a/pom.xml
+++ b/pom.xml
@@ -19,7 +19,7 @@
   <groupId>org.apache.tez</groupId>
   <artifactId>tez</artifactId>
   <packaging>pom</packaging>
-  <version>0.7.0-SNAPSHOT</version>
+  <version>0.8.0-SNAPSHOT</version>
   <name>tez</name>
 
   <licenses>

http://git-wip-us.apache.org/repos/asf/tez/blob/5679b283/tez-api/pom.xml
----------------------------------------------------------------------
diff --git a/tez-api/pom.xml b/tez-api/pom.xml
index e726135..0a5d570 100644
--- a/tez-api/pom.xml
+++ b/tez-api/pom.xml
@@ -20,7 +20,7 @@
   <parent>
     <groupId>org.apache.tez</groupId>
     <artifactId>tez</artifactId>
-    <version>0.7.0-SNAPSHOT</version>
+    <version>0.8.0-SNAPSHOT</version>
   </parent>
   <artifactId>tez-api</artifactId>
 

http://git-wip-us.apache.org/repos/asf/tez/blob/5679b283/tez-common/pom.xml
----------------------------------------------------------------------
diff --git a/tez-common/pom.xml b/tez-common/pom.xml
index 679d1c8..c6922be 100644
--- a/tez-common/pom.xml
+++ b/tez-common/pom.xml
@@ -20,7 +20,7 @@
   <parent>
     <groupId>org.apache.tez</groupId>
     <artifactId>tez</artifactId>
-    <version>0.7.0-SNAPSHOT</version>
+    <version>0.8.0-SNAPSHOT</version>
   </parent>
   <artifactId>tez-common</artifactId>
 

http://git-wip-us.apache.org/repos/asf/tez/blob/5679b283/tez-dag/pom.xml
----------------------------------------------------------------------
diff --git a/tez-dag/pom.xml b/tez-dag/pom.xml
index 97de918..cc005bb 100644
--- a/tez-dag/pom.xml
+++ b/tez-dag/pom.xml
@@ -19,7 +19,7 @@
   <parent>
     <groupId>org.apache.tez</groupId>
     <artifactId>tez</artifactId>
-    <version>0.7.0-SNAPSHOT</version>
+    <version>0.8.0-SNAPSHOT</version>
   </parent>
   <properties>
     <tez.component>tez-dag</tez.component>

http://git-wip-us.apache.org/repos/asf/tez/blob/5679b283/tez-dist/pom.xml
----------------------------------------------------------------------
diff --git a/tez-dist/pom.xml b/tez-dist/pom.xml
index cfc1d32..e78cb8e 100644
--- a/tez-dist/pom.xml
+++ b/tez-dist/pom.xml
@@ -21,7 +21,7 @@
   <parent>
     <groupId>org.apache.tez</groupId>
     <artifactId>tez</artifactId>
-    <version>0.7.0-SNAPSHOT</version>
+    <version>0.8.0-SNAPSHOT</version>
   </parent>
   <artifactId>tez-dist</artifactId>
 

http://git-wip-us.apache.org/repos/asf/tez/blob/5679b283/tez-examples/pom.xml
----------------------------------------------------------------------
diff --git a/tez-examples/pom.xml b/tez-examples/pom.xml
index f4924fb..cd2df5c 100644
--- a/tez-examples/pom.xml
+++ b/tez-examples/pom.xml
@@ -20,7 +20,7 @@
   <parent>
     <groupId>org.apache.tez</groupId>
     <artifactId>tez</artifactId>
-    <version>0.7.0-SNAPSHOT</version>
+    <version>0.8.0-SNAPSHOT</version>
   </parent>
 
   <artifactId>tez-examples</artifactId>

http://git-wip-us.apache.org/repos/asf/tez/blob/5679b283/tez-mapreduce/pom.xml
----------------------------------------------------------------------
diff --git a/tez-mapreduce/pom.xml b/tez-mapreduce/pom.xml
index f2baad0..300f781 100644
--- a/tez-mapreduce/pom.xml
+++ b/tez-mapreduce/pom.xml
@@ -20,7 +20,7 @@
   <parent>
     <groupId>org.apache.tez</groupId>
     <artifactId>tez</artifactId>
-    <version>0.7.0-SNAPSHOT</version>
+    <version>0.8.0-SNAPSHOT</version>
   </parent>
   <artifactId>tez-mapreduce</artifactId>
 

http://git-wip-us.apache.org/repos/asf/tez/blob/5679b283/tez-plugins/pom.xml
----------------------------------------------------------------------
diff --git a/tez-plugins/pom.xml b/tez-plugins/pom.xml
index 1080f2f..9b2a4cb 100644
--- a/tez-plugins/pom.xml
+++ b/tez-plugins/pom.xml
@@ -21,7 +21,7 @@
   <parent>
     <groupId>org.apache.tez</groupId>
     <artifactId>tez</artifactId>
-    <version>0.7.0-SNAPSHOT</version>
+    <version>0.8.0-SNAPSHOT</version>
   </parent>
   <artifactId>tez-plugins</artifactId>
   <packaging>pom</packaging>

http://git-wip-us.apache.org/repos/asf/tez/blob/5679b283/tez-plugins/tez-mbeans-resource-calculator/pom.xml
----------------------------------------------------------------------
diff --git a/tez-plugins/tez-mbeans-resource-calculator/pom.xml b/tez-plugins/tez-mbeans-resource-calculator/pom.xml
index 7428029..4be7f28 100644
--- a/tez-plugins/tez-mbeans-resource-calculator/pom.xml
+++ b/tez-plugins/tez-mbeans-resource-calculator/pom.xml
@@ -19,7 +19,7 @@
   <parent>
     <groupId>org.apache.tez</groupId>
     <artifactId>tez-plugins</artifactId>
-    <version>0.7.0-SNAPSHOT</version>
+    <version>0.8.0-SNAPSHOT</version>
   </parent>
   <artifactId>tez-mbeans-resource-calculator</artifactId>
 

http://git-wip-us.apache.org/repos/asf/tez/blob/5679b283/tez-plugins/tez-yarn-timeline-history-with-acls/pom.xml
----------------------------------------------------------------------
diff --git a/tez-plugins/tez-yarn-timeline-history-with-acls/pom.xml b/tez-plugins/tez-yarn-timeline-history-with-acls/pom.xml
index 6ca51b4..aeae5cf 100644
--- a/tez-plugins/tez-yarn-timeline-history-with-acls/pom.xml
+++ b/tez-plugins/tez-yarn-timeline-history-with-acls/pom.xml
@@ -19,7 +19,7 @@
   <parent>
     <groupId>org.apache.tez</groupId>
     <artifactId>tez-plugins</artifactId>
-    <version>0.7.0-SNAPSHOT</version>
+    <version>0.8.0-SNAPSHOT</version>
   </parent>
   <artifactId>tez-yarn-timeline-history-with-acls</artifactId>
 

http://git-wip-us.apache.org/repos/asf/tez/blob/5679b283/tez-plugins/tez-yarn-timeline-history/pom.xml
----------------------------------------------------------------------
diff --git a/tez-plugins/tez-yarn-timeline-history/pom.xml b/tez-plugins/tez-yarn-timeline-history/pom.xml
index 2978e88..ca76988 100644
--- a/tez-plugins/tez-yarn-timeline-history/pom.xml
+++ b/tez-plugins/tez-yarn-timeline-history/pom.xml
@@ -19,7 +19,7 @@
   <parent>
     <groupId>org.apache.tez</groupId>
     <artifactId>tez-plugins</artifactId>
-    <version>0.7.0-SNAPSHOT</version>
+    <version>0.8.0-SNAPSHOT</version>
   </parent>
   <artifactId>tez-yarn-timeline-history</artifactId>
 

http://git-wip-us.apache.org/repos/asf/tez/blob/5679b283/tez-runtime-internals/pom.xml
----------------------------------------------------------------------
diff --git a/tez-runtime-internals/pom.xml b/tez-runtime-internals/pom.xml
index ad34c2d..3756fa9 100644
--- a/tez-runtime-internals/pom.xml
+++ b/tez-runtime-internals/pom.xml
@@ -20,7 +20,7 @@
   <parent>
     <groupId>org.apache.tez</groupId>
     <artifactId>tez</artifactId>
-    <version>0.7.0-SNAPSHOT</version>
+    <version>0.8.0-SNAPSHOT</version>
   </parent>
   <artifactId>tez-runtime-internals</artifactId>
 

http://git-wip-us.apache.org/repos/asf/tez/blob/5679b283/tez-runtime-library/pom.xml
----------------------------------------------------------------------
diff --git a/tez-runtime-library/pom.xml b/tez-runtime-library/pom.xml
index 0edf169..03e0ec3 100644
--- a/tez-runtime-library/pom.xml
+++ b/tez-runtime-library/pom.xml
@@ -20,7 +20,7 @@
   <parent>
     <groupId>org.apache.tez</groupId>
     <artifactId>tez</artifactId>
-    <version>0.7.0-SNAPSHOT</version>
+    <version>0.8.0-SNAPSHOT</version>
   </parent>
   <artifactId>tez-runtime-library</artifactId>
 

http://git-wip-us.apache.org/repos/asf/tez/blob/5679b283/tez-tests/pom.xml
----------------------------------------------------------------------
diff --git a/tez-tests/pom.xml b/tez-tests/pom.xml
index 1b3378c..91bc753 100644
--- a/tez-tests/pom.xml
+++ b/tez-tests/pom.xml
@@ -20,7 +20,7 @@
   <parent>
     <groupId>org.apache.tez</groupId>
     <artifactId>tez</artifactId>
-    <version>0.7.0-SNAPSHOT</version>
+    <version>0.8.0-SNAPSHOT</version>
   </parent>
   <artifactId>tez-tests</artifactId>
 

http://git-wip-us.apache.org/repos/asf/tez/blob/5679b283/tez-tools/tez-tfile-parser/pom.xml
----------------------------------------------------------------------
diff --git a/tez-tools/tez-tfile-parser/pom.xml b/tez-tools/tez-tfile-parser/pom.xml
index 38a954e..64191ac 100644
--- a/tez-tools/tez-tfile-parser/pom.xml
+++ b/tez-tools/tez-tfile-parser/pom.xml
@@ -22,8 +22,8 @@
     <version>1.0-SNAPSHOT</version>
 
     <properties>
-        <tez-version>0.6.0-SNAPSHOT</tez-version>
-        <hadoop-version>2.4.0</hadoop-version>
+        <tez-version>0.8.0-SNAPSHOT</tez-version>
+        <hadoop-version>2.6.0</hadoop-version>
     </properties>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/tez/blob/5679b283/tez-ui/pom.xml
----------------------------------------------------------------------
diff --git a/tez-ui/pom.xml b/tez-ui/pom.xml
index ca4d7ab..bfc6f01 100644
--- a/tez-ui/pom.xml
+++ b/tez-ui/pom.xml
@@ -21,7 +21,7 @@
   <parent>
     <groupId>org.apache.tez</groupId>
     <artifactId>tez</artifactId>
-    <version>0.7.0-SNAPSHOT</version>
+    <version>0.8.0-SNAPSHOT</version>
   </parent>
   <artifactId>tez-ui</artifactId>
   <packaging>war</packaging>


[49/50] [abbrv] tez git commit: TEZ-2283. Fixes after rebase 04/07. (sseth)

Posted by ss...@apache.org.
TEZ-2283. Fixes after rebase 04/07. (sseth)


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

Branch: refs/heads/TEZ-2003
Commit: d99aae58bd3c9ca1f00b202c1424aaa7209c89f4
Parents: 8c9d3cf
Author: Siddharth Seth <ss...@apache.org>
Authored: Tue Apr 7 13:13:30 2015 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Wed May 6 00:18:08 2015 -0700

----------------------------------------------------------------------
 TEZ-2003-CHANGES.txt                                         | 1 +
 .../java/org/apache/tez/dag/app/rm/TestContainerReuse.java   | 8 ++++----
 .../org/apache/tez/service/impl/ContainerRunnerImpl.java     | 4 +---
 3 files changed, 6 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/d99aae58/TEZ-2003-CHANGES.txt
----------------------------------------------------------------------
diff --git a/TEZ-2003-CHANGES.txt b/TEZ-2003-CHANGES.txt
index d51686d..6a4399c 100644
--- a/TEZ-2003-CHANGES.txt
+++ b/TEZ-2003-CHANGES.txt
@@ -12,5 +12,6 @@ ALL CHANGES:
   TEZ-2175. Task priority should be available to the TaskCommunicator plugin.
   TEZ-2187. Allow TaskCommunicators to report failed / killed attempts.
   TEZ-2241. Miscellaneous fixes after last reabse.
+  TEZ-2283. Fixes after rebase 04/07.
 
 INCOMPATIBLE CHANGES:

http://git-wip-us.apache.org/repos/asf/tez/blob/d99aae58/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 d45346a..8b4ff07 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
@@ -1242,9 +1242,9 @@ public class TestContainerReuse {
     assertEquals(1, assignEvent.getRemoteTaskLocalResources().size());
 
     // Task assigned to container completed successfully. Container should be re-used.
-    taskSchedulerEventHandler.handleEvent(new AMSchedulerEventTAEnded(ta111, container1.getId(), TaskAttemptState.SUCCEEDED));
+    taskSchedulerEventHandler.handleEvent(new AMSchedulerEventTAEnded(ta111, container1.getId(), TaskAttemptState.SUCCEEDED, null, 0));
     drainableAppCallback.drain();
-    verify(taskScheduler).deallocateTask(eq(ta111), eq(true));
+    verify(taskScheduler).deallocateTask(eq(ta111), eq(true), eq((TaskAttemptEndReason)null));
     verify(taskSchedulerEventHandler).taskAllocated(eq(ta112), any(Object.class), eq(container1));
     verify(rmClient, times(0)).releaseAssignedContainer(eq(container1.getId()));
     eventHandler.verifyNoInvocations(AMContainerEventStopRequest.class);
@@ -1254,9 +1254,9 @@ public class TestContainerReuse {
 
     // Task assigned to container completed successfully.
     // Verify reuse across hosts.
-    taskSchedulerEventHandler.handleEvent(new AMSchedulerEventTAEnded(ta112, container1.getId(), TaskAttemptState.SUCCEEDED));
+    taskSchedulerEventHandler.handleEvent(new AMSchedulerEventTAEnded(ta112, container1.getId(), TaskAttemptState.SUCCEEDED, null, 0));
     drainableAppCallback.drain();
-    verify(taskScheduler).deallocateTask(eq(ta112), eq(true));
+    verify(taskScheduler).deallocateTask(eq(ta112), eq(true), eq((TaskAttemptEndReason)null));
     verify(rmClient, times(0)).releaseAssignedContainer(eq(container1.getId()));
     eventHandler.verifyNoInvocations(AMContainerEventStopRequest.class);
     eventHandler.reset();

http://git-wip-us.apache.org/repos/asf/tez/blob/d99aae58/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/ContainerRunnerImpl.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/ContainerRunnerImpl.java b/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/ContainerRunnerImpl.java
index e7c02c8..f3fc442 100644
--- a/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/ContainerRunnerImpl.java
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/ContainerRunnerImpl.java
@@ -69,10 +69,8 @@ import org.apache.tez.runtime.common.objectregistry.ObjectRegistryImpl;
 import org.apache.tez.runtime.task.TezChild;
 import org.apache.tez.runtime.task.TezChild.ContainerExecutionResult;
 import org.apache.tez.shufflehandler.ShuffleHandler;
-import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos;
 import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos.RunContainerRequestProto;
 import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos.SubmitWorkRequestProto;
-import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos.TaskSpecProto;
 import org.apache.tez.util.ProtoConverters;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -446,7 +444,7 @@ public class ContainerRunnerImpl extends AbstractService implements ContainerRun
           request.getContainerIdString());
 
       taskRunner = new TezTaskRunner(conf, taskUgi, localDirs,
-          ProtoConverters.getTaskSpecfromProto(request.getTaskSpec()), umbilical,
+          ProtoConverters.getTaskSpecfromProto(request.getTaskSpec()),
           request.getAppAttemptNumber(),
           serviceConsumerMetadata, envMap, startedInputsMap, taskReporter, executor, objectRegistry,
           pid,


[10/50] [abbrv] tez git commit: TEZ-2231. Addendum patch. Create project by-laws. (hitesh)

Posted by ss...@apache.org.
TEZ-2231. Addendum patch. Create project by-laws. (hitesh)


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

Branch: refs/heads/TEZ-2003
Commit: 1a5317578fb93cda9cefe27ad1ef90c9e73d08b8
Parents: c924e8a
Author: Hitesh Shah <hi...@apache.org>
Authored: Fri May 1 11:56:29 2015 -0700
Committer: Hitesh Shah <hi...@apache.org>
Committed: Fri May 1 11:56:29 2015 -0700

----------------------------------------------------------------------
 docs/src/site/markdown/by-laws.md | 18 ++++++++++++++++++
 1 file changed, 18 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/1a531757/docs/src/site/markdown/by-laws.md
----------------------------------------------------------------------
diff --git a/docs/src/site/markdown/by-laws.md b/docs/src/site/markdown/by-laws.md
index 9b23201..1decc37 100644
--- a/docs/src/site/markdown/by-laws.md
+++ b/docs/src/site/markdown/by-laws.md
@@ -1,3 +1,21 @@
+!--
+   Licensed to the Apache Software Foundation (ASF) under one or more
+   contributor license agreements.  See the NOTICE file distributed with
+   this work for additional information regarding copyright ownership.
+   The ASF licenses this file to You under the Apache License, Version 2.0
+   (the "License"); you may not use this file except in compliance with
+   the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+   Unless required by applicable law or agreed to in writing, software
+   distributed under the License is distributed on an "AS IS" BASIS,
+   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+   See the License for the specific language governing permissions and
+   limitations under the License.
+-->
+<head><title>Apache Tez Project Bylaws</title></head>
+
 # Apache Tez Project Bylaws
 
 This document defines the bylaws under which the Apache Tez project operates. It defines the roles and responsibilities of the project, who may vote, how voting works, how conflicts are resolved, etc.


[39/50] [abbrv] tez git commit: TEZ-2175. Task priority should be available to the TaskCommunicator plugin. (sseth)

Posted by ss...@apache.org.
TEZ-2175. Task priority should be available to the TaskCommunicator plugin. (sseth)


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

Branch: refs/heads/TEZ-2003
Commit: 25eed21967e5f01bfc152ad9a045a06c44d074e3
Parents: baf2f24
Author: Siddharth Seth <ss...@apache.org>
Authored: Tue Mar 10 00:47:07 2015 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Wed May 6 00:18:07 2015 -0700

----------------------------------------------------------------------
 TEZ-2003-CHANGES.txt                                            | 1 +
 .../src/main/java/org/apache/tez/dag/api/TaskCommunicator.java  | 2 +-
 .../org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java    | 2 +-
 .../java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java    | 3 ++-
 .../dag/app/taskcomm/TezTestServiceTaskCommunicatorImpl.java    | 5 +++--
 5 files changed, 8 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/25eed219/TEZ-2003-CHANGES.txt
----------------------------------------------------------------------
diff --git a/TEZ-2003-CHANGES.txt b/TEZ-2003-CHANGES.txt
index 5c5fd8e..7726815 100644
--- a/TEZ-2003-CHANGES.txt
+++ b/TEZ-2003-CHANGES.txt
@@ -9,5 +9,6 @@ ALL CHANGES:
   TEZ-2131. Add additional tests for tasks running in the AM.
   TEZ-2138. Fix minor bugs in adding default scheduler, getting launchers.
   TEZ-2139. Update tez version to 0.7.0-TEZ-2003-SNAPSHOT.
+  TEZ-2175. Task priority should be available to the TaskCommunicator plugin.
 
 INCOMPATIBLE CHANGES:

http://git-wip-us.apache.org/repos/asf/tez/blob/25eed219/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicator.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicator.java b/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicator.java
index c9f85e0..82eed20 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicator.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicator.java
@@ -43,7 +43,7 @@ public abstract class TaskCommunicator extends AbstractService {
   public abstract void registerRunningTaskAttempt(ContainerId containerId, TaskSpec taskSpec,
                                                   Map<String, LocalResource> additionalResources,
                                                   Credentials credentials,
-                                                  boolean credentialsChanged);
+                                                  boolean credentialsChanged, int priority);
 
   // TODO TEZ-2003 Remove reference to TaskAttemptID
   public abstract void unregisterRunningTaskAttempt(TezTaskAttemptID taskAttemptID);

http://git-wip-us.apache.org/repos/asf/tez/blob/25eed219/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 1f0bb0e..a24a654 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
@@ -360,7 +360,7 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
     }
     taskCommunicators[taskCommId].registerRunningTaskAttempt(containerId, amContainerTask.getTask(),
         amContainerTask.getAdditionalResources(), amContainerTask.getCredentials(),
-        amContainerTask.haveCredentialsChanged());
+        amContainerTask.haveCredentialsChanged(), amContainerTask.getPriority());
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/tez/blob/25eed219/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java
index 0bf1b5d..f288748 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java
@@ -194,7 +194,8 @@ public class TezTaskCommunicatorImpl extends TaskCommunicator {
   @Override
   public void registerRunningTaskAttempt(ContainerId containerId, TaskSpec taskSpec,
                                          Map<String, LocalResource> additionalResources,
-                                         Credentials credentials, boolean credentialsChanged) {
+                                         Credentials credentials, boolean credentialsChanged,
+                                         int priority) {
 
     ContainerInfo containerInfo = registeredContainers.get(containerId);
     Preconditions.checkNotNull(containerInfo,

http://git-wip-us.apache.org/repos/asf/tez/blob/25eed219/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/taskcomm/TezTestServiceTaskCommunicatorImpl.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/taskcomm/TezTestServiceTaskCommunicatorImpl.java b/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/taskcomm/TezTestServiceTaskCommunicatorImpl.java
index 78cdcde..a327caf 100644
--- a/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/taskcomm/TezTestServiceTaskCommunicatorImpl.java
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/taskcomm/TezTestServiceTaskCommunicatorImpl.java
@@ -100,9 +100,10 @@ public class TezTestServiceTaskCommunicatorImpl extends TezTaskCommunicatorImpl
   public void registerRunningTaskAttempt(final ContainerId containerId, final TaskSpec taskSpec,
                                          Map<String, LocalResource> additionalResources,
                                          Credentials credentials,
-                                         boolean credentialsChanged)  {
+                                         boolean credentialsChanged,
+                                         int priority)  {
     super.registerRunningTaskAttempt(containerId, taskSpec, additionalResources, credentials,
-        credentialsChanged);
+        credentialsChanged, priority);
     SubmitWorkRequestProto requestProto = null;
     try {
       requestProto = constructSubmitWorkRequest(containerId, taskSpec);


[19/50] [abbrv] tez git commit: TEZ-2401. Tez UI: All-dag page has duration keep counting for KILLED dag. (pramachandran)

Posted by ss...@apache.org.
TEZ-2401. Tez UI: All-dag page has duration keep counting for KILLED dag. (pramachandran)


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

Branch: refs/heads/TEZ-2003
Commit: 0a6a7d3b671ef46658fe89d203e5b771dbc3fda5
Parents: b005462
Author: Prakash Ramachandran <pr...@hortonworks.com>
Authored: Tue May 5 16:59:10 2015 +0530
Committer: Prakash Ramachandran <pr...@hortonworks.com>
Committed: Tue May 5 16:59:10 2015 +0530

----------------------------------------------------------------------
 CHANGES.txt                                                  | 1 +
 tez-ui/src/main/webapp/app/scripts/helpers/date.js           | 1 +
 .../src/main/webapp/app/scripts/helpers/handlebarHelpers.js  | 8 ++++++--
 3 files changed, 8 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/0a6a7d3b/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 816c7a5..ba03aa3 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -9,6 +9,7 @@ INCOMPATIBLE CHANGES
   TEZ-1993. Implement a pluggable InputSizeEstimator for grouping fairly
 
 ALL CHANGES:
+  TEZ-2401. Tez UI: All-dag page has duration keep counting for KILLED dag.
   TEZ-2392. Have all readers throw an Exception on incorrect next() usage.
   TEZ-2408. TestTaskAttempt fails to compile against hadoop-2.4 and hadoop-2.2.
   TEZ-2405. PipelinedSorter can throw NPE with custom compartor.

http://git-wip-us.apache.org/repos/asf/tez/blob/0a6a7d3b/tez-ui/src/main/webapp/app/scripts/helpers/date.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/scripts/helpers/date.js b/tez-ui/src/main/webapp/app/scripts/helpers/date.js
index 10df10b..5990c24 100644
--- a/tez-ui/src/main/webapp/app/scripts/helpers/date.js
+++ b/tez-ui/src/main/webapp/app/scripts/helpers/date.js
@@ -207,6 +207,7 @@ App.Helpers.date = {
    * @method duration
    */
   duration: function (startTime, endTime) {
+    if (!startTime || !endTime) return undefined;
     var duration = 0;
     if (startTime && startTime > 0) {
       if (!endTime || endTime < 1) {

http://git-wip-us.apache.org/repos/asf/tez/blob/0a6a7d3b/tez-ui/src/main/webapp/app/scripts/helpers/handlebarHelpers.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/scripts/helpers/handlebarHelpers.js b/tez-ui/src/main/webapp/app/scripts/helpers/handlebarHelpers.js
index f42a504..5cbdef5 100644
--- a/tez-ui/src/main/webapp/app/scripts/helpers/handlebarHelpers.js
+++ b/tez-ui/src/main/webapp/app/scripts/helpers/handlebarHelpers.js
@@ -48,9 +48,13 @@ Em.Handlebars.helper('formatNumThousands', function (value) {
  * @method formatDuration
  */
 Em.Handlebars.helper('formatDuration', function(startTime, endTime) {
+  if (!endTime || !startTime) {
+    return 'Not Available';
+  }
+
 	// unixtimestamp is in seconds. javascript expects milliseconds.
-	if (endTime < startTime || !!endTime) {
-		end = new Date().getTime();
+	if (endTime < startTime) {
+		endTime = new Date().getTime();
 	}
 
 	return App.Helpers.date.durationSummary(startTime, endTime);


[15/50] [abbrv] tez git commit: TEZ-2379. org.apache.hadoop.yarn.state.InvalidStateTransitonException: Invalid event: T_ATTEMPT_KILLED at KILLED. (hitesh)

Posted by ss...@apache.org.
TEZ-2379. org.apache.hadoop.yarn.state.InvalidStateTransitonException: Invalid event: T_ATTEMPT_KILLED at KILLED. (hitesh)


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

Branch: refs/heads/TEZ-2003
Commit: ba6d7e0e9106ad26c48bda8bb8caa7e4a890c2b1
Parents: 5218f48
Author: Hitesh Shah <hi...@apache.org>
Authored: Mon May 4 12:20:46 2015 -0700
Committer: Hitesh Shah <hi...@apache.org>
Committed: Mon May 4 12:20:46 2015 -0700

----------------------------------------------------------------------
 CHANGES.txt                                     |  2 +
 .../apache/tez/dag/app/dag/impl/TaskImpl.java   | 21 ++++++--
 .../tez/dag/app/dag/impl/TestTaskImpl.java      | 53 +++++++++++++++++++-
 3 files changed, 72 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/ba6d7e0e/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 6c19770..0027e98 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -335,6 +335,8 @@ TEZ-UI CHANGES (TEZ-8):
 Release 0.5.4: Unreleased
 
 ALL CHANGES:
+  TEZ-2379. org.apache.hadoop.yarn.state.InvalidStateTransitonException:
+    Invalid event: T_ATTEMPT_KILLED at KILLED.
   TEZ-2397. Translation of LocalResources via Tez plan serialization can be lossy.
   TEZ-2221. VertexGroup name should be unqiue
   TEZ-1521. VertexDataMovementEventsGeneratedEvent may be logged twice in recovery log

http://git-wip-us.apache.org/repos/asf/tez/blob/ba6d7e0e/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 8b63734..15382a8 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
@@ -117,7 +117,7 @@ public class TaskImpl implements Task, EventHandler<TaskEvent> {
   protected final EventHandler eventHandler;
   private final TezTaskID taskId;
   private Map<TezTaskAttemptID, TaskAttempt> attempts;
-  private final int maxFailedAttempts;
+  protected final int maxFailedAttempts;
   protected final Clock clock;
   private final Vertex vertex;
   private final Lock readLock;
@@ -256,14 +256,29 @@ public class TaskImpl implements Task, EventHandler<TaskEvent> {
         EnumSet.of(
             TaskEventType.T_TERMINATE,
             TaskEventType.T_SCHEDULE,
-            TaskEventType.T_ADD_SPEC_ATTEMPT))
+            TaskEventType.T_ADD_SPEC_ATTEMPT,
+            TaskEventType.T_ATTEMPT_KILLED))
 
     // Transitions from KILLED state
+    // Ignorable event: T_ATTEMPT_KILLED
+    // Refer to TEZ-2379
+    // T_ATTEMPT_KILLED can show up in KILLED state as
+    // a SUCCEEDED attempt can still transition to KILLED after receiving
+    // a KILL event.
+    // This could happen when there is a race where the task receives a
+    // kill event, it tries to kill all running attempts and a potential
+    // running attempt succeeds before it receives the kill event.
+    // The task will then receive both a SUCCEEDED and KILLED
+    // event from the same attempt.
+    // Duplicate events from a single attempt in KILL_WAIT are handled
+    // properly. However, the subsequent T_ATTEMPT_KILLED event might
+    // be received after the task reaches its terminal KILLED state.
     .addTransition(TaskStateInternal.KILLED, TaskStateInternal.KILLED,
         EnumSet.of(
             TaskEventType.T_TERMINATE,
             TaskEventType.T_SCHEDULE,
-            TaskEventType.T_ADD_SPEC_ATTEMPT))
+            TaskEventType.T_ADD_SPEC_ATTEMPT,
+            TaskEventType.T_ATTEMPT_KILLED))
 
     // create the topology tables
     .installTopology();

http://git-wip-us.apache.org/repos/asf/tez/blob/ba6d7e0e/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 66e6724..9da3fab 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
@@ -177,6 +177,12 @@ public class TestTaskImpl {
     assertTaskKillWaitState();
   }
 
+  private void failTask(TezTaskID taskId) {
+    mockTask.handle(new TaskEventTermination(taskId, TaskAttemptTerminationCause.TERMINATED_AT_SHUTDOWN, null));
+    assertTaskKillWaitState();
+  }
+
+
   private void killScheduledTaskAttempt(TezTaskAttemptID attemptId) {
     mockTask.handle(new TaskEventTAUpdate(attemptId,
         TaskEventType.T_ATTEMPT_KILLED));
@@ -289,7 +295,6 @@ public class TestTaskImpl {
     killTask(taskId);
     mockTask.handle(new TaskEventTAUpdate(mockTask.getLastAttempt().getID(),
         TaskEventType.T_ATTEMPT_KILLED));
-
     assertEquals(TaskStateInternal.KILLED, mockTask.getInternalState());
     verifyOutgoingEvents(eventHandler.events, VertexEventType.V_TASK_COMPLETED);
   }
@@ -377,6 +382,52 @@ public class TestTaskImpl {
     killRunningTaskAttempt(mockTask.getLastAttempt().getID());
   }
 
+  /**
+   * {@link TaskState#KILLED}->{@link TaskState#KILLED}
+   */
+  @Test(timeout = 5000)
+  public void testKilledAttemptAtTaskKilled() {
+    LOG.info("--- START: testKilledAttemptAtTaskKilled ---");
+    TezTaskID taskId = getNewTaskID();
+    scheduleTaskAttempt(taskId);
+    launchTaskAttempt(mockTask.getLastAttempt().getID());
+    killTask(taskId);
+    mockTask.handle(new TaskEventTAUpdate(mockTask.getLastAttempt().getID(),
+        TaskEventType.T_ATTEMPT_KILLED));
+    assertEquals(TaskStateInternal.KILLED, mockTask.getInternalState());
+
+    // Send duplicate kill for same attempt
+    // This will not happen in practice but this is to simulate handling
+    // of killed attempts in killed state.
+    mockTask.handle(new TaskEventTAUpdate(mockTask.getLastAttempt().getID(),
+        TaskEventType.T_ATTEMPT_KILLED));
+    assertEquals(TaskStateInternal.KILLED, mockTask.getInternalState());
+
+  }
+
+  /**
+   * {@link TaskState#FAILED}->{@link TaskState#FAILED}
+   */
+  @Test(timeout = 5000)
+  public void testKilledAttemptAtTaskFailed() {
+    LOG.info("--- START: testKilledAttemptAtTaskFailed ---");
+    TezTaskID taskId = getNewTaskID();
+    scheduleTaskAttempt(taskId);
+    for (int i = 0; i < mockTask.maxFailedAttempts; ++i) {
+      mockTask.handle(new TaskEventTAUpdate(mockTask.getLastAttempt().getID(),
+          TaskEventType.T_ATTEMPT_FAILED));
+    }
+    assertEquals(TaskStateInternal.FAILED, mockTask.getInternalState());
+
+    // Send kill for an attempt
+    mockTask.handle(new TaskEventTAUpdate(mockTask.getLastAttempt().getID(),
+        TaskEventType.T_ATTEMPT_KILLED));
+    assertEquals(TaskStateInternal.FAILED, mockTask.getInternalState());
+
+  }
+
+
+
   @Test(timeout = 5000)
   public void testFetchedEventsModifyUnderlyingList() {
     // Tests to ensure that adding an event to a task, does not affect the


[22/50] [abbrv] tez git commit: TEZ-2415. PMC RDF needs to use asfext:pmc, not asfext:PMC. (hitesh)

Posted by ss...@apache.org.
TEZ-2415. PMC RDF needs to use asfext:pmc, not asfext:PMC. (hitesh)


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

Branch: refs/heads/TEZ-2003
Commit: bce7ff211f1b557a4aca46db8e839d5c2a0c8890
Parents: 146ab07
Author: Hitesh Shah <hi...@apache.org>
Authored: Tue May 5 15:28:46 2015 -0700
Committer: Hitesh Shah <hi...@apache.org>
Committed: Tue May 5 15:29:54 2015 -0700

----------------------------------------------------------------------
 CHANGES.txt                         | 1 +
 docs/src/site/resources/pmc/tez.rdf | 9 ++-------
 2 files changed, 3 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/bce7ff21/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 7ba8021..c4ae033 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -9,6 +9,7 @@ INCOMPATIBLE CHANGES
   TEZ-1993. Implement a pluggable InputSizeEstimator for grouping fairly
 
 ALL CHANGES:
+  TEZ-2415. PMC RDF needs to use asfext:pmc, not asfext:PMC.
   TEZ-1752. Inputs / Outputs in the Runtime library should be interruptable.
   TEZ-2401. Tez UI: All-dag page has duration keep counting for KILLED dag.
   TEZ-2392. Have all readers throw an Exception on incorrect next() usage.

http://git-wip-us.apache.org/repos/asf/tez/blob/bce7ff21/docs/src/site/resources/pmc/tez.rdf
----------------------------------------------------------------------
diff --git a/docs/src/site/resources/pmc/tez.rdf b/docs/src/site/resources/pmc/tez.rdf
index 3086f4a..86faad0 100644
--- a/docs/src/site/resources/pmc/tez.rdf
+++ b/docs/src/site/resources/pmc/tez.rdf
@@ -17,7 +17,7 @@
          xmlns:rdf="http://www.w3.org/1999/02/22-rdf-syntax-ns#" 
          xmlns:asfext="http://projects.apache.org/ns/asfext#"
          xmlns:foaf="http://xmlns.com/foaf/0.1/">
-  <asfext:PMC rdf:about="tez">
+  <asfext:pmc rdf:about="tez">
     <asfext:name>Apache Tez</asfext:name>
     <foaf:homepage rdf:resource="http://tez.apache.org/"/>
     <asfext:chair>
@@ -28,11 +28,6 @@
     <asfext:charter>Apache Tez is an effort to develop a generic application framework which can be used to process arbitrarily complex DAGs of data-processing tasks and also a re-usable set of data-processing primitives which can be used by other projects.</asfext:charter>
     <asfext:member>
       <foaf:Person>
-        <foaf:name>Alan Gates</foaf:name>
-      </foaf:Person>
-    </asfext:member>
-    <asfext:member>
-      <foaf:Person>
         <foaf:name>Arun C. Murthy</foaf:name>
       </foaf:Person>
     </asfext:member>
@@ -186,6 +181,6 @@
         <foaf:name>Vinod Kumar Vavilapalli</foaf:name>
       </foaf:Person>
     </asfext:member>
-  </asfext:PMC>
+  </asfext:pmc>
 
 </rdf:RDF>


[02/50] [abbrv] tez git commit: TEZ-2360. per-io counters flag should generate both overall and per-edge counters (pramachandran)

Posted by ss...@apache.org.
TEZ-2360. per-io counters flag should generate both overall and per-edge counters (pramachandran)


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

Branch: refs/heads/TEZ-2003
Commit: 765afd236fd178ebc315c1f912102578ec100c70
Parents: e36f962
Author: Prakash Ramachandran <pr...@hortonworks.com>
Authored: Fri May 1 00:48:16 2015 +0530
Committer: Prakash Ramachandran <pr...@hortonworks.com>
Committed: Fri May 1 00:48:16 2015 +0530

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 .../runtime/api/impl/TezCountersDelegate.java   |  74 +++++++-
 .../java/org/apache/tez/test/TestTezJobs.java   | 184 +++++++++++++++++++
 3 files changed, 255 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/765afd23/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index d67db81..aa72320 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -9,6 +9,7 @@ INCOMPATIBLE CHANGES
   TEZ-1993. Implement a pluggable InputSizeEstimator for grouping fairly
 
 ALL CHANGES:
+  TEZ-2360. per-io counters flag should generate both overall and per-edge counters
   TEZ-2389. Tez UI: Sort by attempt-no is incorrect in attempts pages.
   TEZ-2383. Cleanup input/output/processor contexts in LogicalIOProcessorRuntimeTask.
   TEZ-2084. Tez UI: Stacktrace format info is lost in diagnostics

http://git-wip-us.apache.org/repos/asf/tez/blob/765afd23/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezCountersDelegate.java
----------------------------------------------------------------------
diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezCountersDelegate.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezCountersDelegate.java
index 5286839..3c79530 100644
--- a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezCountersDelegate.java
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezCountersDelegate.java
@@ -18,7 +18,12 @@
 
 package org.apache.tez.runtime.api.impl;
 
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
 import org.apache.tez.common.TezUtilsInternal;
+import org.apache.tez.common.counters.AbstractCounter;
 import org.apache.tez.common.counters.TaskCounter;
 import org.apache.tez.common.counters.TezCounter;
 import org.apache.tez.common.counters.TezCounters;
@@ -27,7 +32,7 @@ public class TezCountersDelegate extends TezCounters {
 
   private final String groupModifier;
   private final TezCounters original;
-  
+
   public TezCountersDelegate(TezCounters original, String taskVertexName, String edgeVertexName,
       String type) {
     this.original = original;
@@ -39,10 +44,71 @@ public class TezCountersDelegate extends TezCounters {
   // the standard mechanism to find a counter.
   @Override
   public TezCounter findCounter(String groupName, String counterName) {
+    String simpleGroupName;
     if (groupName.equals(TaskCounter.class.getName())) {
-      groupName = TaskCounter.class.getSimpleName();
+      simpleGroupName = TaskCounter.class.getSimpleName();
+    } else  {
+      simpleGroupName = groupName;
+    }
+    String modifiedGroupName = simpleGroupName + "_" + this.groupModifier;
+    final TezCounter modifiedGroupCounter = original.findCounter(modifiedGroupName, counterName);
+    final TezCounter originalGroupCounter = original.findCounter(groupName, counterName);
+    return new CompositeCounter(modifiedGroupCounter, originalGroupCounter);
+  }
+
+  /*
+   * A counter class to wrap multiple counters. increment operation will increment both counters
+   */
+  private static class CompositeCounter extends AbstractCounter {
+
+    TezCounter modifiedCounter;
+    TezCounter originalCounter;
+
+    public CompositeCounter(TezCounter modifiedCounter, TezCounter originalCounter) {
+      this.modifiedCounter = modifiedCounter;
+      this.originalCounter = originalCounter;
+    }
+
+    @Override
+    public String getName() {
+      return modifiedCounter.getName();
+    }
+
+    @Override
+    public String getDisplayName() {
+      return modifiedCounter.getName();
+    }
+
+    @Override
+    public long getValue() {
+      return modifiedCounter.getValue();
+    }
+
+    @Override
+    public void setValue(long value) {
+      modifiedCounter.setValue(value);
+      originalCounter.setValue(value);
+    }
+
+    @Override
+    public void increment(long increment) {
+      modifiedCounter.increment(increment);
+      originalCounter.increment(increment);
+    }
+
+    @Override
+    public TezCounter getUnderlyingCounter() {
+      return this;
+    }
+
+    @Override
+    public void write(DataOutput dataOutput) throws IOException {
+      assert false : "shouldn't be called";
+    }
+
+    @Override
+    public void readFields(DataInput dataInput) throws IOException {
+      assert false : "shouldn't be called";
     }
-    String modifiedGroupName = groupName + "_" + this.groupModifier;
-    return original.findCounter(modifiedGroupName, counterName);
   }
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/765afd23/tez-tests/src/test/java/org/apache/tez/test/TestTezJobs.java
----------------------------------------------------------------------
diff --git a/tez-tests/src/test/java/org/apache/tez/test/TestTezJobs.java b/tez-tests/src/test/java/org/apache/tez/test/TestTezJobs.java
index db212fa..13b0c03 100644
--- a/tez-tests/src/test/java/org/apache/tez/test/TestTezJobs.java
+++ b/tez-tests/src/test/java/org/apache/tez/test/TestTezJobs.java
@@ -19,6 +19,7 @@
 package org.apache.tez.test;
 
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
@@ -29,6 +30,7 @@ import java.io.IOException;
 import java.io.InputStreamReader;
 import java.io.OutputStreamWriter;
 import java.nio.ByteBuffer;
+import java.util.Arrays;
 import java.util.BitSet;
 import java.util.EnumSet;
 import java.util.HashSet;
@@ -41,6 +43,12 @@ import java.util.concurrent.locks.ReentrantLock;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
 
+import org.apache.tez.common.counters.CounterGroup;
+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.api.client.StatusGetOpts;
+import org.apache.tez.dag.api.client.VertexStatus;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -313,6 +321,103 @@ public class TestTezJobs {
   }
 
   @Test(timeout = 60000)
+  public void testPerIOCounterAggregation() throws Exception {
+    String baseDir = "/tmp/perIOCounterAgg/";
+    Path inPath1 = new Path(baseDir + "inPath1");
+    Path inPath2 = new Path(baseDir + "inPath2");
+    Path outPath = new Path(baseDir + "outPath");
+    final Set<String> expectedResults = generateSortMergeJoinInput(inPath1, inPath2);
+    Path stagingDirPath = new Path("/tmp/tez-staging-dir");
+    remoteFs.mkdirs(stagingDirPath);
+
+    TezConfiguration conf = new TezConfiguration(mrrTezCluster.getConfig());
+    conf.setBoolean(TezConfiguration.TEZ_TASK_GENERATE_COUNTERS_PER_IO, true);
+    TezClient tezClient = TezClient.create(SortMergeJoinHelper.class.getSimpleName(), conf);
+    tezClient.start();
+
+    SortMergeJoinHelper sortMergeJoinHelper = new SortMergeJoinHelper(tezClient);
+    sortMergeJoinHelper.setConf(conf);
+
+    String[] args = new String[] {
+        "-D" + TezConfiguration.TEZ_AM_STAGING_DIR + "=" + stagingDirPath.toString(),
+        inPath1.toString(), inPath2.toString(), "1", outPath.toString() };
+    assertEquals(0, sortMergeJoinHelper.run(conf, args, tezClient));
+
+    verifySortMergeJoinInput(outPath, expectedResults);
+
+    String joinerVertexName = "joiner";
+    String input1Name = "input1";
+    String input2Name = "input2";
+    String joinOutputName = "joinOutput";
+    Set<StatusGetOpts> statusOpts = new HashSet<StatusGetOpts>();
+    statusOpts.add(StatusGetOpts.GET_COUNTERS);
+    VertexStatus joinerVertexStatus =
+        sortMergeJoinHelper.dagClient.getVertexStatus(joinerVertexName, statusOpts);
+    final TezCounters joinerCounters = joinerVertexStatus.getVertexCounters();
+    final CounterGroup aggregatedGroup = joinerCounters.getGroup(TaskCounter.class.getCanonicalName());
+    final CounterGroup input1Group = joinerCounters.getGroup(
+        TaskCounter.class.getSimpleName() + "_" + joinerVertexName + "_INPUT_" + input1Name);
+    final CounterGroup input2Group = joinerCounters.getGroup(
+        TaskCounter.class.getSimpleName() + "_" + joinerVertexName + "_INPUT_" + input2Name);
+    assertTrue("aggregated counter group cannot be empty", aggregatedGroup.size() > 0);
+    assertTrue("per io group for input1 cannot be empty", input1Group.size() > 0);
+    assertTrue("per io group for input1 cannot be empty", input2Group.size() > 0);
+
+    List<TaskCounter> countersToVerifyAgg = Arrays.asList(
+        TaskCounter.ADDITIONAL_SPILLS_BYTES_READ,
+        TaskCounter.ADDITIONAL_SPILLS_BYTES_WRITTEN,
+        TaskCounter.COMBINE_INPUT_RECORDS,
+        TaskCounter.MERGED_MAP_OUTPUTS,
+        TaskCounter.NUM_DISK_TO_DISK_MERGES,
+        TaskCounter.NUM_FAILED_SHUFFLE_INPUTS,
+        TaskCounter.NUM_MEM_TO_DISK_MERGES,
+        TaskCounter.NUM_SHUFFLED_INPUTS,
+        TaskCounter.NUM_SKIPPED_INPUTS,
+        TaskCounter.REDUCE_INPUT_GROUPS,
+        TaskCounter.REDUCE_INPUT_RECORDS,
+        TaskCounter.SHUFFLE_BYTES,
+        TaskCounter.SHUFFLE_BYTES_DECOMPRESSED,
+        TaskCounter.SHUFFLE_BYTES_DISK_DIRECT,
+        TaskCounter.SHUFFLE_BYTES_TO_DISK,
+        TaskCounter.SHUFFLE_BYTES_TO_MEM,
+        TaskCounter.SPILLED_RECORDS
+    );
+
+    int nonZeroCounters = 0;
+    // verify that the sum of the counter values for edges add up to the aggregated counter value.
+    for(TaskCounter c : countersToVerifyAgg) {
+      TezCounter aggregatedCounter = aggregatedGroup.findCounter(c.name(), false);
+      TezCounter input1Counter = input1Group.findCounter(c.name(), false);
+      TezCounter input2Counter = input2Group.findCounter(c.name(), false);
+      assertNotNull("aggregated counter cannot be null " + c.name(), aggregatedCounter);
+      assertNotNull("input1 counter cannot be null " + c.name(), input1Counter);
+      assertNotNull("input2 counter cannot be null " + c.name(), input2Counter);
+
+      assertEquals("aggregated counter does not match sum of input counters " + c.name(),
+          aggregatedCounter.getValue(), input1Counter.getValue() + input2Counter.getValue());
+
+      if (aggregatedCounter.getValue() > 0) {
+        nonZeroCounters++;
+      }
+    }
+
+    // ensure that at least one of the counters tested above were non-zero.
+    assertTrue("At least one of the counter should be non-zero. invalid test ", nonZeroCounters > 0);
+
+    CounterGroup joinerOutputGroup = joinerCounters.getGroup(
+        TaskCounter.class.getSimpleName() + "_" + joinerVertexName + "_OUTPUT_" + joinOutputName);
+    String outputCounterName = TaskCounter.OUTPUT_RECORDS.name();
+    TezCounter aggregateCounter = aggregatedGroup.findCounter(outputCounterName, false);
+    TezCounter joinerOutputCounter = joinerOutputGroup.findCounter(outputCounterName, false);
+    assertNotNull("aggregated counter cannot be null " + outputCounterName, aggregateCounter);
+    assertNotNull("output counter cannot be null " + outputCounterName, joinerOutputCounter);
+    assertTrue("counter value is zero. test is invalid", aggregateCounter.getValue() > 0);
+    assertEquals("aggregated counter does not match sum of output counters " + outputCounterName,
+        aggregateCounter.getValue(), joinerOutputCounter.getValue());
+  }
+
+
+  @Test(timeout = 60000)
   public void testSortMergeJoinExampleDisableSplitGrouping() throws Exception {
     SortMergeJoinExample sortMergeJoinExample = new SortMergeJoinExample();
     sortMergeJoinExample.setConf(conf);
@@ -982,4 +1087,83 @@ public class TestTezJobs {
       }
     }
   }
+
+  private static class SortMergeJoinHelper extends SortMergeJoinExample {
+    private final TezClient tezClientInternal;
+    private DAGClient dagClient;
+
+    public SortMergeJoinHelper(TezClient tezClient) {
+      this.tezClientInternal = tezClient;
+    }
+
+    @Override
+    public int runDag(DAG dag, boolean printCounters, Logger logger) throws TezException,
+        InterruptedException, IOException {
+      tezClientInternal.waitTillReady();
+      dagClient = tezClientInternal.submitDAG(dag);
+      Set<StatusGetOpts> getOpts = new HashSet<StatusGetOpts>();
+      if (printCounters) {
+        getOpts.add(StatusGetOpts.GET_COUNTERS);
+      }
+
+      DAGStatus dagStatus;
+      dagStatus = dagClient.waitForCompletionWithStatusUpdates(getOpts);
+
+      if (dagStatus.getState() != DAGStatus.State.SUCCEEDED) {
+        logger.info("DAG diagnostics: " + dagStatus.getDiagnostics());
+        return -1;
+      }
+      return 0;
+    }
+  }
+
+  private Set<String> generateSortMergeJoinInput(Path inPath1, Path inPath2) throws
+      IOException {
+    remoteFs.mkdirs(inPath1);
+    remoteFs.mkdirs(inPath2);
+
+    Set<String> expectedResult = new HashSet<String>();
+    FSDataOutputStream out1 = remoteFs.create(new Path(inPath1, "file"));
+    FSDataOutputStream out2 = remoteFs.create(new Path(inPath2, "file"));
+
+    BufferedWriter writer1 = new BufferedWriter(new OutputStreamWriter(out1));
+    BufferedWriter writer2 = new BufferedWriter(new OutputStreamWriter(out2));
+    for (int i = 0; i < 20; i++) {
+      String term = "term" + i;
+      writer1.write(term);
+      writer1.newLine();
+      if (i % 2 == 0) {
+        writer2.write(term);
+        writer2.newLine();
+        expectedResult.add(term);
+      }
+    }
+    writer1.close();
+    writer2.close();
+    out1.close();
+    out2.close();
+
+    return expectedResult;
+  }
+
+  private void verifySortMergeJoinInput(Path outPath, Set<String> expectedResult) throws
+      IOException {
+    FileStatus[] statuses = remoteFs.listStatus(outPath, new PathFilter() {
+      public boolean accept(Path p) {
+        String name = p.getName();
+        return !name.startsWith("_") && !name.startsWith(".");
+      }
+    });
+    assertEquals(1, statuses.length);
+    FSDataInputStream inStream = remoteFs.open(statuses[0].getPath());
+    BufferedReader reader = new BufferedReader(new InputStreamReader(inStream));
+    String line;
+    while ((line = reader.readLine()) != null) {
+      assertTrue(expectedResult.remove(line));
+    }
+    reader.close();
+    inStream.close();
+    assertEquals(0, expectedResult.size());
+  }
+
 }


[27/50] [abbrv] tez git commit: TEZ-2131. Add additional tests for tasks running in the AM. (sseth)

Posted by ss...@apache.org.
TEZ-2131. Add additional tests for tasks running in the AM. (sseth)


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

Branch: refs/heads/TEZ-2003
Commit: bead03d3eb4da00d3cb76086c4bbef9569134681
Parents: f7862e8
Author: Siddharth Seth <ss...@apache.org>
Authored: Fri Feb 20 17:23:18 2015 -0800
Committer: Siddharth Seth <ss...@apache.org>
Committed: Wed May 6 00:13:29 2015 -0700

----------------------------------------------------------------------
 TEZ-2003-CHANGES.txt                                      |  1 +
 .../org/apache/tez/tests/TestExternalTezServices.java     | 10 +++++++---
 2 files changed, 8 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/bead03d3/TEZ-2003-CHANGES.txt
----------------------------------------------------------------------
diff --git a/TEZ-2003-CHANGES.txt b/TEZ-2003-CHANGES.txt
index 76496c9..4b0a12b 100644
--- a/TEZ-2003-CHANGES.txt
+++ b/TEZ-2003-CHANGES.txt
@@ -6,5 +6,6 @@ ALL CHANGES:
   TEZ-2122. Setup pluggable components at AM/Vertex level.
   TEZ-2123. Fix component managers to use pluggable components. (Enable hybrid mode)
   TEZ-2125. Create a task communicator for local mode. Allow tasks to run in the AM.
+  TEZ-2131. Add additional tests for tasks running in the AM.
 
 INCOMPATIBLE CHANGES:

http://git-wip-us.apache.org/repos/asf/tez/blob/bead03d3/tez-ext-service-tests/src/test/java/org/apache/tez/tests/TestExternalTezServices.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/tests/TestExternalTezServices.java b/tez-ext-service-tests/src/test/java/org/apache/tez/tests/TestExternalTezServices.java
index 01c2080..0ec972b 100644
--- a/tez-ext-service-tests/src/test/java/org/apache/tez/tests/TestExternalTezServices.java
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/tests/TestExternalTezServices.java
@@ -40,7 +40,6 @@ import org.apache.tez.service.MiniTezTestServiceCluster;
 import org.apache.tez.test.MiniTezCluster;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
-import org.junit.Ignore;
 import org.junit.Test;
 
 public class TestExternalTezServices {
@@ -229,6 +228,13 @@ public class TestExternalTezServices {
   }
 
   @Test(timeout = 60000)
+  public void testAllInAM() throws Exception {
+    int expectedExternalSubmissions = 0; // All in AM
+    runJoinValidate("AllInAM", expectedExternalSubmissions, PROPS_IN_AM,
+        PROPS_IN_AM, PROPS_IN_AM);
+  }
+
+  @Test(timeout = 60000)
   public void testMixed1() throws Exception { // M-ExtService, R-containers
     int expectedExternalSubmissions = 4 + 0; //4 for 4 src files, 0 for num reducers.
     runJoinValidate("Mixed1", expectedExternalSubmissions, PROPS_EXT_SERVICE_PUSH,
@@ -263,8 +269,6 @@ public class TestExternalTezServices {
         PROPS_EXT_SERVICE_PUSH, PROPS_IN_AM);
   }
 
-
-  @Ignore // Re-activate this after the AM registers the shuffle token with the launcher.
   @Test(timeout = 60000)
   public void testMixed6() throws Exception { // M - AM, R - Service
     int expectedExternalSubmissions = 0 + 3; // 3 for R in service


[20/50] [abbrv] tez git commit: TEZ-2369. Add a few unit tests for RootInputInitializerManager. (sseth)

Posted by ss...@apache.org.
TEZ-2369. Add a few unit tests for RootInputInitializerManager. (sseth)


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

Branch: refs/heads/TEZ-2003
Commit: 8c44f2484b626ce9a7a3408bd3b0d7e0df2b1a24
Parents: 0a6a7d3
Author: Siddharth Seth <ss...@apache.org>
Authored: Tue May 5 11:55:47 2015 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Tue May 5 11:55:47 2015 -0700

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 .../app/dag/RootInputInitializerManager.java    |   1 +
 .../dag/TestRootInputInitializerManager.java    | 201 +++++++++++++++++++
 3 files changed, 203 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/8c44f248/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index ba03aa3..d7a1e1f 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -339,6 +339,7 @@ TEZ-UI CHANGES (TEZ-8):
 Release 0.5.4: Unreleased
 
 ALL CHANGES:
+  TEZ-2369. Add a few unit tests for RootInputInitializerManager.
   TEZ-2379. org.apache.hadoop.yarn.state.InvalidStateTransitonException:
     Invalid event: T_ATTEMPT_KILLED at KILLED.
   TEZ-2397. Translation of LocalResources via Tez plan serialization can be lossy.

http://git-wip-us.apache.org/repos/asf/tez/blob/8c44f248/tez-dag/src/main/java/org/apache/tez/dag/app/dag/RootInputInitializerManager.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/RootInputInitializerManager.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/RootInputInitializerManager.java
index 7156e60..4ee00fa 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/RootInputInitializerManager.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/RootInputInitializerManager.java
@@ -425,6 +425,7 @@ public class RootInputInitializerManager {
             if (taskAttemptIndex == successfulAttempt) {
               toForwardEvents.add((InputInitializerEvent) tezEvent.getEvent());
             }
+            // Drop all other events which have the same source task Id.
             eventIterator.remove();
           }
         }

http://git-wip-us.apache.org/repos/asf/tez/blob/8c44f248/tez-dag/src/test/java/org/apache/tez/dag/app/dag/TestRootInputInitializerManager.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/TestRootInputInitializerManager.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/TestRootInputInitializerManager.java
new file mode 100644
index 0000000..89eb2a6
--- /dev/null
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/TestRootInputInitializerManager.java
@@ -0,0 +1,201 @@
+/*
+ * 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.
+ */
+
+package org.apache.tez.dag.app.dag;
+
+import static org.junit.Assert.assertEquals;
+import static org.mockito.Matchers.any;
+import static org.mockito.Mockito.RETURNS_DEEP_STUBS;
+import static org.mockito.Mockito.doReturn;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.never;
+import static org.mockito.Mockito.reset;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import java.util.List;
+
+import com.google.common.collect.Lists;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.tez.dag.api.InputDescriptor;
+import org.apache.tez.dag.api.InputInitializerDescriptor;
+import org.apache.tez.dag.api.RootInputLeafOutput;
+import org.apache.tez.dag.api.oldrecords.TaskState;
+import org.apache.tez.dag.app.AppContext;
+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.InputInitializer;
+import org.apache.tez.runtime.api.InputInitializerContext;
+import org.apache.tez.runtime.api.events.InputInitializerEvent;
+import org.apache.tez.runtime.api.impl.EventMetaData;
+import org.apache.tez.runtime.api.impl.TezEvent;
+import org.junit.Test;
+import org.mockito.ArgumentCaptor;
+
+public class TestRootInputInitializerManager {
+
+  // Simple testing. No events if task doesn't succeed.
+  // Also exercises path where two attempts are reported as successful via the stateChangeNotifier.
+  // Primarily a failure scenario, when a Task moves back to running from success
+  // Order event1, success1, event2, success2
+  @SuppressWarnings("unchecked")
+  @Test(timeout = 5000)
+  public void testEventBeforeSuccess() throws Exception {
+    InputDescriptor id = mock(InputDescriptor.class);
+    InputInitializerDescriptor iid = mock(InputInitializerDescriptor.class);
+    RootInputLeafOutput<InputDescriptor, InputInitializerDescriptor> rootInput =
+        new RootInputLeafOutput<InputDescriptor, InputInitializerDescriptor>("InputName", id, iid);
+
+    InputInitializer initializer = mock(InputInitializer.class);
+    InputInitializerContext initializerContext = mock(InputInitializerContext.class);
+    Vertex vertex = mock(Vertex.class);
+    StateChangeNotifier stateChangeNotifier = mock(StateChangeNotifier.class);
+    AppContext appContext = mock(AppContext.class, RETURNS_DEEP_STUBS);
+
+    RootInputInitializerManager.InitializerWrapper initializerWrapper =
+        new RootInputInitializerManager.InitializerWrapper(rootInput, initializer,
+            initializerContext, vertex, stateChangeNotifier, appContext);
+
+    ApplicationId appId = ApplicationId.newInstance(1000, 1);
+    TezDAGID dagId = TezDAGID.getInstance(appId, 1);
+    TezVertexID srcVertexId = TezVertexID.getInstance(dagId, 2);
+    TezTaskID srcTaskId1 = TezTaskID.getInstance(srcVertexId, 3);
+    Vertex srcVertex = mock(Vertex.class);
+    Task srcTask1 = mock(Task.class);
+    doReturn(TaskState.RUNNING).when(srcTask1).getState();
+    doReturn(srcTask1).when(srcVertex).getTask(srcTaskId1.getId());
+    when(appContext.getCurrentDAG().getVertex(any(String.class))).thenReturn(srcVertex);
+
+    String srcVertexName = "srcVertexName";
+    List<TezEvent> eventList = Lists.newLinkedList();
+
+
+    // First Attempt send event
+    TezTaskAttemptID srcTaskAttemptId11 = TezTaskAttemptID.getInstance(srcTaskId1, 1);
+    EventMetaData sourceInfo11 =
+        new EventMetaData(EventMetaData.EventProducerConsumerType.PROCESSOR, srcVertexName, null,
+            srcTaskAttemptId11);
+    InputInitializerEvent e1 = InputInitializerEvent.create("fakeVertex", "fakeInput", null);
+    TezEvent te1 = new TezEvent(e1, sourceInfo11);
+    eventList.add(te1);
+    initializerWrapper.handleInputInitializerEvents(eventList);
+
+    verify(initializer, never()).handleInputInitializerEvent(any(List.class));
+    eventList.clear();
+
+    // First attempt, Task success notification
+    initializerWrapper.onTaskSucceeded(srcVertexName, srcTaskId1, srcTaskAttemptId11.getId());
+    ArgumentCaptor<List> argumentCaptor = ArgumentCaptor.forClass(List.class);
+    verify(initializer, times(1)).handleInputInitializerEvent(argumentCaptor.capture());
+    List<InputInitializerEvent> invokedEvents = argumentCaptor.getValue();
+    assertEquals(1, invokedEvents.size());
+
+    reset(initializer);
+
+    // 2nd attempt send event
+    TezTaskAttemptID srcTaskAttemptId12 = TezTaskAttemptID.getInstance(srcTaskId1, 2);
+    EventMetaData sourceInfo12 =
+        new EventMetaData(EventMetaData.EventProducerConsumerType.PROCESSOR, srcVertexName, null,
+            srcTaskAttemptId12);
+    InputInitializerEvent e2 = InputInitializerEvent.create("fakeVertex", "fakeInput", null);
+    TezEvent te2 = new TezEvent(e2, sourceInfo12);
+    eventList.add(te2);
+    initializerWrapper.handleInputInitializerEvents(eventList);
+
+    verify(initializer, never()).handleInputInitializerEvent(any(List.class));
+    eventList.clear();
+    reset(initializer);
+
+    // 2nd attempt succeeded
+    initializerWrapper.onTaskSucceeded(srcVertexName, srcTaskId1, srcTaskAttemptId12.getId());
+    verify(initializer, never()).handleInputInitializerEvent(argumentCaptor.capture());
+  }
+
+  // Order event1 success1, success2, event2
+  // Primarily a failure scenario, when a Task moves back to running from success
+  @SuppressWarnings("unchecked")
+  @Test(timeout = 5000)
+  public void testSuccessBeforeEvent() throws Exception {
+    InputDescriptor id = mock(InputDescriptor.class);
+    InputInitializerDescriptor iid = mock(InputInitializerDescriptor.class);
+    RootInputLeafOutput<InputDescriptor, InputInitializerDescriptor> rootInput =
+        new RootInputLeafOutput<InputDescriptor, InputInitializerDescriptor>("InputName", id, iid);
+
+    InputInitializer initializer = mock(InputInitializer.class);
+    InputInitializerContext initializerContext = mock(InputInitializerContext.class);
+    Vertex vertex = mock(Vertex.class);
+    StateChangeNotifier stateChangeNotifier = mock(StateChangeNotifier.class);
+    AppContext appContext = mock(AppContext.class, RETURNS_DEEP_STUBS);
+
+    RootInputInitializerManager.InitializerWrapper initializerWrapper =
+        new RootInputInitializerManager.InitializerWrapper(rootInput, initializer,
+            initializerContext, vertex, stateChangeNotifier, appContext);
+
+    ApplicationId appId = ApplicationId.newInstance(1000, 1);
+    TezDAGID dagId = TezDAGID.getInstance(appId, 1);
+    TezVertexID srcVertexId = TezVertexID.getInstance(dagId, 2);
+    TezTaskID srcTaskId1 = TezTaskID.getInstance(srcVertexId, 3);
+    Vertex srcVertex = mock(Vertex.class);
+    Task srcTask1 = mock(Task.class);
+    doReturn(TaskState.RUNNING).when(srcTask1).getState();
+    doReturn(srcTask1).when(srcVertex).getTask(srcTaskId1.getId());
+    when(appContext.getCurrentDAG().getVertex(any(String.class))).thenReturn(srcVertex);
+
+    String srcVertexName = "srcVertexName";
+    List<TezEvent> eventList = Lists.newLinkedList();
+
+
+    // First Attempt send event
+    TezTaskAttemptID srcTaskAttemptId11 = TezTaskAttemptID.getInstance(srcTaskId1, 1);
+    EventMetaData sourceInfo11 =
+        new EventMetaData(EventMetaData.EventProducerConsumerType.PROCESSOR, srcVertexName, null,
+            srcTaskAttemptId11);
+    InputInitializerEvent e1 = InputInitializerEvent.create("fakeVertex", "fakeInput", null);
+    TezEvent te1 = new TezEvent(e1, sourceInfo11);
+    eventList.add(te1);
+    initializerWrapper.handleInputInitializerEvents(eventList);
+
+    verify(initializer, never()).handleInputInitializerEvent(any(List.class));
+    eventList.clear();
+
+    // First attempt, Task success notification
+    initializerWrapper.onTaskSucceeded(srcVertexName, srcTaskId1, srcTaskAttemptId11.getId());
+    ArgumentCaptor<List> argumentCaptor = ArgumentCaptor.forClass(List.class);
+    verify(initializer, times(1)).handleInputInitializerEvent(argumentCaptor.capture());
+    List<InputInitializerEvent> invokedEvents = argumentCaptor.getValue();
+    assertEquals(1, invokedEvents.size());
+
+    reset(initializer);
+
+
+    TezTaskAttemptID srcTaskAttemptId12 = TezTaskAttemptID.getInstance(srcTaskId1, 2);
+    // 2nd attempt succeeded
+    initializerWrapper.onTaskSucceeded(srcVertexName, srcTaskId1, srcTaskAttemptId12.getId());
+    verify(initializer, never()).handleInputInitializerEvent(any(List.class));
+
+    // 2nd attempt send event
+    EventMetaData sourceInfo12 =
+        new EventMetaData(EventMetaData.EventProducerConsumerType.PROCESSOR, srcVertexName, null,
+            srcTaskAttemptId12);
+    InputInitializerEvent e2 = InputInitializerEvent.create("fakeVertex", "fakeInput", null);
+    TezEvent te2 = new TezEvent(e2, sourceInfo12);
+    eventList.add(te2);
+    initializerWrapper.handleInputInitializerEvents(eventList);
+
+    verify(initializer, never()).handleInputInitializerEvent(any(List.class));
+  }
+}


[33/50] [abbrv] tez git commit: TEZ-2006. Task communication plane needs to be pluggable. (sseth)

Posted by ss...@apache.org.
TEZ-2006. Task communication plane needs to be pluggable. (sseth)


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

Branch: refs/heads/TEZ-2003
Commit: 56986504695e995a87c8923a9713002ab1380468
Parents: 0fc4c69
Author: Siddharth Seth <ss...@apache.org>
Authored: Thu Feb 12 11:25:45 2015 -0800
Committer: Siddharth Seth <ss...@apache.org>
Committed: Wed May 6 00:13:29 2015 -0700

----------------------------------------------------------------------
 TEZ-2003-CHANGES.txt                            |   1 +
 .../apache/tez/dag/api/TaskCommunicator.java    |  54 ++
 .../tez/dag/api/TaskCommunicatorContext.java    |  48 ++
 .../tez/dag/api/TaskHeartbeatRequest.java       |  63 +++
 .../tez/dag/api/TaskHeartbeatResponse.java      |  39 ++
 .../java/org/apache/tez/dag/app/AppContext.java |   3 +
 .../org/apache/tez/dag/app/DAGAppMaster.java    |   5 +
 .../dag/app/TaskAttemptListenerImpTezDag.java   | 517 +++++++------------
 .../tez/dag/app/TezTaskCommunicatorImpl.java    | 474 +++++++++++++++++
 .../app/launcher/LocalContainerLauncher.java    |  10 +-
 .../tez/dag/app/rm/container/AMContainer.java   |   3 +-
 .../rm/container/AMContainerEventAssignTA.java  |   2 +
 .../dag/app/rm/container/AMContainerImpl.java   |   1 +
 .../apache/tez/dag/app/MockDAGAppMaster.java    |  25 +-
 .../app/TestTaskAttemptListenerImplTezDag.java  |  82 +--
 15 files changed, 965 insertions(+), 362 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/56986504/TEZ-2003-CHANGES.txt
----------------------------------------------------------------------
diff --git a/TEZ-2003-CHANGES.txt b/TEZ-2003-CHANGES.txt
index 1822fcb..d7e4be5 100644
--- a/TEZ-2003-CHANGES.txt
+++ b/TEZ-2003-CHANGES.txt
@@ -1,4 +1,5 @@
 ALL CHANGES:
   TEZ-2019. Temporarily allow the scheduler and launcher to be specified via configuration.
+  TEZ-2006. Task communication plane needs to be pluggable.
 
 INCOMPATIBLE CHANGES:

http://git-wip-us.apache.org/repos/asf/tez/blob/56986504/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicator.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicator.java b/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicator.java
new file mode 100644
index 0000000..97f9c16
--- /dev/null
+++ b/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicator.java
@@ -0,0 +1,54 @@
+/*
+ * 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.
+ */
+
+package org.apache.tez.dag.api;
+
+import java.net.InetSocketAddress;
+import java.util.Map;
+
+import org.apache.hadoop.security.Credentials;
+import org.apache.hadoop.service.AbstractService;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.LocalResource;
+import org.apache.tez.dag.records.TezTaskAttemptID;
+import org.apache.tez.runtime.api.impl.TaskSpec;
+
+// TODO TEZ-2003 Move this into the tez-api module
+public abstract class TaskCommunicator extends AbstractService {
+  public TaskCommunicator(String name) {
+    super(name);
+  }
+
+  // TODO TEZ-2003 Ideally, don't expose YARN containerId; instead expose a Tez specific construct.
+  // TODO When talking to an external service, this plugin implementer may need access to a host:port
+  public abstract void registerRunningContainer(ContainerId containerId, String hostname, int port);
+
+  // TODO TEZ-2003 Ideally, don't expose YARN containerId; instead expose a Tez specific construct.
+  public abstract void registerContainerEnd(ContainerId containerId);
+
+  // TODO TEZ-2003 TaskSpec breakup into a clean interface
+  // TODO TEZ-2003 Add support for priority
+  public abstract void registerRunningTaskAttempt(ContainerId containerId, TaskSpec taskSpec,
+                                                  Map<String, LocalResource> additionalResources,
+                                                  Credentials credentials,
+                                                  boolean credentialsChanged);
+
+  // TODO TEZ-2003 Remove reference to TaskAttemptID
+  public abstract void unregisterRunningTaskAttempt(TezTaskAttemptID taskAttemptID);
+
+  // TODO TEZ-2003 This doesn't necessarily belong here. A server may not start within the AM.
+  public abstract InetSocketAddress getAddress();
+
+  // TODO Eventually. Add methods here to support preemption of tasks.
+}

http://git-wip-us.apache.org/repos/asf/tez/blob/56986504/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicatorContext.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicatorContext.java b/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicatorContext.java
new file mode 100644
index 0000000..9b2d889
--- /dev/null
+++ b/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicatorContext.java
@@ -0,0 +1,48 @@
+/*
+ * 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.
+ */
+
+package org.apache.tez.dag.api;
+
+import java.io.IOException;
+
+import org.apache.hadoop.security.Credentials;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.tez.dag.records.TezTaskAttemptID;
+
+
+// Do not make calls into this from within a held lock.
+
+// TODO TEZ-2003 Move this into the tez-api module
+public interface TaskCommunicatorContext {
+
+  // TODO TEZ-2003 Add signalling back into this to indicate errors - e.g. Container unregsitered, task no longer running, etc.
+
+  // TODO TEZ-2003 Maybe add book-keeping as a helper library, instead of each impl tracking container to task etc.
+
+  ApplicationAttemptId getApplicationAttemptId();
+  Credentials getCredentials();
+
+  // TODO TEZ-2003 Move to vertex, taskIndex, version
+  boolean canCommit(TezTaskAttemptID taskAttemptId) throws IOException;
+
+  TaskHeartbeatResponse heartbeat(TaskHeartbeatRequest request) throws IOException, TezException;
+
+  boolean isKnownContainer(ContainerId containerId);
+
+  // TODO TEZ-2003 Move to vertex, taskIndex, version
+  void taskStartedRemotely(TezTaskAttemptID taskAttemptID, ContainerId containerId);
+
+  // TODO Eventually Add methods to report availability stats to the scheduler.
+}

http://git-wip-us.apache.org/repos/asf/tez/blob/56986504/tez-dag/src/main/java/org/apache/tez/dag/api/TaskHeartbeatRequest.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/api/TaskHeartbeatRequest.java b/tez-dag/src/main/java/org/apache/tez/dag/api/TaskHeartbeatRequest.java
new file mode 100644
index 0000000..f6bc8f0
--- /dev/null
+++ b/tez-dag/src/main/java/org/apache/tez/dag/api/TaskHeartbeatRequest.java
@@ -0,0 +1,63 @@
+/*
+ * 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.
+ */
+
+package org.apache.tez.dag.api;
+
+import java.util.List;
+
+import org.apache.tez.dag.records.TezTaskAttemptID;
+import org.apache.tez.runtime.api.impl.TezEvent;
+
+// TODO TEZ-2003 Move this into the tez-api module
+public class TaskHeartbeatRequest {
+
+  // TODO TEZ-2003 Ideally containerIdentifier should not be part of the request.
+  // Replace with a task lookup - vertex name + task index
+  private final String containerIdentifier;
+  // TODO TEZ-2003 Get rid of the task attemptId reference if possible
+  private final TezTaskAttemptID taskAttemptId;
+  private final List<TezEvent> events;
+  private final int startIndex;
+  private final int maxEvents;
+
+
+  public TaskHeartbeatRequest(String containerIdentifier, TezTaskAttemptID taskAttemptId, List<TezEvent> events, int startIndex,
+                              int maxEvents) {
+    this.containerIdentifier = containerIdentifier;
+    this.taskAttemptId = taskAttemptId;
+    this.events = events;
+    this.startIndex = startIndex;
+    this.maxEvents = maxEvents;
+  }
+
+  public String getContainerIdentifier() {
+    return containerIdentifier;
+  }
+
+  public TezTaskAttemptID getTaskAttemptId() {
+    return taskAttemptId;
+  }
+
+  public List<TezEvent> getEvents() {
+    return events;
+  }
+
+  public int getStartIndex() {
+    return startIndex;
+  }
+
+  public int getMaxEvents() {
+    return maxEvents;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tez/blob/56986504/tez-dag/src/main/java/org/apache/tez/dag/api/TaskHeartbeatResponse.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/api/TaskHeartbeatResponse.java b/tez-dag/src/main/java/org/apache/tez/dag/api/TaskHeartbeatResponse.java
new file mode 100644
index 0000000..c82a743
--- /dev/null
+++ b/tez-dag/src/main/java/org/apache/tez/dag/api/TaskHeartbeatResponse.java
@@ -0,0 +1,39 @@
+/*
+ * 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.
+ */
+
+package org.apache.tez.dag.api;
+
+import java.util.List;
+
+import org.apache.tez.runtime.api.impl.TezEvent;
+
+// TODO TEZ-2003 Move this into the tez-api module
+public class TaskHeartbeatResponse {
+
+  private final boolean shouldDie;
+  private List<TezEvent> events;
+
+  public TaskHeartbeatResponse(boolean shouldDie, List<TezEvent> events) {
+    this.shouldDie = shouldDie;
+    this.events = events;
+  }
+
+  public boolean isShouldDie() {
+    return shouldDie;
+  }
+
+  public List<TezEvent> getEvents() {
+    return events;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tez/blob/56986504/tez-dag/src/main/java/org/apache/tez/dag/app/AppContext.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/AppContext.java b/tez-dag/src/main/java/org/apache/tez/dag/app/AppContext.java
index 4781784..37f7624 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/AppContext.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/AppContext.java
@@ -24,6 +24,7 @@ import java.util.Set;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
@@ -106,4 +107,6 @@ public interface AppContext {
   String[] getLocalDirs();
 
   String getAMUser();
+
+  Credentials getAppCredentials();
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/56986504/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 73ee56e..bfc2d58 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
@@ -1486,6 +1486,11 @@ public class DAGAppMaster extends AbstractService {
     }
 
     @Override
+    public Credentials getAppCredentials() {
+      return amCredentials;
+    }
+
+    @Override
     public Map<ApplicationAccessType, String> getApplicationACLs() {
       if (getServiceState() != STATE.STARTED) {
         throw new TezUncheckedException(

http://git-wip-us.apache.org/repos/asf/tez/blob/56986504/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 d96da83..c34723a 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
@@ -18,15 +18,14 @@
 package org.apache.tez.dag.app;
 
 import java.io.IOException;
-import java.net.InetAddress;
+import java.lang.reflect.Constructor;
+import java.lang.reflect.InvocationTargetException;
 import java.net.InetSocketAddress;
 import java.net.URISyntaxException;
 import java.net.UnknownHostException;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 
@@ -38,216 +37,212 @@ import org.apache.tez.runtime.api.events.TaskStatusUpdateEvent;
 import org.apache.tez.runtime.api.impl.EventType;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
-import org.apache.hadoop.ipc.ProtocolSignature;
-import org.apache.hadoop.ipc.RPC;
-import org.apache.hadoop.ipc.Server;
-import org.apache.hadoop.net.NetUtils;
-import org.apache.hadoop.security.authorize.PolicyProvider;
+import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.service.AbstractService;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.tez.common.ReflectionUtils;
+import org.apache.tez.dag.api.TaskCommunicator;
+import org.apache.tez.dag.api.TaskCommunicatorContext;
+import org.apache.tez.dag.api.TaskHeartbeatResponse;
+import org.apache.tez.dag.api.TezConfiguration;
 import org.apache.tez.dag.api.TezException;
 import org.apache.tez.dag.api.TezUncheckedException;
 import org.apache.hadoop.yarn.api.records.ContainerId;
-import org.apache.hadoop.yarn.api.records.LocalResource;
 import org.apache.hadoop.yarn.util.ConverterUtils;
-import org.apache.tez.common.ContainerContext;
-import org.apache.tez.common.ContainerTask;
-import org.apache.tez.common.TezConverterUtils;
-import org.apache.tez.common.TezLocalResource;
-import org.apache.tez.common.TezTaskUmbilicalProtocol;
-import org.apache.tez.dag.api.TezConfiguration;
+import org.apache.tez.dag.api.TaskHeartbeatRequest;
 import org.apache.tez.dag.app.dag.DAG;
 import org.apache.tez.dag.app.dag.Task;
 import org.apache.tez.dag.app.dag.event.TaskAttemptEventStartedRemotely;
 import org.apache.tez.dag.app.dag.event.VertexEventRouteEvent;
+import org.apache.tez.dag.app.rm.TaskSchedulerService;
 import org.apache.tez.dag.app.rm.container.AMContainerTask;
-import org.apache.tez.dag.app.security.authorize.TezAMPolicyProvider;
 import org.apache.tez.dag.records.TezTaskAttemptID;
 import org.apache.tez.dag.records.TezVertexID;
 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.common.security.JobTokenSecretManager;
 
-import com.google.common.collect.Maps;
 
 @SuppressWarnings("unchecked")
+@InterfaceAudience.Private
 public class TaskAttemptListenerImpTezDag extends AbstractService implements
-    TezTaskUmbilicalProtocol, TaskAttemptListener {
-
-  private static final ContainerTask TASK_FOR_INVALID_JVM = new ContainerTask(
-      null, true, null, null, false);
+    TaskAttemptListener, TaskCommunicatorContext {
 
   private static final Logger LOG = LoggerFactory
       .getLogger(TaskAttemptListenerImpTezDag.class);
 
   private final AppContext context;
+  private TaskCommunicator taskCommunicator;
 
   protected final TaskHeartbeatHandler taskHeartbeatHandler;
   protected final ContainerHeartbeatHandler containerHeartbeatHandler;
-  private final JobTokenSecretManager jobTokenSecretManager;
-  private InetSocketAddress address;
-  private Server server;
-
-  static class ContainerInfo {
-    ContainerInfo() {
-      this.lastReponse = null;
-      this.lastRequestId = 0;
-      this.amContainerTask = null;
-      this.taskPulled = false;
+
+  private final TaskHeartbeatResponse RESPONSE_SHOULD_DIE = new TaskHeartbeatResponse(true, null);
+
+  private final ConcurrentMap<TezTaskAttemptID, ContainerId> registeredAttempts =
+      new ConcurrentHashMap<TezTaskAttemptID, ContainerId>();
+  private final ConcurrentMap<ContainerId, ContainerInfo> registeredContainers =
+      new ConcurrentHashMap<ContainerId, ContainerInfo>();
+
+  // Defined primarily to work around ConcurrentMaps not accepting null values
+  private static final class ContainerInfo {
+    TezTaskAttemptID taskAttemptId;
+    ContainerInfo(TezTaskAttemptID taskAttemptId) {
+      this.taskAttemptId = taskAttemptId;
     }
-    long lastRequestId;
-    TezHeartbeatResponse lastReponse;
-    AMContainerTask amContainerTask;
-    boolean taskPulled;
   }
 
-  private ConcurrentMap<TezTaskAttemptID, ContainerId> attemptToInfoMap =
-      new ConcurrentHashMap<TezTaskAttemptID, ContainerId>();
+  private static final ContainerInfo NULL_CONTAINER_INFO = new ContainerInfo(null);
 
-  private ConcurrentHashMap<ContainerId, ContainerInfo> registeredContainers =
-      new ConcurrentHashMap<ContainerId, ContainerInfo>();
 
   public TaskAttemptListenerImpTezDag(AppContext context,
-      TaskHeartbeatHandler thh, ContainerHeartbeatHandler chh,
-      JobTokenSecretManager jobTokenSecretManager) {
+                                      TaskHeartbeatHandler thh, ContainerHeartbeatHandler chh,
+                                      // TODO TEZ-2003 pre-merge. Remove reference to JobTokenSecretManager.
+                                      JobTokenSecretManager jobTokenSecretManager) {
     super(TaskAttemptListenerImpTezDag.class.getName());
     this.context = context;
-    this.jobTokenSecretManager = jobTokenSecretManager;
     this.taskHeartbeatHandler = thh;
     this.containerHeartbeatHandler = chh;
+    this.taskCommunicator = new TezTaskCommunicatorImpl(this);
   }
 
   @Override
-  public void serviceStart() {
-    startRpcServer();
-  }
-
-  protected void startRpcServer() {
-    Configuration conf = getConfig();
-    if (!conf.getBoolean(TezConfiguration.TEZ_LOCAL_MODE, TezConfiguration.TEZ_LOCAL_MODE_DEFAULT)) {
-      try {
-        server = new RPC.Builder(conf)
-            .setProtocol(TezTaskUmbilicalProtocol.class)
-            .setBindAddress("0.0.0.0")
-            .setPort(0)
-            .setInstance(this)
-            .setNumHandlers(
-                conf.getInt(TezConfiguration.TEZ_AM_TASK_LISTENER_THREAD_COUNT,
-                    TezConfiguration.TEZ_AM_TASK_LISTENER_THREAD_COUNT_DEFAULT))
-            .setSecretManager(jobTokenSecretManager).build();
-
-        // Enable service authorization?
-        if (conf.getBoolean(
-            CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHORIZATION,
-            false)) {
-          refreshServiceAcls(conf, new TezAMPolicyProvider());
-        }
-
-        server.start();
-        this.address = NetUtils.getConnectAddress(server);
-      } catch (IOException e) {
-        throw new TezUncheckedException(e);
-      }
+  public void serviceInit(Configuration conf) {
+    String taskCommClassName = conf.get(TezConfiguration.TEZ_AM_TASK_COMMUNICATOR_CLASS);
+    if (taskCommClassName == null) {
+      LOG.info("Using Default Task Communicator");
+      this.taskCommunicator = new TezTaskCommunicatorImpl(this);
     } else {
+      LOG.info("Using TaskCommunicator: " + taskCommClassName);
+      Class<? extends TaskCommunicator> taskCommClazz = (Class<? extends TaskCommunicator>) ReflectionUtils
+          .getClazz(taskCommClassName);
       try {
-        this.address = new InetSocketAddress(InetAddress.getLocalHost(), 0);
-      } catch (UnknownHostException e) {
+        Constructor<? extends TaskCommunicator> ctor = taskCommClazz.getConstructor(TaskCommunicatorContext.class);
+        ctor.setAccessible(true);
+        this.taskCommunicator = ctor.newInstance(this);
+      } catch (NoSuchMethodException e) {
+        throw new TezUncheckedException(e);
+      } catch (InvocationTargetException e) {
+        throw new TezUncheckedException(e);
+      } catch (InstantiationException e) {
+        throw new TezUncheckedException(e);
+      } catch (IllegalAccessException e) {
         throw new TezUncheckedException(e);
-      }
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Not starting TaskAttemptListener RPC in LocalMode");
       }
     }
   }
 
-  void refreshServiceAcls(Configuration configuration,
-      PolicyProvider policyProvider) {
-    this.server.refreshServiceAcl(configuration, policyProvider);
+  @Override
+  public void serviceStart() {
+    taskCommunicator.init(getConfig());
+    taskCommunicator.start();
   }
 
   @Override
   public void serviceStop() {
-    stopRpcServer();
-  }
-
-  protected void stopRpcServer() {
-    if (server != null) {
-      server.stop();
+    if (taskCommunicator != null) {
+      taskCommunicator.stop();
+      taskCommunicator = null;
     }
   }
 
-  public InetSocketAddress getAddress() {
-    return address;
-  }
-
   @Override
-  public long getProtocolVersion(String protocol, long clientVersion)
-      throws IOException {
-    return versionID;
+  public ApplicationAttemptId getApplicationAttemptId() {
+    return context.getApplicationAttemptId();
   }
 
   @Override
-  public ProtocolSignature getProtocolSignature(String protocol,
-      long clientVersion, int clientMethodsHash) throws IOException {
-    return ProtocolSignature.getProtocolSignature(this, protocol,
-        clientVersion, clientMethodsHash);
+  public Credentials getCredentials() {
+    return context.getAppCredentials();
   }
 
   @Override
-  public ContainerTask getTask(ContainerContext containerContext)
-      throws IOException {
+  public TaskHeartbeatResponse heartbeat(TaskHeartbeatRequest request)
+      throws IOException, TezException {
+    ContainerId containerId = ConverterUtils.toContainerId(request
+        .getContainerIdentifier());
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Received heartbeat from container"
+          + ", request=" + request);
+    }
 
-    ContainerTask task = null;
+    if (!registeredContainers.containsKey(containerId)) {
+      LOG.warn("Received task heartbeat from unknown container with id: " + containerId +
+          ", asking it to die");
+      return RESPONSE_SHOULD_DIE;
+    }
 
-    if (containerContext == null || containerContext.getContainerIdentifier() == null) {
-      LOG.info("Invalid task request with an empty containerContext or containerId");
-      task = TASK_FOR_INVALID_JVM;
-    } else {
-      ContainerId containerId = ConverterUtils.toContainerId(containerContext
-          .getContainerIdentifier());
+    // A heartbeat can come in anytime. The AM may have made a decision to kill a running task/container
+    // meanwhile. If the decision is processed through the pipeline before the heartbeat is processed,
+    // the heartbeat will be dropped. Otherwise the heartbeat will be processed - and the system
+    // know how to handle this - via FailedInputEvents for example (relevant only if the heartbeat has events).
+    // So - avoiding synchronization.
+
+    pingContainerHeartbeatHandler(containerId);
+    List<TezEvent> outEvents = null;
+    TezTaskAttemptID taskAttemptID = request.getTaskAttemptId();
+    if (taskAttemptID != null) {
+      ContainerId containerIdFromMap = registeredAttempts.get(taskAttemptID);
+      if (containerIdFromMap == null || !containerIdFromMap.equals(containerId)) {
+        // This can happen when a task heartbeats. Meanwhile the container is unregistered.
+        // The information will eventually make it through to the plugin via a corresponding unregister.
+        // There's a race in that case between the unregister making it through, and this method returning.
+        // TODO TEZ-2003. An exception back is likely a better approach than sending a shouldDie = true,
+        // so that the plugin can handle the scenario. Alternately augment the response with error codes.
+        // Error codes would be better than exceptions.
+        LOG.info("Attempt: " + taskAttemptID + " is not recognized for heartbeats");
+        return RESPONSE_SHOULD_DIE;
+      }
+
+      List<TezEvent> inEvents = request.getEvents();
       if (LOG.isDebugEnabled()) {
-        LOG.debug("Container with id: " + containerId + " asked for a task");
+        LOG.debug("Ping from " + taskAttemptID.toString() +
+            " events: " + (inEvents != null ? inEvents.size() : -1));
       }
-      if (!registeredContainers.containsKey(containerId)) {
-        if(context.getAllContainers().get(containerId) == null) {
-          LOG.info("Container with id: " + containerId
-              + " is invalid and will be killed");
-        } else {
-          LOG.info("Container with id: " + containerId
-              + " is valid, but no longer registered, and will be killed");
-        }
-        task = TASK_FOR_INVALID_JVM;
-      } else {
-        pingContainerHeartbeatHandler(containerId);
-        task = getContainerTask(containerId);
-        if (task == null) {
-          if (LOG.isDebugEnabled()) {
-            LOG.debug("No task current assigned to Container with id: " + containerId);
-          }
-        } else if (task == TASK_FOR_INVALID_JVM) { 
-          LOG.info("Container with id: " + containerId
-              + " is valid, but no longer registered, and will be killed. Race condition.");          
+
+      List<TezEvent> otherEvents = new ArrayList<TezEvent>();
+      for (TezEvent tezEvent : ListUtils.emptyIfNull(inEvents)) {
+        final EventType eventType = tezEvent.getEventType();
+        if (eventType == EventType.TASK_STATUS_UPDATE_EVENT ||
+            eventType == EventType.TASK_ATTEMPT_COMPLETED_EVENT) {
+          context.getEventHandler()
+              .handle(getTaskAttemptEventFromTezEvent(taskAttemptID, tezEvent));
         } else {
-          context.getEventHandler().handle(
-              new TaskAttemptEventStartedRemotely(task.getTaskSpec()
-                  .getTaskAttemptID(), containerId, context
-                  .getApplicationACLs()));
-          LOG.info("Container with id: " + containerId + " given task: "
-              + task.getTaskSpec().getTaskAttemptID());
+          otherEvents.add(tezEvent);
         }
       }
+      if(!otherEvents.isEmpty()) {
+        TezVertexID vertexId = taskAttemptID.getTaskID().getVertexID();
+        context.getEventHandler().handle(
+            new VertexEventRouteEvent(vertexId, Collections.unmodifiableList(otherEvents)));
+      }
+      taskHeartbeatHandler.pinged(taskAttemptID);
+      outEvents = context
+          .getCurrentDAG()
+          .getVertex(taskAttemptID.getTaskID().getVertexID())
+          .getTask(taskAttemptID.getTaskID())
+          .getTaskAttemptTezEvents(taskAttemptID, request.getStartIndex(),
+              request.getMaxEvents());
     }
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("getTask returning task: " + task);
-    }
-    return task;
+    return new TaskHeartbeatResponse(false, outEvents);
+  }
+
+  @Override
+  public boolean isKnownContainer(ContainerId containerId) {
+    return context.getAllContainers().get(containerId) != null;
+  }
+
+  @Override
+  public void taskStartedRemotely(TezTaskAttemptID taskAttemptID, ContainerId containerId) {
+    context.getEventHandler().handle(new TaskAttemptEventStartedRemotely(taskAttemptID, containerId, null));
+    pingContainerHeartbeatHandler(containerId);
   }
 
   /**
    * Child checking whether it can commit.
-   *
+   * <p/>
    * <br/>
    * Repeatedly polls the ApplicationMaster whether it
    * {@link Task#canCommit(TezTaskAttemptID)} This is * a legacy from the
@@ -270,25 +265,12 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
   }
 
   @Override
-  public void unregisterTaskAttempt(TezTaskAttemptID attemptId) {
-    ContainerId containerId = attemptToInfoMap.get(attemptId);
-    if(containerId == null) {
-      LOG.warn("Unregister task attempt: " + attemptId + " from unknown container");
-      return;
-    }
-    ContainerInfo containerInfo = registeredContainers.get(containerId);
-    if(containerInfo == null) {
-      LOG.warn("Unregister task attempt: " + attemptId +
-          " from non-registered container: " + containerId);
-      return;
-    }
-    synchronized (containerInfo) {
-      containerInfo.amContainerTask = null;
-      attemptToInfoMap.remove(attemptId);
-    }
-
+  public InetSocketAddress getAddress() {
+    return taskCommunicator.getAddress();
   }
 
+  // The TaskAttemptListener register / unregister methods in this class are not thread safe.
+  // The Tez framework should not invoke these methods from multiple threads.
   @Override
   public void dagComplete(DAG dag) {
     // TODO TEZ-2335. Cleanup TaskHeartbeat handler structures.
@@ -308,50 +290,82 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
   @Override
   public void registerRunningContainer(ContainerId containerId) {
     if (LOG.isDebugEnabled()) {
-      LOG.debug("ContainerId: " + containerId
-          + " registered with TaskAttemptListener");
+      LOG.debug("ContainerId: " + containerId + " registered with TaskAttemptListener");
     }
-    ContainerInfo oldInfo = registeredContainers.put(containerId, new ContainerInfo());
-    if(oldInfo != null) {
+    ContainerInfo oldInfo = registeredContainers.put(containerId, NULL_CONTAINER_INFO);
+    if (oldInfo != null) {
       throw new TezUncheckedException(
           "Multiple registrations for containerId: " + containerId);
     }
+    NodeId nodeId = context.getAllContainers().get(containerId).getContainer().getNodeId();
+    taskCommunicator.registerRunningContainer(containerId, nodeId.getHost(), nodeId.getPort());
+  }
+
+  @Override
+  public void unregisterRunningContainer(ContainerId containerId) {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Unregistering Container from TaskAttemptListener: " + containerId);
+    }
+    ContainerInfo containerInfo = registeredContainers.remove(containerId);
+    if (containerInfo.taskAttemptId != null) {
+      registeredAttempts.remove(containerInfo.taskAttemptId);
+    }
+    taskCommunicator.registerContainerEnd(containerId);
   }
 
   @Override
   public void registerTaskAttempt(AMContainerTask amContainerTask,
-      ContainerId containerId) {
+                                  ContainerId containerId) {
     ContainerInfo containerInfo = registeredContainers.get(containerId);
-    if(containerInfo == null) {
+    if (containerInfo == null) {
       throw new TezUncheckedException("Registering task attempt: "
           + amContainerTask.getTask().getTaskAttemptID() + " to unknown container: " + containerId);
     }
-    synchronized (containerInfo) {
-      if(containerInfo.amContainerTask != null) {
-        throw new TezUncheckedException("Registering task attempt: "
-            + amContainerTask.getTask().getTaskAttemptID() + " to container: " + containerId
-            + " with existing assignment to: " + containerInfo.amContainerTask.getTask().getTaskAttemptID());
-      }
-      containerInfo.amContainerTask = amContainerTask;
-      containerInfo.taskPulled = false;
-
-      ContainerId containerIdFromMap =
-          attemptToInfoMap.put(amContainerTask.getTask().getTaskAttemptID(), containerId);
-      if(containerIdFromMap != null) {
-        throw new TezUncheckedException("Registering task attempt: "
-            + amContainerTask.getTask().getTaskAttemptID() + " to container: " + containerId
-            + " when already assigned to: " + containerIdFromMap);
-      }
+    if (containerInfo.taskAttemptId != null) {
+      throw new TezUncheckedException("Registering task attempt: "
+          + amContainerTask.getTask().getTaskAttemptID() + " to container: " + containerId
+          + " with existing assignment to: " +
+          containerInfo.taskAttemptId);
     }
+
+    if (containerInfo.taskAttemptId != null) {
+      throw new TezUncheckedException("Registering task attempt: "
+          + amContainerTask.getTask().getTaskAttemptID() + " to container: " + containerId
+          + " with existing assignment to: " +
+          containerInfo.taskAttemptId);
+    }
+
+    // Explicitly putting in a new entry so that synchronization is not required on the existing element in the map.
+    registeredContainers.put(containerId, new ContainerInfo(amContainerTask.getTask().getTaskAttemptID()));
+
+    ContainerId containerIdFromMap = registeredAttempts.put(
+        amContainerTask.getTask().getTaskAttemptID(), containerId);
+    if (containerIdFromMap != null) {
+      throw new TezUncheckedException("Registering task attempt: "
+          + amContainerTask.getTask().getTaskAttemptID() + " to container: " + containerId
+          + " when already assigned to: " + containerIdFromMap);
+    }
+    taskCommunicator.registerRunningTaskAttempt(containerId, amContainerTask.getTask(),
+        amContainerTask.getAdditionalResources(), amContainerTask.getCredentials(),
+        amContainerTask.haveCredentialsChanged());
   }
 
   @Override
-  public void unregisterRunningContainer(ContainerId containerId) {
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Unregistering Container from TaskAttemptListener: "
-          + containerId);
+  public void unregisterTaskAttempt(TezTaskAttemptID attemptId) {
+    ContainerId containerId = registeredAttempts.remove(attemptId);
+    if (containerId == null) {
+      LOG.warn("Unregister task attempt: " + attemptId + " from unknown container");
+      return;
+    }
+    ContainerInfo containerInfo = registeredContainers.get(containerId);
+    if (containerInfo == null) {
+      LOG.warn("Unregister task attempt: " + attemptId +
+          " from non-registered container: " + containerId);
+      return;
     }
-    registeredContainers.remove(containerId);
+    // Explicitly putting in a new entry so that synchronization is not required on the existing element in the map.
+    registeredContainers.put(containerId, NULL_CONTAINER_INFO);
+    taskCommunicator.unregisterRunningTaskAttempt(attemptId);
   }
 
   private void pingContainerHeartbeatHandler(ContainerId containerId) {
@@ -359,7 +373,7 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
   }
 
   private void pingContainerHeartbeatHandler(TezTaskAttemptID taskAttemptId) {
-    ContainerId containerId = attemptToInfoMap.get(taskAttemptId);
+    ContainerId containerId = registeredAttempts.get(taskAttemptId);
     if (containerId != null) {
       containerHeartbeatHandler.pinged(containerId);
     } else {
@@ -368,91 +382,6 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
     }
   }
 
-  @Override
-  public TezHeartbeatResponse heartbeat(TezHeartbeatRequest request)
-      throws IOException, TezException {
-    ContainerId containerId = ConverterUtils.toContainerId(request
-        .getContainerIdentifier());
-    long requestId = request.getRequestId();
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Received heartbeat from container"
-          + ", request=" + request);
-    }
-
-    ContainerInfo containerInfo = registeredContainers.get(containerId);
-    if(containerInfo == null) {
-      LOG.warn("Received task heartbeat from unknown container with id: " + containerId +
-          ", asking it to die");
-      TezHeartbeatResponse response = new TezHeartbeatResponse();
-      response.setLastRequestId(requestId);
-      response.setShouldDie();
-      return response;
-    }
-
-    synchronized (containerInfo) {
-      pingContainerHeartbeatHandler(containerId);
-
-      if(containerInfo.lastRequestId == requestId) {
-        LOG.warn("Old sequenceId received: " + requestId
-            + ", Re-sending last response to client");
-        return containerInfo.lastReponse;
-      }
-
-      TezHeartbeatResponse response = new TezHeartbeatResponse();
-      response.setLastRequestId(requestId);
-
-      TezTaskAttemptID taskAttemptID = request.getCurrentTaskAttemptID();
-      if (taskAttemptID != null) {
-        ContainerId containerIdFromMap = attemptToInfoMap.get(taskAttemptID);
-        if(containerIdFromMap == null || !containerIdFromMap.equals(containerId)) {
-          throw new TezException("Attempt " + taskAttemptID
-            + " is not recognized for heartbeat");
-        }
-
-        if(containerInfo.lastRequestId+1 != requestId) {
-          throw new TezException("Container " + containerId
-              + " has invalid request id. Expected: "
-              + containerInfo.lastRequestId+1
-              + " and actual: " + requestId);
-        }
-
-        List<TezEvent> inEvents = request.getEvents();
-        if (LOG.isDebugEnabled()) {
-          LOG.debug("Ping from " + taskAttemptID.toString() +
-              " events: " + (inEvents != null? inEvents.size() : -1));
-        }
-
-        List<TezEvent> otherEvents = new ArrayList<TezEvent>();
-        for (TezEvent tezEvent : ListUtils.emptyIfNull(inEvents)) {
-          final EventType eventType = tezEvent.getEventType();
-          if (eventType == EventType.TASK_STATUS_UPDATE_EVENT ||
-              eventType == EventType.TASK_ATTEMPT_COMPLETED_EVENT) {
-            context.getEventHandler()
-                .handle(getTaskAttemptEventFromTezEvent(taskAttemptID, tezEvent));
-          } else {
-            otherEvents.add(tezEvent);
-          }
-        }
-        if(!otherEvents.isEmpty()) {
-          TezVertexID vertexId = taskAttemptID.getTaskID().getVertexID();
-          context.getEventHandler().handle(
-              new VertexEventRouteEvent(vertexId, Collections.unmodifiableList(otherEvents)));
-        }
-        taskHeartbeatHandler.pinged(taskAttemptID);
-        List<TezEvent> outEvents = context
-            .getCurrentDAG()
-            .getVertex(taskAttemptID.getTaskID().getVertexID())
-            .getTask(taskAttemptID.getTaskID())
-            .getTaskAttemptTezEvents(taskAttemptID, request.getStartIndex(),
-                request.getMaxEvents());
-        response.setEvents(outEvents);
-      }
-      containerInfo.lastRequestId = requestId;
-      containerInfo.lastReponse = response;
-      return response;
-    }
-  }
-
   private TaskAttemptEvent getTaskAttemptEventFromTezEvent(TezTaskAttemptID taskAttemptID,
                                                            TezEvent tezEvent) {
     final EventType eventType = tezEvent.getEventType();
@@ -475,51 +404,7 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
     return taskAttemptEvent;
   }
 
-  private Map<String, TezLocalResource> convertLocalResourceMap(Map<String, LocalResource> ylrs)
-      throws IOException {
-    Map<String, TezLocalResource> tlrs = Maps.newHashMap();
-    if (ylrs != null) {
-      for (Entry<String, LocalResource> ylrEntry : ylrs.entrySet()) {
-        TezLocalResource tlr;
-        try {
-          tlr = TezConverterUtils.convertYarnLocalResourceToTez(ylrEntry.getValue());
-        } catch (URISyntaxException e) {
-         throw new IOException(e);
-        }
-        tlrs.put(ylrEntry.getKey(), tlr);
-      }
-    }
-    return tlrs;
-  }
-
-  private ContainerTask getContainerTask(ContainerId containerId) throws IOException {
-    ContainerTask containerTask = null;
-    ContainerInfo containerInfo = registeredContainers.get(containerId);
-    if (containerInfo == null) {
-      // This can happen if an unregisterTask comes in after we've done the initial checks for
-      // registered containers. (Race between getTask from the container, and a potential STOP_CONTAINER
-      // from somewhere within the AM)
-      // Implies that an un-registration has taken place and the container needs to be asked to die.
-      LOG.info("Container with id: " + containerId
-          + " is valid, but no longer registered, and will be killed");
-      containerTask = TASK_FOR_INVALID_JVM;
-    } else {
-      synchronized (containerInfo) {
-        if (containerInfo.amContainerTask != null) {
-          if (!containerInfo.taskPulled) {
-            containerInfo.taskPulled = true;
-            AMContainerTask amContainerTask = containerInfo.amContainerTask;
-            containerTask = new ContainerTask(amContainerTask.getTask(), false,
-                convertLocalResourceMap(amContainerTask.getAdditionalResources()),
-                amContainerTask.getCredentials(), amContainerTask.haveCredentialsChanged());
-          } else {
-            containerTask = null;
-          }
-        } else {
-          containerTask = null;
-        }
-      }
-    }
-    return containerTask;
+  public TaskCommunicator getTaskCommunicator() {
+    return taskCommunicator;
   }
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/56986504/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java
new file mode 100644
index 0000000..5652937
--- /dev/null
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java
@@ -0,0 +1,474 @@
+/*
+ * 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.
+ */
+
+package org.apache.tez.dag.app;
+
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.net.URISyntaxException;
+import java.net.UnknownHostException;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Maps;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
+import org.apache.hadoop.ipc.ProtocolSignature;
+import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.ipc.Server;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.security.Credentials;
+import org.apache.hadoop.security.authorize.PolicyProvider;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.LocalResource;
+import org.apache.hadoop.yarn.util.ConverterUtils;
+import org.apache.tez.common.*;
+import org.apache.tez.common.ContainerContext;
+import org.apache.tez.common.security.JobTokenIdentifier;
+import org.apache.tez.common.security.JobTokenSecretManager;
+import org.apache.tez.common.security.TokenCache;
+import org.apache.tez.dag.api.TaskCommunicator;
+import org.apache.tez.dag.api.TaskCommunicatorContext;
+import org.apache.tez.dag.api.TaskHeartbeatRequest;
+import org.apache.tez.dag.api.TaskHeartbeatResponse;
+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.app.security.authorize.TezAMPolicyProvider;
+import org.apache.tez.dag.records.TezTaskAttemptID;
+import org.apache.tez.runtime.api.impl.TaskSpec;
+import org.apache.tez.runtime.api.impl.TezHeartbeatRequest;
+import org.apache.tez.runtime.api.impl.TezHeartbeatResponse;
+
+@InterfaceAudience.Private
+public class TezTaskCommunicatorImpl extends TaskCommunicator {
+
+  private static final Log LOG = LogFactory.getLog(TezTaskCommunicatorImpl.class);
+
+  private static final ContainerTask TASK_FOR_INVALID_JVM = new ContainerTask(
+      null, true, null, null, false);
+
+  private final TaskCommunicatorContext taskCommunicatorContext;
+
+  private final ConcurrentMap<ContainerId, ContainerInfo> registeredContainers =
+      new ConcurrentHashMap<ContainerId, ContainerInfo>();
+  private final ConcurrentMap<TaskAttempt, ContainerId> attemptToContainerMap =
+      new ConcurrentHashMap<TaskAttempt, ContainerId>();
+
+  private final TezTaskUmbilicalProtocol taskUmbilical;
+  private InetSocketAddress address;
+  private Server server;
+
+  private static final class ContainerInfo {
+
+    ContainerInfo(ContainerId containerId) {
+      this.containerId = containerId;
+    }
+
+    ContainerId containerId;
+    TezHeartbeatResponse lastResponse = null;
+    TaskSpec taskSpec = null;
+    long lastRequestId = 0;
+    Map<String, LocalResource> additionalLRs = null;
+    Credentials credentials = null;
+    boolean credentialsChanged = false;
+    boolean taskPulled = false;
+
+    void reset() {
+      taskSpec = null;
+      additionalLRs = null;
+      credentials = null;
+      credentialsChanged = false;
+      taskPulled = false;
+    }
+  }
+
+
+
+  /**
+   * Construct the service.
+   */
+  public TezTaskCommunicatorImpl(TaskCommunicatorContext taskCommunicatorContext) {
+    super(TezTaskCommunicatorImpl.class.getName());
+    this.taskCommunicatorContext = taskCommunicatorContext;
+    this.taskUmbilical = new TezTaskUmbilicalProtocolImpl();
+  }
+
+
+  @Override
+  public void serviceStart() {
+
+    startRpcServer();
+  }
+
+  @Override
+  public void serviceStop() {
+    stopRpcServer();
+  }
+
+  protected void startRpcServer() {
+    Configuration conf = getConfig();
+    if (!conf.getBoolean(TezConfiguration.TEZ_LOCAL_MODE, TezConfiguration.TEZ_LOCAL_MODE_DEFAULT)) {
+      try {
+        JobTokenSecretManager jobTokenSecretManager =
+            new JobTokenSecretManager();
+        Token<JobTokenIdentifier> sessionToken = TokenCache.getSessionToken(taskCommunicatorContext.getCredentials());
+        jobTokenSecretManager.addTokenForJob(
+            taskCommunicatorContext.getApplicationAttemptId().getApplicationId().toString(), sessionToken);
+
+        server = new RPC.Builder(conf)
+            .setProtocol(TezTaskUmbilicalProtocol.class)
+            .setBindAddress("0.0.0.0")
+            .setPort(0)
+            .setInstance(taskUmbilical)
+            .setNumHandlers(
+                conf.getInt(TezConfiguration.TEZ_AM_TASK_LISTENER_THREAD_COUNT,
+                    TezConfiguration.TEZ_AM_TASK_LISTENER_THREAD_COUNT_DEFAULT))
+            .setSecretManager(jobTokenSecretManager).build();
+
+        // Enable service authorization?
+        if (conf.getBoolean(
+            CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHORIZATION,
+            false)) {
+          refreshServiceAcls(conf, new TezAMPolicyProvider());
+        }
+
+        server.start();
+        this.address = NetUtils.getConnectAddress(server);
+      } catch (IOException e) {
+        throw new TezUncheckedException(e);
+      }
+    } else {
+      try {
+        this.address = new InetSocketAddress(InetAddress.getLocalHost(), 0);
+      } catch (UnknownHostException e) {
+        throw new TezUncheckedException(e);
+      }
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Not starting TaskAttemptListener RPC in LocalMode");
+      }
+    }
+  }
+
+  protected void stopRpcServer() {
+    if (server != null) {
+      server.stop();
+      server = null;
+    }
+  }
+
+  private void refreshServiceAcls(Configuration configuration,
+                                  PolicyProvider policyProvider) {
+    this.server.refreshServiceAcl(configuration, policyProvider);
+  }
+
+  @Override
+  public void registerRunningContainer(ContainerId containerId, String host, int port) {
+    ContainerInfo oldInfo = registeredContainers.putIfAbsent(containerId, new ContainerInfo(containerId));
+    if (oldInfo != null) {
+      throw new TezUncheckedException("Multiple registrations for containerId: " + containerId);
+    }
+  }
+
+  @Override
+  public void registerContainerEnd(ContainerId containerId) {
+    ContainerInfo containerInfo = registeredContainers.remove(containerId);
+    if (containerInfo != null) {
+      synchronized(containerInfo) {
+        if (containerInfo.taskSpec != null && containerInfo.taskSpec.getTaskAttemptID() != null) {
+          attemptToContainerMap.remove(containerInfo.taskSpec.getTaskAttemptID());
+        }
+      }
+    }
+  }
+
+  @Override
+  public void registerRunningTaskAttempt(ContainerId containerId, TaskSpec taskSpec,
+                                         Map<String, LocalResource> additionalResources,
+                                         Credentials credentials, boolean credentialsChanged) {
+
+    ContainerInfo containerInfo = registeredContainers.get(containerId);
+    Preconditions.checkNotNull(containerInfo,
+        "Cannot register task attempt: " + taskSpec.getTaskAttemptID() + " to unknown container: " +
+            containerId);
+    synchronized (containerInfo) {
+      if (containerInfo.taskSpec != null) {
+        throw new TezUncheckedException(
+            "Cannot register task: " + taskSpec.getTaskAttemptID() + " to container: " +
+                containerId + " , with pre-existing assignment: " +
+                containerInfo.taskSpec.getTaskAttemptID());
+      }
+      containerInfo.taskSpec = taskSpec;
+      containerInfo.additionalLRs = additionalResources;
+      containerInfo.credentials = credentials;
+      containerInfo.credentialsChanged = credentialsChanged;
+      containerInfo.taskPulled = false;
+
+      ContainerId oldId = attemptToContainerMap.putIfAbsent(new TaskAttempt(taskSpec.getTaskAttemptID()), containerId);
+      if (oldId != null) {
+        throw new TezUncheckedException(
+            "Attempting to register an already registered taskAttempt with id: " +
+                taskSpec.getTaskAttemptID() + " to containerId: " + containerId +
+                ". Already registered to containerId: " + oldId);
+      }
+    }
+
+  }
+
+  @Override
+  public void unregisterRunningTaskAttempt(TezTaskAttemptID taskAttemptID) {
+    TaskAttempt taskAttempt = new TaskAttempt(taskAttemptID);
+    ContainerId containerId = attemptToContainerMap.remove(taskAttempt);
+    if(containerId == null) {
+      LOG.warn("Unregister task attempt: " + taskAttempt + " from unknown container");
+      return;
+    }
+    ContainerInfo containerInfo = registeredContainers.get(containerId);
+    if (containerInfo == null) {
+      LOG.warn("Unregister task attempt: " + taskAttempt +
+          " from non-registered container: " + containerId);
+      return;
+    }
+    synchronized (containerInfo) {
+      containerInfo.reset();
+      attemptToContainerMap.remove(taskAttempt);
+    }
+  }
+
+  @Override
+  public InetSocketAddress getAddress() {
+    return address;
+  }
+
+  public TezTaskUmbilicalProtocol getUmbilical() {
+    return this.taskUmbilical;
+  }
+
+  private class TezTaskUmbilicalProtocolImpl implements TezTaskUmbilicalProtocol {
+
+    @Override
+    public ContainerTask getTask(ContainerContext containerContext) throws IOException {
+      ContainerTask task = null;
+      if (containerContext == null || containerContext.getContainerIdentifier() == null) {
+        LOG.info("Invalid task request with an empty containerContext or containerId");
+        task = TASK_FOR_INVALID_JVM;
+      } else {
+        ContainerId containerId = ConverterUtils.toContainerId(containerContext
+            .getContainerIdentifier());
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Container with id: " + containerId + " asked for a task");
+        }
+        task = getContainerTask(containerId);
+        if (task != null && !task.shouldDie()) {
+          taskCommunicatorContext
+              .taskStartedRemotely(task.getTaskSpec().getTaskAttemptID(), containerId);
+        }
+      }
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("getTask returning task: " + task);
+      }
+      return task;
+    }
+
+    @Override
+    public boolean canCommit(TezTaskAttemptID taskAttemptId) throws IOException {
+      return taskCommunicatorContext.canCommit(taskAttemptId);
+    }
+
+    @Override
+    public TezHeartbeatResponse heartbeat(TezHeartbeatRequest request) throws IOException,
+        TezException {
+      ContainerId containerId = ConverterUtils.toContainerId(request.getContainerIdentifier());
+      long requestId = request.getRequestId();
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Received heartbeat from container"
+            + ", request=" + request);
+      }
+
+      ContainerInfo containerInfo = registeredContainers.get(containerId);
+      if (containerInfo == null) {
+        LOG.warn("Received task heartbeat from unknown container with id: " + containerId +
+            ", asking it to die");
+        TezHeartbeatResponse response = new TezHeartbeatResponse();
+        response.setLastRequestId(requestId);
+        response.setShouldDie();
+        return response;
+      }
+
+      synchronized (containerInfo) {
+        if (containerInfo.lastRequestId == requestId) {
+          LOG.warn("Old sequenceId received: " + requestId
+              + ", Re-sending last response to client");
+          return containerInfo.lastResponse;
+        }
+      }
+
+      TaskHeartbeatResponse tResponse = null;
+
+
+      TezTaskAttemptID taskAttemptID = request.getCurrentTaskAttemptID();
+      if (taskAttemptID != null) {
+        synchronized (containerInfo) {
+          ContainerId containerIdFromMap = attemptToContainerMap.get(new TaskAttempt(taskAttemptID));
+          if (containerIdFromMap == null || !containerIdFromMap.equals(containerId)) {
+            throw new TezException("Attempt " + taskAttemptID
+                + " is not recognized for heartbeat");
+          }
+
+          if (containerInfo.lastRequestId + 1 != requestId) {
+            throw new TezException("Container " + containerId
+                + " has invalid request id. Expected: "
+                + containerInfo.lastRequestId + 1
+                + " and actual: " + requestId);
+          }
+        }
+        TaskHeartbeatRequest tRequest = new TaskHeartbeatRequest(request.getContainerIdentifier(),
+            request.getCurrentTaskAttemptID(), request.getEvents(), request.getStartIndex(),
+            request.getMaxEvents());
+        tResponse = taskCommunicatorContext.heartbeat(tRequest);
+      }
+      TezHeartbeatResponse response;
+      if (tResponse == null) {
+        response = new TezHeartbeatResponse();
+      } else {
+        response = new TezHeartbeatResponse(tResponse.getEvents());
+      }
+      response.setLastRequestId(requestId);
+      containerInfo.lastRequestId = requestId;
+      containerInfo.lastResponse = response;
+      return response;
+    }
+
+
+    // TODO Remove this method once we move to the Protobuf RPC engine
+    @Override
+    public long getProtocolVersion(String protocol, long clientVersion) throws IOException {
+      return versionID;
+    }
+
+    // TODO Remove this method once we move to the Protobuf RPC engine
+    @Override
+    public ProtocolSignature getProtocolSignature(String protocol, long clientVersion,
+                                                  int clientMethodsHash) throws IOException {
+      return ProtocolSignature.getProtocolSignature(this, protocol,
+          clientVersion, clientMethodsHash);
+    }
+  }
+
+  private ContainerTask getContainerTask(ContainerId containerId) throws IOException {
+    ContainerInfo containerInfo = registeredContainers.get(containerId);
+    ContainerTask task = null;
+    if (containerInfo == null) {
+      if (taskCommunicatorContext.isKnownContainer(containerId)) {
+        LOG.info("Container with id: " + containerId
+            + " is valid, but no longer registered, and will be killed");
+      } else {
+        LOG.info("Container with id: " + containerId
+            + " is invalid and will be killed");
+      }
+      task = TASK_FOR_INVALID_JVM;
+    } else {
+      synchronized (containerInfo) {
+        if (containerInfo.taskSpec != null) {
+          if (!containerInfo.taskPulled) {
+            containerInfo.taskPulled = true;
+            task = constructContainerTask(containerInfo);
+          } else {
+            if (LOG.isDebugEnabled()) {
+              LOG.debug("Task " + containerInfo.taskSpec.getTaskAttemptID() +
+                  " already sent to container: " + containerId);
+            }
+            task = null;
+          }
+        } else {
+          task = null;
+          if (LOG.isDebugEnabled()) {
+            LOG.debug("No task assigned yet for running container: " + containerId);
+          }
+        }
+      }
+    }
+    return task;
+  }
+
+  private ContainerTask constructContainerTask(ContainerInfo containerInfo) throws IOException {
+    return new ContainerTask(containerInfo.taskSpec, false,
+        convertLocalResourceMap(containerInfo.additionalLRs), containerInfo.credentials,
+        containerInfo.credentialsChanged);
+  }
+
+  private Map<String, TezLocalResource> convertLocalResourceMap(Map<String, LocalResource> ylrs)
+      throws IOException {
+    Map<String, TezLocalResource> tlrs = Maps.newHashMap();
+    if (ylrs != null) {
+      for (Map.Entry<String, LocalResource> ylrEntry : ylrs.entrySet()) {
+        TezLocalResource tlr;
+        try {
+          tlr = TezConverterUtils.convertYarnLocalResourceToTez(ylrEntry.getValue());
+        } catch (URISyntaxException e) {
+          throw new IOException(e);
+        }
+        tlrs.put(ylrEntry.getKey(), tlr);
+      }
+    }
+    return tlrs;
+  }
+
+
+  // Holder for Task information, which eventually will likely be VertexImplm taskIndex, attemptIndex
+  private static class TaskAttempt {
+    // TODO TEZ-2003 Change this to work with VertexName, int id, int version
+    // TODO TEZ-2003 Avoid constructing this unit all over the place
+    private TezTaskAttemptID taskAttemptId;
+
+    TaskAttempt(TezTaskAttemptID taskAttemptId) {
+      this.taskAttemptId = taskAttemptId;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) {
+        return true;
+      }
+      if (!(o instanceof TaskAttempt)) {
+        return false;
+      }
+
+      TaskAttempt that = (TaskAttempt) o;
+
+      if (!taskAttemptId.equals(that.taskAttemptId)) {
+        return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      return taskAttemptId.hashCode();
+    }
+
+    @Override
+    public String toString() {
+      return "TaskAttempt{" + "taskAttemptId=" + taskAttemptId + '}';
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tez/blob/56986504/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/LocalContainerLauncher.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/LocalContainerLauncher.java b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/LocalContainerLauncher.java
index 9faf8c0..e9ba9d7 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/LocalContainerLauncher.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/LocalContainerLauncher.java
@@ -59,6 +59,8 @@ import org.apache.tez.dag.api.TezException;
 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.app.TaskAttemptListenerImpTezDag;
+import org.apache.tez.dag.app.TezTaskCommunicatorImpl;
 import org.apache.tez.dag.app.rm.NMCommunicatorEvent;
 import org.apache.tez.dag.app.rm.NMCommunicatorLaunchRequestEvent;
 import org.apache.tez.dag.app.rm.NMCommunicatorStopRequestEvent;
@@ -86,7 +88,7 @@ public class LocalContainerLauncher extends AbstractService implements
 
   private static final Logger LOG = LoggerFactory.getLogger(LocalContainerLauncher.class);
   private final AppContext context;
-  private final TaskAttemptListener taskAttemptListener;
+  private final TezTaskUmbilicalProtocol taskUmbilicalProtocol;
   private final AtomicBoolean serviceStopped = new AtomicBoolean(false);
   private final String workingDirectory;
   private final Map<String, String> localEnv = new HashMap<String, String>();
@@ -114,7 +116,9 @@ public class LocalContainerLauncher extends AbstractService implements
                                 String workingDirectory) throws UnknownHostException {
     super(LocalContainerLauncher.class.getName());
     this.context = context;
-    this.taskAttemptListener = taskAttemptListener;
+    TaskAttemptListenerImpTezDag taListener = (TaskAttemptListenerImpTezDag)taskAttemptListener;
+    TezTaskCommunicatorImpl taskComm = (TezTaskCommunicatorImpl) taListener.getTaskCommunicator();
+    this.taskUmbilicalProtocol = taskComm.getUmbilical();
     this.workingDirectory = workingDirectory;
     AuxiliaryServiceHelper.setServiceDataIntoEnv(
         ShuffleUtils.SHUFFLE_HANDLER_SERVICE_ID, ByteBuffer.allocate(4).putInt(0), localEnv);
@@ -215,7 +219,7 @@ public class LocalContainerLauncher extends AbstractService implements
         tezChild =
             createTezChild(context.getAMConf(), event.getContainerId(), tokenIdentifier,
                 context.getApplicationAttemptId().getAttemptId(), context.getLocalDirs(),
-                (TezTaskUmbilicalProtocol) taskAttemptListener,
+                taskUmbilicalProtocol,
                 TezCommonUtils.parseCredentialsBytes(event.getContainerLaunchContext().getTokens().array()));
       } catch (InterruptedException e) {
         handleLaunchFailed(e, event.getContainerId());

http://git-wip-us.apache.org/repos/asf/tez/blob/56986504/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainer.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainer.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainer.java
index a6b403d..0fc2e12 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainer.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainer.java
@@ -22,6 +22,7 @@ import java.util.List;
 
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.tez.dag.records.TezTaskAttemptID;
 
@@ -32,5 +33,5 @@ public interface AMContainer extends EventHandler<AMContainerEvent>{
   public Container getContainer();
   public List<TezTaskAttemptID> getAllTaskAttempts();
   public TezTaskAttemptID getCurrentTaskAttempt();
-  
+
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/56986504/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 682cd02..0398882 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
@@ -27,6 +27,8 @@ import org.apache.tez.runtime.api.impl.TaskSpec;
 
 public class AMContainerEventAssignTA extends AMContainerEvent {
 
+  // TODO TEZ-2003. Add the task priority to this event.
+
   private final TezTaskAttemptID attemptId;
   // TODO Maybe have tht TAL pull the remoteTask from the TaskAttempt itself ?
   private final TaskSpec remoteTaskSpec;

http://git-wip-us.apache.org/repos/asf/tez/blob/56986504/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 330f2b7..1acec9c 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
@@ -35,6 +35,7 @@ import org.apache.hadoop.yarn.api.records.ContainerExitStatus;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
 import org.apache.hadoop.yarn.api.records.LocalResource;
+import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.event.Event;
 import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.state.InvalidStateTransitonException;

http://git-wip-us.apache.org/repos/asf/tez/blob/56986504/tez-dag/src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java b/tez-dag/src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java
index 18286b5..0b2ea2f 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java
@@ -37,6 +37,7 @@ import java.util.concurrent.atomic.AtomicLong;
 import org.apache.tez.dag.app.dag.DAG;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+import com.google.common.annotations.VisibleForTesting;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.service.AbstractService;
@@ -50,7 +51,10 @@ import org.apache.tez.client.TezApiVersionInfo;
 import org.apache.tez.common.ContainerContext;
 import org.apache.tez.common.ContainerTask;
 import org.apache.tez.common.counters.TezCounters;
+import org.apache.tez.dag.api.TaskHeartbeatRequest;
+import org.apache.tez.dag.api.TaskHeartbeatResponse;
 import org.apache.tez.dag.api.TezConfiguration;
+import org.apache.tez.dag.api.TaskCommunicator;
 import org.apache.tez.dag.api.TezUncheckedException;
 import org.apache.tez.dag.app.launcher.ContainerLauncher;
 import org.apache.tez.dag.app.rm.NMCommunicatorEvent;
@@ -72,8 +76,6 @@ import org.apache.tez.runtime.api.impl.TaskSpec;
 import org.apache.tez.runtime.api.impl.TaskStatistics;
 import org.apache.tez.runtime.api.impl.TezEvent;
 import org.apache.tez.runtime.api.impl.EventMetaData.EventProducerConsumerType;
-import org.apache.tez.runtime.api.impl.TezHeartbeatRequest;
-import org.apache.tez.runtime.api.impl.TezHeartbeatResponse;
 
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
@@ -126,6 +128,7 @@ public class MockDAGAppMaster extends DAGAppMaster {
     Map<ContainerId, ContainerData> containers = Maps.newConcurrentMap();
     ArrayBlockingQueue<Worker> workers;
     TaskAttemptListenerImpTezDag taListener;
+    TezTaskCommunicatorImpl taskCommunicator;
     
     AtomicBoolean startScheduling = new AtomicBoolean(true);
     AtomicBoolean goFlag;
@@ -188,6 +191,7 @@ public class MockDAGAppMaster extends DAGAppMaster {
     @Override
     public void serviceStart() throws Exception {
       taListener = (TaskAttemptListenerImpTezDag) getTaskAttemptListener();
+      taskCommunicator = (TezTaskCommunicatorImpl) taListener.getTaskCommunicator();
       eventHandlingThread = new Thread(this);
       eventHandlingThread.start();
       ExecutorService rawExecutor = Executors.newFixedThreadPool(handlerConcurrency,
@@ -327,10 +331,10 @@ public class MockDAGAppMaster extends DAGAppMaster {
       }
     }
     
-    private void doHeartbeat(TezHeartbeatRequest request, ContainerData cData) throws Exception {
+    private void doHeartbeat(TaskHeartbeatRequest request, ContainerData cData) throws Exception {
       long startTime = System.nanoTime();
       long startCpuTime = threadMxBean.getCurrentThreadCpuTime();
-      TezHeartbeatResponse response = taListener.heartbeat(request);
+      TaskHeartbeatResponse response = taListener.heartbeat(request);
       if (response.shouldDie()) {
         cData.remove();
       } else {
@@ -381,7 +385,8 @@ public class MockDAGAppMaster extends DAGAppMaster {
         try {
           if (cData.taId == null) {
             // if container is not assigned a task, ask for a task
-            ContainerTask cTask = taListener.getTask(new ContainerContext(cData.cIdStr));
+            ContainerTask cTask =
+                taskCommunicator.getUmbilical().getTask(new ContainerContext(cData.cIdStr));
             if (cTask != null) {
               if (cTask.shouldDie()) {
                 cData.remove();
@@ -427,8 +432,9 @@ public class MockDAGAppMaster extends DAGAppMaster {
               float progress = updateProgress ? cData.numUpdates/maxUpdates : 0f;
               events.add(new TezEvent(new TaskStatusUpdateEvent(counters, progress, stats), new EventMetaData(
                   EventProducerConsumerType.SYSTEM, cData.vName, "", cData.taId)));
-              TezHeartbeatRequest request = new TezHeartbeatRequest(cData.numUpdates, events,
-                  cData.cIdStr, cData.taId, cData.nextFromEventId, 10000);
+              TaskHeartbeatRequest request =
+                  new TaskHeartbeatRequest(cData.cIdStr, cData.taId, events, cData.nextFromEventId,
+                      10000);
               doHeartbeat(request, cData);
             } else if (version != null && cData.taId.getId() <= version.intValue()) {
               preemptContainer(cData);
@@ -438,8 +444,9 @@ public class MockDAGAppMaster extends DAGAppMaster {
               List<TezEvent> events = Collections.singletonList(new TezEvent(
                   new TaskAttemptCompletedEvent(), new EventMetaData(
                       EventProducerConsumerType.SYSTEM, cData.vName, "", cData.taId)));
-              TezHeartbeatRequest request = new TezHeartbeatRequest(++cData.numUpdates, events,
-                  cData.cIdStr, cData.taId, cData.nextFromEventId, 10000);
+              TaskHeartbeatRequest request =
+                  new TaskHeartbeatRequest(cData.cIdStr, cData.taId, events, cData.nextFromEventId,
+                      10000);
               doHeartbeat(request, cData);
               cData.clear();
             }

http://git-wip-us.apache.org/repos/asf/tez/blob/56986504/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag.java b/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag.java
index ec4f99a..286e897 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag.java
@@ -1,16 +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.
- */
+* 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.
+*/
 
 package org.apache.tez.dag.app;
 
@@ -18,6 +18,7 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
+import static org.mockito.Matchers.any;
 import static org.mockito.Mockito.doReturn;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.times;
@@ -33,6 +34,7 @@ import java.util.Map;
 import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.event.Event;
 import org.apache.hadoop.yarn.event.EventHandler;
@@ -40,6 +42,12 @@ import org.apache.tez.common.ContainerContext;
 import org.apache.tez.common.ContainerTask;
 import org.apache.tez.common.security.JobTokenSecretManager;
 import org.apache.tez.dag.api.TezException;
+import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.hadoop.yarn.event.EventHandler;
+import org.apache.tez.common.ContainerContext;
+import org.apache.tez.common.ContainerTask;
+import org.apache.tez.common.TezTaskUmbilicalProtocol;
+import org.apache.tez.dag.api.TaskCommunicatorContext;
 import org.apache.tez.dag.app.dag.DAG;
 import org.apache.tez.dag.app.dag.Task;
 import org.apache.tez.dag.app.dag.Vertex;
@@ -99,9 +107,18 @@ public class TestTaskAttemptListenerImplTezDag {
     doReturn(dag).when(appContext).getCurrentDAG();
     doReturn(appAcls).when(appContext).getApplicationACLs();
     doReturn(amContainerMap).when(appContext).getAllContainers();
-
-    taskAttemptListener = new TaskAttemptListenerImplForTest(appContext,
-        mock(TaskHeartbeatHandler.class), mock(ContainerHeartbeatHandler.class), null);
+    NodeId nodeId = NodeId.newInstance("localhost", 0);
+    AMContainer amContainer = mock(AMContainer.class);
+    Container container = mock(Container.class);
+    doReturn(nodeId).when(container).getNodeId();
+    doReturn(amContainer).when(amContainerMap).get(any(ContainerId.class));
+    doReturn(container).when(amContainer).getContainer();
+
+    taskAttemptListener =
+        new TaskAttemptListenerImpTezDag(appContext, mock(TaskHeartbeatHandler.class),
+            mock(ContainerHeartbeatHandler.class), null);
+    TezTaskCommunicatorImpl taskCommunicator = (TezTaskCommunicatorImpl)taskAttemptListener.getTaskCommunicator();
+    TezTaskUmbilicalProtocol tezUmbilical = taskCommunicator.getUmbilical();
 
     taskSpec = mock(TaskSpec.class);
     doReturn(taskAttemptID).when(taskSpec).getTaskAttemptID();
@@ -113,32 +130,30 @@ public class TestTaskAttemptListenerImplTezDag {
   public void testGetTask() throws IOException {
 
     ContainerId containerId1 = createContainerId(appId, 1);
-    doReturn(mock(AMContainer.class)).when(amContainerMap).get(containerId1);
     ContainerContext containerContext1 = new ContainerContext(containerId1.toString());
-    containerTask = taskAttemptListener.getTask(containerContext1);
+    containerTask = tezUmbilical.getTask(containerContext1);
     assertTrue(containerTask.shouldDie());
 
     ContainerId containerId2 = createContainerId(appId, 2);
-    doReturn(mock(AMContainer.class)).when(amContainerMap).get(containerId2);
     ContainerContext containerContext2 = new ContainerContext(containerId2.toString());
     taskAttemptListener.registerRunningContainer(containerId2);
-    containerTask = taskAttemptListener.getTask(containerContext2);
+    containerTask = tezUmbilical.getTask(containerContext2);
     assertNull(containerTask);
 
     // Valid task registered
     taskAttemptListener.registerTaskAttempt(amContainerTask, containerId2);
-    containerTask = taskAttemptListener.getTask(containerContext2);
+    containerTask = tezUmbilical.getTask(containerContext2);
     assertFalse(containerTask.shouldDie());
     assertEquals(taskSpec, containerTask.getTaskSpec());
 
     // Task unregistered. Should respond to heartbeats
-    taskAttemptListener.unregisterTaskAttempt(taskAttemptID);
-    containerTask = taskAttemptListener.getTask(containerContext2);
+    taskAttemptListener.unregisterTaskAttempt(taskAttemptId);
+    containerTask = tezUmbilical.getTask(containerContext2);
     assertNull(containerTask);
 
     // Container unregistered. Should send a shouldDie = true
     taskAttemptListener.unregisterRunningContainer(containerId2);
-    containerTask = taskAttemptListener.getTask(containerContext2);
+    containerTask = tezUmbilical.getTask(containerContext2);
     assertTrue(containerTask.shouldDie());
 
     ContainerId containerId3 = createContainerId(appId, 3);
@@ -152,27 +167,30 @@ public class TestTaskAttemptListenerImplTezDag {
     AMContainerTask amContainerTask2 = new AMContainerTask(taskSpec, null, null, false, 0);
     taskAttemptListener.registerTaskAttempt(amContainerTask2, containerId3);
     taskAttemptListener.unregisterRunningContainer(containerId3);
-    containerTask = taskAttemptListener.getTask(containerContext3);
+    containerTask = tezUmbilical.getTask(containerContext3);
     assertTrue(containerTask.shouldDie());
   }
 
   @Test(timeout = 5000)
   public void testGetTaskMultiplePulls() throws IOException {
+    TezTaskCommunicatorImpl taskCommunicator = (TezTaskCommunicatorImpl)taskAttemptListener.getTaskCommunicator();
+    TezTaskUmbilicalProtocol tezUmbilical = taskCommunicator.getUmbilical();
+
     ContainerId containerId1 = createContainerId(appId, 1);
     doReturn(mock(AMContainer.class)).when(amContainerMap).get(containerId1);
     ContainerContext containerContext1 = new ContainerContext(containerId1.toString());
     taskAttemptListener.registerRunningContainer(containerId1);
-    containerTask = taskAttemptListener.getTask(containerContext1);
+    containerTask = tezUmbilical.getTask(containerContext1);
     assertNull(containerTask);
 
     // Register task
     taskAttemptListener.registerTaskAttempt(amContainerTask, containerId1);
-    containerTask = taskAttemptListener.getTask(containerContext1);
+    containerTask = tezUmbilical.getTask(containerContext1);
     assertFalse(containerTask.shouldDie());
     assertEquals(taskSpec, containerTask.getTaskSpec());
 
     // Try pulling again - simulates re-use pull
-    containerTask = taskAttemptListener.getTask(containerContext1);
+    containerTask = tezUmbilical.getTask(containerContext1);
     assertNull(containerTask);
   }
 
@@ -250,13 +268,11 @@ public class TestTaskAttemptListenerImplTezDag {
     return ContainerId.newInstance(appAttemptId, containerIdx);
   }
 
-  private static class TaskAttemptListenerImplForTest extends TaskAttemptListenerImpTezDag {
+  private static class TezTaskCommunicatorImplForTest extends TezTaskCommunicatorImpl {
 
-    public TaskAttemptListenerImplForTest(AppContext context,
-                                          TaskHeartbeatHandler thh,
-                                          ContainerHeartbeatHandler chh,
-                                          JobTokenSecretManager jobTokenSecretManager) {
-      super(context, thh, chh, jobTokenSecretManager);
+    public TezTaskCommunicatorImplForTest(
+        TaskCommunicatorContext taskCommunicatorContext) {
+      super(taskCommunicatorContext);
     }
 
     @Override


[05/50] [abbrv] tez git commit: TEZ-2395. Tez UI: Minimum/Maximum Duration show a empty bracket next to 0 secs when you purposefully failed a job. (pramachandran)

Posted by ss...@apache.org.
TEZ-2395. Tez UI: Minimum/Maximum Duration show a empty bracket next to 0 secs when you purposefully failed a job. (pramachandran)


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

Branch: refs/heads/TEZ-2003
Commit: fcd6bb686750b21ea35696312088e1f4bbedce1a
Parents: 96efae0
Author: Prakash Ramachandran <pr...@hortonworks.com>
Authored: Fri May 1 13:59:22 2015 +0530
Committer: Prakash Ramachandran <pr...@hortonworks.com>
Committed: Fri May 1 13:59:22 2015 +0530

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../controllers/vertex_index_controller.js      | 35 +++++++++++++++-----
 .../main/webapp/app/templates/vertex/index.hbs  |  8 ++---
 3 files changed, 32 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/fcd6bb68/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index e7079bf..c8f8a3a 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -9,6 +9,7 @@ INCOMPATIBLE CHANGES
   TEZ-1993. Implement a pluggable InputSizeEstimator for grouping fairly
 
 ALL CHANGES:
+  TEZ-2395. Tez UI: Minimum/Maximum Duration show a empty bracket next to 0 secs when you purposefully failed a job.
   TEZ-2360. per-io counters flag should generate both overall and per-edge counters
   TEZ-2389. Tez UI: Sort by attempt-no is incorrect in attempts pages.
   TEZ-2383. Cleanup input/output/processor contexts in LogicalIOProcessorRuntimeTask.

http://git-wip-us.apache.org/repos/asf/tez/blob/fcd6bb68/tez-ui/src/main/webapp/app/scripts/controllers/vertex_index_controller.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/scripts/controllers/vertex_index_controller.js b/tez-ui/src/main/webapp/app/scripts/controllers/vertex_index_controller.js
index f104c11..ebddf53 100644
--- a/tez-ui/src/main/webapp/app/scripts/controllers/vertex_index_controller.js
+++ b/tez-ui/src/main/webapp/app/scripts/controllers/vertex_index_controller.js
@@ -38,7 +38,7 @@ App.VertexIndexController = Em.ObjectController.extend(App.ModelRefreshMixin, {
   //TODO: TEZ-1705 : Create a parent class and move this function there to avoid duplication.
   iconStatus: function() {
     return App.Helpers.misc.getStatusClassForEntity(this.get('model.status'));
-  }.property('id', 'status', 'counterGroups'),
+  }.property('id', 'model.status'),
 
   progressStr: function() {
     var pct;
@@ -46,25 +46,44 @@ App.VertexIndexController = Em.ObjectController.extend(App.ModelRefreshMixin, {
       pct = App.Helpers.number.fractionToPercentage(this.get('progress'));
     }
     return pct;
-  }.property('id', 'status', 'progress'),
+  }.property('id', 'status', 'progress', 'model.status'),
 
   hasFailedTasks: function() {
     return this.get('failedTasks') > 0;
-  }.property('id', 'counterGroups'),
+  }.property('failedTasks'),
   
   failedTasksLink: function() {
     return '#tasks?status=FAILED&parentType=TEZ_VERTEX_ID&parentID=' + this.get('id');
-  }.property(),
+  }.property('id'),
 
   hasFirstTaskStarted: function() {
     return !!this.get('firstTaskStartTime') && !!this.get('firstTasksToStart');
-  }.property(),
+  }.property('firstTaskStartTime', 'firstTasksToStart'),
 
   hasLastTaskFinished: function() {
     return !!this.get('lastTaskFinishTime') && !!this.get('lastTasksToFinish');
-  }.property(),
+  }.property('lastTaskFinishTime', 'lastTasksToFinish'),
 
   hasStats: function() {
-    return !!this.get('avgTaskDuration') || !!this.get('minTaskDuration') || !!this.get('maxTaskDuration');
-  }.property()
+    return (this.get('numTasks') || 0) > 0 ||
+           (this.get('sucessfulTasks') || 0) > 0 ||
+           (this.get('failedTasks') || 0 ) > 0 ||
+           (this.get('killedTasks') || 0) > 0 ||
+           this.get('showAvgTaskDuration') ||
+           this.get('showMinTaskDuration') ||
+           this.get('showMaxTaskDuration');
+  }.property('numTasks', 'sucessfulTasks', 'failedTasks', 'killedTasks', 'showAvgTaskDuration',
+    'showMinTaskDuration', 'showMaxTaskDuration'),
+
+  showAvgTaskDuration: function() {
+    return (this.get('avgTaskDuration') || 0) > 0;
+  }.property('avgTaskDuration'),
+
+  showMinTaskDuration: function() {
+    return (this.get('minTaskDuration') || 0) > 0;
+  }.property('minTaskDuration'),
+
+  showMaxTaskDuration: function() {
+    return (this.get('maxTaskDuration') || 0) > 0;
+  }.property('maxTaskDuration'),
 });
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tez/blob/fcd6bb68/tez-ui/src/main/webapp/app/templates/vertex/index.hbs
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/templates/vertex/index.hbs b/tez-ui/src/main/webapp/app/templates/vertex/index.hbs
index 2f2eeda..acdce99 100644
--- a/tez-ui/src/main/webapp/app/templates/vertex/index.hbs
+++ b/tez-ui/src/main/webapp/app/templates/vertex/index.hbs
@@ -130,23 +130,23 @@
             <td>Failed Tasks</td>
             <td>
               {{failedTasks}}
-            </td>
               {{#if hasFailedTasks}}
                 <a href='{{unbound failedTasksLink}}'>FailedTasks</a>
               {{/if}}
+            </td>
           </tr>
           <tr>
             <td>Killed Tasks</td>
             <td>{{killedTasks}}</td>
           </tr>
 
-          {{#if avgTaskDuration}}
+          {{#if showAvgTaskDuration}}
             <tr>
               <td>Average Duration</td>
               <td>{{formatTimeMillis avgTaskDuration}}</td>
             </tr>
           {{/if}}
-          {{#if minTaskDuration}}
+          {{#if showMinTaskDuration}}
             <tr>
               <td>Minimum Duration</td>
               <td>{{formatTimeMillis minTaskDuration}}
@@ -159,7 +159,7 @@
               </td>
             </tr>
           {{/if}}
-          {{#if maxTaskDuration}}
+          {{#if showMaxTaskDuration}}
             <tr>
               <td>Maximum Duration</td>
               <td>{{formatTimeMillis maxTaskDuration}}


[26/50] [abbrv] tez git commit: TEZ-2019. Temporarily allow the scheduler and launcher to be specified via configuration. (sseth)

Posted by ss...@apache.org.
TEZ-2019. Temporarily allow the scheduler and launcher to be specified
via configuration. (sseth)


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

Branch: refs/heads/TEZ-2003
Commit: 0fc4c6941ef8d7892ac6a652e73319fd6db81e3b
Parents: 5679b28
Author: Siddharth Seth <ss...@apache.org>
Authored: Fri Jan 30 16:02:32 2015 -0800
Committer: Siddharth Seth <ss...@apache.org>
Committed: Wed May 6 00:13:28 2015 -0700

----------------------------------------------------------------------
 TEZ-2003-CHANGES.txt                            |  4 +++
 .../apache/tez/dag/api/TezConfiguration.java    |  6 ++++
 .../org/apache/tez/dag/app/DAGAppMaster.java    | 30 ++++++++++++++++-
 .../dag/app/rm/TaskSchedulerEventHandler.java   | 34 ++++++++++++++++++--
 .../org/apache/tez/runtime/task/TezChild.java   |  3 +-
 5 files changed, 73 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/0fc4c694/TEZ-2003-CHANGES.txt
----------------------------------------------------------------------
diff --git a/TEZ-2003-CHANGES.txt b/TEZ-2003-CHANGES.txt
new file mode 100644
index 0000000..1822fcb
--- /dev/null
+++ b/TEZ-2003-CHANGES.txt
@@ -0,0 +1,4 @@
+ALL CHANGES:
+  TEZ-2019. Temporarily allow the scheduler and launcher to be specified via configuration.
+
+INCOMPATIBLE CHANGES:

http://git-wip-us.apache.org/repos/asf/tez/blob/0fc4c694/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java b/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java
index f64172e..87ccd19 100644
--- a/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java
+++ b/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java
@@ -1168,6 +1168,12 @@ public class TezConfiguration extends Configuration {
       + "tez-ui.webservice.enable";
   public static final boolean TEZ_AM_WEBSERVICE_ENABLE_DEFAULT = true;
 
+  @ConfigurationScope(Scope.VERTEX)
+  public static final String TEZ_AM_CONTAINER_LAUNCHER_CLASS = TEZ_AM_PREFIX + "container-launcher.class";
+  @ConfigurationScope(Scope.VERTEX)
+  public static final String TEZ_AM_TASK_SCHEDULER_CLASS = TEZ_AM_PREFIX + "task-scheduler.class";
+
+
   // TODO only validate property here, value can also be validated if necessary
   public static void validateProperty(String property, Scope usedScope) {
     Scope validScope = PropertyScope.get(property);

http://git-wip-us.apache.org/repos/asf/tez/blob/0fc4c694/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 3e3d6f0..73ee56e 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
@@ -25,6 +25,8 @@ import java.io.FileInputStream;
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.PrintWriter;
+import java.lang.reflect.Constructor;
+import java.lang.reflect.InvocationTargetException;
 import java.net.URI;
 import java.net.URISyntaxException;
 import java.net.URL;
@@ -93,6 +95,7 @@ import org.apache.hadoop.yarn.util.SystemClock;
 import org.apache.tez.common.AsyncDispatcher;
 import org.apache.tez.common.AsyncDispatcherConcurrent;
 import org.apache.tez.common.GcTimeUpdater;
+import org.apache.tez.common.ReflectionUtils;
 import org.apache.tez.common.TezCommonUtils;
 import org.apache.tez.common.TezConverterUtils;
 import org.apache.tez.common.TezUtilsInternal;
@@ -1034,9 +1037,34 @@ public class DAGAppMaster extends AbstractService {
   protected ContainerLauncher
       createContainerLauncher(final AppContext context) throws UnknownHostException {
     if(isLocal){
+      LOG.info("Creating LocalContainerLauncher");
       return new LocalContainerLauncher(context, taskAttemptListener, workingDirectory);
     } else {
-      return new ContainerLauncherImpl(context);
+      // TODO: Temporary reflection with specific parameters until a clean interface is defined.
+      String containerLauncherClassName = getConfig().get(TezConfiguration.TEZ_AM_CONTAINER_LAUNCHER_CLASS);
+      if (containerLauncherClassName == null) {
+        LOG.info("Creating Default Container Launcher");
+        return new ContainerLauncherImpl(context);
+      } else {
+        LOG.info("Creating container launcher : " + containerLauncherClassName);
+        Class<? extends ContainerLauncher> containerLauncherClazz = (Class<? extends ContainerLauncher>) ReflectionUtils.getClazz(
+            containerLauncherClassName);
+        try {
+          Constructor<? extends ContainerLauncher> ctor = containerLauncherClazz
+              .getConstructor(AppContext.class, Configuration.class, TaskAttemptListener.class);
+          ctor.setAccessible(true);
+          ContainerLauncher instance = ctor.newInstance(context, getConfig(), taskAttemptListener);
+          return instance;
+        } catch (NoSuchMethodException e) {
+          throw new TezUncheckedException(e);
+        } catch (InvocationTargetException e) {
+          throw new TezUncheckedException(e);
+        } catch (InstantiationException e) {
+          throw new TezUncheckedException(e);
+        } catch (IllegalAccessException e) {
+          throw new TezUncheckedException(e);
+        }
+      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/tez/blob/0fc4c694/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerEventHandler.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerEventHandler.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerEventHandler.java
index 19db660..62f82db 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerEventHandler.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerEventHandler.java
@@ -18,6 +18,8 @@
 
 package org.apache.tez.dag.app.rm;
 
+import java.lang.reflect.Constructor;
+import java.lang.reflect.InvocationTargetException;
 import java.net.InetSocketAddress;
 import java.nio.ByteBuffer;
 import java.util.List;
@@ -42,6 +44,7 @@ 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.ReflectionUtils;
 import org.apache.tez.dag.api.TezConfiguration;
 import org.apache.tez.dag.api.TezUncheckedException;
 import org.apache.tez.dag.api.TaskLocationHint;
@@ -329,12 +332,39 @@ public class TaskSchedulerEventHandler extends AbstractService
     boolean isLocal = getConfig().getBoolean(TezConfiguration.TEZ_LOCAL_MODE,
         TezConfiguration.TEZ_LOCAL_MODE_DEFAULT);
     if (isLocal) {
+      LOG.info("Using TaskScheduler: LocalTaskSchedulerService");
       return new LocalTaskSchedulerService(this, this.containerSignatureMatcher,
           host, port, trackingUrl, appContext);
     }
     else {
-      return new YarnTaskSchedulerService(this, this.containerSignatureMatcher,
-          host, port, trackingUrl, appContext);
+      String schedulerClassName = getConfig().get(TezConfiguration.TEZ_AM_TASK_SCHEDULER_CLASS);
+      if (schedulerClassName == null) {
+        LOG.info("Using TaskScheduler: YarnTaskSchedulerService");
+        return new YarnTaskSchedulerService(this, this.containerSignatureMatcher,
+            host, port, trackingUrl, appContext);
+      } else {
+        LOG.info("Using custom TaskScheduler: " + schedulerClassName);
+        // TODO Temporary reflection with specific parameters. Remove once there is a clean interface.
+        Class<? extends TaskSchedulerService> taskSchedulerClazz =
+            (Class<? extends TaskSchedulerService>) ReflectionUtils.getClazz(schedulerClassName);
+        try {
+          Constructor<? extends TaskSchedulerService> ctor = taskSchedulerClazz
+              .getConstructor(TaskSchedulerAppCallback.class, AppContext.class, String.class,
+                  Integer.class, String.class, Configuration.class);
+          ctor.setAccessible(true);
+          TaskSchedulerService taskSchedulerService =
+              ctor.newInstance(this, appContext, host, port, trackingUrl, getConfig());
+          return taskSchedulerService;
+        } catch (NoSuchMethodException e) {
+          throw new TezUncheckedException(e);
+        } catch (InvocationTargetException e) {
+          throw new TezUncheckedException(e);
+        } catch (InstantiationException e) {
+          throw new TezUncheckedException(e);
+        } catch (IllegalAccessException e) {
+          throw new TezUncheckedException(e);
+        }
+      }
     }
   }
   

http://git-wip-us.apache.org/repos/asf/tez/blob/0fc4c694/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezChild.java
----------------------------------------------------------------------
diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezChild.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezChild.java
index bfec349..fd55992 100644
--- a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezChild.java
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezChild.java
@@ -355,7 +355,8 @@ public class TezChild {
       }
       if (!isLocal) {
         RPC.stopProxy(umbilical);
-        LogManager.shutdown();
+        // TODO Temporary change. Revert. Ideally, move this over to the main method in TezChild if possible.
+//        LogManager.shutdown();
       }
     }
   }


[32/50] [abbrv] tez git commit: TEZ-2138. Fix minor bugs in adding default scheduler, getting launchers. (sseth)

Posted by ss...@apache.org.
TEZ-2138. Fix minor bugs in adding default scheduler, getting launchers.  (sseth)


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

Branch: refs/heads/TEZ-2003
Commit: 921bffc1616f614ff1e46992ee200d23c1f0f546
Parents: bead03d
Author: Siddharth Seth <ss...@apache.org>
Authored: Mon Feb 23 20:53:24 2015 -0800
Committer: Siddharth Seth <ss...@apache.org>
Committed: Wed May 6 00:13:29 2015 -0700

----------------------------------------------------------------------
 TEZ-2003-CHANGES.txt                                         | 1 +
 .../src/main/java/org/apache/tez/dag/app/DAGAppMaster.java   | 2 +-
 .../java/org/apache/tez/dag/app/dag/impl/VertexImpl.java     | 4 ++++
 .../org/apache/tez/dag/app/rm/TaskSchedulerEventHandler.java | 1 +
 .../dag/app/launcher/TezTestServiceContainerLauncher.java    | 8 +++++---
 5 files changed, 12 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/921bffc1/TEZ-2003-CHANGES.txt
----------------------------------------------------------------------
diff --git a/TEZ-2003-CHANGES.txt b/TEZ-2003-CHANGES.txt
index 4b0a12b..4377f57 100644
--- a/TEZ-2003-CHANGES.txt
+++ b/TEZ-2003-CHANGES.txt
@@ -7,5 +7,6 @@ ALL CHANGES:
   TEZ-2123. Fix component managers to use pluggable components. (Enable hybrid mode)
   TEZ-2125. Create a task communicator for local mode. Allow tasks to run in the AM.
   TEZ-2131. Add additional tests for tasks running in the AM.
+  TEZ-2138. Fix minor bugs in adding default scheduler, getting launchers.
 
 INCOMPATIBLE CHANGES:

http://git-wip-us.apache.org/repos/asf/tez/blob/921bffc1/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 701eca8..1ea369e 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
@@ -1507,7 +1507,7 @@ public class DAGAppMaster extends AbstractService {
 
     @Override
     public Integer getContainerLauncherIdentifier(String name) {
-      return taskCommunicators.get(name);
+      return containerLaunchers.get(name);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/tez/blob/921bffc1/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 81e1732..11164e9 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
@@ -951,6 +951,10 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex, EventHandl
         .get(TezConfiguration.TEZ_AM_VERTEX_TASK_COMMUNICATOR_NAME, tezDefaultComponentName);
     String containerLauncherName = vertexConf
         .get(TezConfiguration.TEZ_AM_VERTEX_CONTAINER_LAUNCHER_NAME, tezDefaultComponentName);
+
+    LOG.info("Vertex: " + logIdentifier + " configured with TaskScheduler=" + taskSchedulerName +
+        ", ContainerLauncher=" + containerLauncherName + ", TaskComm=" + taskCommName);
+
     taskSchedulerIdentifier = appContext.getTaskScheduerIdentifier(taskSchedulerName);
     taskCommunicatorIdentifier = appContext.getTaskCommunicatorIdentifier(taskCommName);
     containerLauncherIdentifier = appContext.getContainerLauncherIdentifier(containerLauncherName);

http://git-wip-us.apache.org/repos/asf/tez/blob/921bffc1/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerEventHandler.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerEventHandler.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerEventHandler.java
index 5a8e9fe..8e5fc71 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerEventHandler.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerEventHandler.java
@@ -174,6 +174,7 @@ public class TaskSchedulerEventHandler extends AbstractService
           for (int i = 0 ; i < schedulerClasses.length ; i++) { // Copy over the rest.
             this.taskSchedulerClasses[i] = schedulerClasses[i];
           }
+          this.taskSchedulerClasses[foundYarnTaskSchedulerIndex] = TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT;
         } else {
           this.taskSchedulerClasses = schedulerClasses;
         }

http://git-wip-us.apache.org/repos/asf/tez/blob/921bffc1/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/launcher/TezTestServiceContainerLauncher.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/launcher/TezTestServiceContainerLauncher.java b/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/launcher/TezTestServiceContainerLauncher.java
index 27356bc..cb6308c 100644
--- a/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/launcher/TezTestServiceContainerLauncher.java
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/launcher/TezTestServiceContainerLauncher.java
@@ -23,6 +23,7 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.service.AbstractService;
 import org.apache.hadoop.yarn.api.ApplicationConstants;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.util.Clock;
 import org.apache.tez.dag.app.AppContext;
@@ -54,6 +55,7 @@ public class TezTestServiceContainerLauncher extends AbstractService implements
   private final int servicePort;
   private final TezTestServiceCommunicator communicator;
   private final Clock clock;
+  private final ApplicationAttemptId appAttemptId;
 
 
   // Configuration passed in here to set up final parameters
@@ -70,6 +72,7 @@ public class TezTestServiceContainerLauncher extends AbstractService implements
     this.communicator = new TezTestServiceCommunicator(numThreads);
     this.context = appContext;
     this.tokenIdentifier = context.getApplicationID().toString();
+    this.appAttemptId = appContext.getApplicationAttemptId();
     this.tal = tal;
   }
 
@@ -128,9 +131,8 @@ public class TezTestServiceContainerLauncher extends AbstractService implements
     RunContainerRequestProto.Builder builder = RunContainerRequestProto.newBuilder();
     InetSocketAddress address = tal.getTaskCommunicator(event.getTaskCommId()).getAddress();
     builder.setAmHost(address.getHostName()).setAmPort(address.getPort());
-    builder.setAppAttemptNumber(event.getContainer().getId().getApplicationAttemptId().getAttemptId());
-    builder.setApplicationIdString(
-        event.getContainer().getId().getApplicationAttemptId().getApplicationId().toString());
+    builder.setAppAttemptNumber(appAttemptId.getAttemptId());
+    builder.setApplicationIdString(appAttemptId.getApplicationId().toString());
     builder.setTokenIdentifier(tokenIdentifier);
     builder.setContainerIdString(event.getContainer().getId().toString());
     builder.setCredentialsBinary(


[41/50] [abbrv] tez git commit: TEZ-2388. Send dag identifier as part of the fetcher request string. (sseth)

Posted by ss...@apache.org.
TEZ-2388. Send dag identifier as part of the fetcher request string. (sseth)


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

Branch: refs/heads/TEZ-2003
Commit: e8791286ab87cf797f79d2d90b49f206febf151a
Parents: 48d9842
Author: Siddharth Seth <ss...@apache.org>
Authored: Wed Apr 29 08:20:05 2015 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Wed May 6 00:18:08 2015 -0700

----------------------------------------------------------------------
 TEZ-2003-CHANGES.txt                                  |  1 +
 .../tez/runtime/library/common/shuffle/Fetcher.java   | 14 ++++++++------
 .../runtime/library/common/shuffle/ShuffleUtils.java  |  8 +++++---
 .../library/common/shuffle/impl/ShuffleManager.java   |  2 +-
 .../ShuffleInputEventHandlerOrderedGrouped.java       |  2 +-
 .../runtime/library/common/shuffle/TestFetcher.java   |  6 +++---
 6 files changed, 19 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/e8791286/TEZ-2003-CHANGES.txt
----------------------------------------------------------------------
diff --git a/TEZ-2003-CHANGES.txt b/TEZ-2003-CHANGES.txt
index d42aaf8..9fc9ed3 100644
--- a/TEZ-2003-CHANGES.txt
+++ b/TEZ-2003-CHANGES.txt
@@ -19,5 +19,6 @@ ALL CHANGES:
   TEZ-2347. Expose additional information in TaskCommunicatorContext.
   TEZ-2361. Propagate dag completion to TaskCommunicator.
   TEZ-2381. Fixes after rebase 04/28.
+  TEZ-2388. Send dag identifier as part of the fetcher request string.
 
 INCOMPATIBLE CHANGES:

http://git-wip-us.apache.org/repos/asf/tez/blob/e8791286/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/Fetcher.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/Fetcher.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/Fetcher.java
index 48fe0f2..a553210 100644
--- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/Fetcher.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/Fetcher.java
@@ -87,6 +87,7 @@ public class Fetcher extends CallableWithNdc<FetchResult> {
   private final FetcherCallback fetcherCallback;
   private final FetchedInputAllocator inputManager;
   private final ApplicationId appId;
+  private final int dagIdentifier;
   
   private final String logIdentifier;
 
@@ -125,7 +126,7 @@ public class Fetcher extends CallableWithNdc<FetchResult> {
   private final boolean isDebugEnabled = LOG.isDebugEnabled();
 
   private Fetcher(FetcherCallback fetcherCallback, HttpConnectionParams params,
-      FetchedInputAllocator inputManager, ApplicationId appId,
+      FetchedInputAllocator inputManager, ApplicationId appId, int dagIdentifier,
       JobTokenSecretManager jobTokenSecretManager, String srcNameTrimmed, Configuration conf,
       RawLocalFileSystem localFs,
       LocalDirAllocator localDirAllocator,
@@ -137,6 +138,7 @@ public class Fetcher extends CallableWithNdc<FetchResult> {
     this.inputManager = inputManager;
     this.jobTokenSecretMgr = jobTokenSecretManager;
     this.appId = appId;
+    this.dagIdentifier = dagIdentifier;
     this.pathToAttemptMap = new HashMap<String, InputAttemptIdentifier>();
     this.httpConnectionParams = params;
     this.conf = conf;
@@ -401,7 +403,7 @@ public class Fetcher extends CallableWithNdc<FetchResult> {
   private HostFetchResult setupConnection(List<InputAttemptIdentifier> attempts) {
     try {
       StringBuilder baseURI = ShuffleUtils.constructBaseURIForShuffleHandler(host,
-          port, partition, appId.toString(), httpConnectionParams.isSSLShuffleEnabled());
+          port, partition, appId.toString(), dagIdentifier, httpConnectionParams.isSSLShuffleEnabled());
       this.url = ShuffleUtils.constructInputURL(baseURI.toString(), attempts,
           httpConnectionParams.getKeepAlive());
 
@@ -901,21 +903,21 @@ public class Fetcher extends CallableWithNdc<FetchResult> {
 
     public FetcherBuilder(FetcherCallback fetcherCallback,
         HttpConnectionParams params, FetchedInputAllocator inputManager,
-        ApplicationId appId, JobTokenSecretManager jobTokenSecretMgr, String srcNameTrimmed,
+        ApplicationId appId, int dagIdentifier, JobTokenSecretManager jobTokenSecretMgr, String srcNameTrimmed,
         Configuration conf, boolean localDiskFetchEnabled, String localHostname) {
-      this.fetcher = new Fetcher(fetcherCallback, params, inputManager, appId,
+      this.fetcher = new Fetcher(fetcherCallback, params, inputManager, appId, dagIdentifier,
           jobTokenSecretMgr, srcNameTrimmed, conf, null, null, null, localDiskFetchEnabled,
           false, localHostname);
     }
 
     public FetcherBuilder(FetcherCallback fetcherCallback,
         HttpConnectionParams params, FetchedInputAllocator inputManager,
-        ApplicationId appId, JobTokenSecretManager jobTokenSecretMgr, String srcNameTrimmed,
+        ApplicationId appId, int dagIdentifier, JobTokenSecretManager jobTokenSecretMgr, String srcNameTrimmed,
         Configuration conf, RawLocalFileSystem localFs,
         LocalDirAllocator localDirAllocator, Path lockPath,
         boolean localDiskFetchEnabled, boolean sharedFetchEnabled,
         String localHostname) {
-      this.fetcher = new Fetcher(fetcherCallback, params, inputManager, appId,
+      this.fetcher = new Fetcher(fetcherCallback, params, inputManager, appId, dagIdentifier,
           jobTokenSecretMgr, srcNameTrimmed, conf, localFs, localDirAllocator,
           lockPath, localDiskFetchEnabled, sharedFetchEnabled, localHostname);
     }

http://git-wip-us.apache.org/repos/asf/tez/blob/e8791286/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/ShuffleUtils.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/ShuffleUtils.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/ShuffleUtils.java
index 46489ed..d7cb7c1 100644
--- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/ShuffleUtils.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/ShuffleUtils.java
@@ -174,19 +174,21 @@ public class ShuffleUtils {
 
   // TODO NEWTEZ handle ssl shuffle
   public static StringBuilder constructBaseURIForShuffleHandler(String host,
-      int port, int partition, String appId, boolean sslShuffle) {
+      int port, int partition, String appId, int dagIdentifier, boolean sslShuffle) {
     return constructBaseURIForShuffleHandler(host + ":" + String.valueOf(port),
-      partition, appId, sslShuffle);
+      partition, appId, dagIdentifier, sslShuffle);
   }
   
   public static StringBuilder constructBaseURIForShuffleHandler(String hostIdentifier,
-      int partition, String appId, boolean sslShuffle) {
+      int partition, String appId, int dagIdentifier, boolean sslShuffle) {
     final String http_protocol = (sslShuffle) ? "https://" : "http://";
     StringBuilder sb = new StringBuilder(http_protocol);
     sb.append(hostIdentifier);
     sb.append("/");
     sb.append("mapOutput?job=");
     sb.append(appId.replace("application", "job"));
+    sb.append("&dag=");
+    sb.append(String.valueOf(dagIdentifier));
     sb.append("&reduce=");
     sb.append(String.valueOf(partition));
     sb.append("&map=");

http://git-wip-us.apache.org/repos/asf/tez/blob/e8791286/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleManager.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleManager.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleManager.java
index d47e652..a8d3553 100644
--- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleManager.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleManager.java
@@ -387,7 +387,7 @@ public class ShuffleManager implements FetcherCallback {
     }
 
     FetcherBuilder fetcherBuilder = new FetcherBuilder(ShuffleManager.this,
-      httpConnectionParams, inputManager, inputContext.getApplicationId(),
+      httpConnectionParams, inputManager, inputContext.getApplicationId(), inputContext.getDagIdentifier(),
         jobTokenSecretMgr, srcNameTrimmed, conf, localFs, localDirAllocator,
         lockDisk, localDiskFetchEnabled, sharedFetchEnabled,
         inputContext.getExecutionContext().getHostName());

http://git-wip-us.apache.org/repos/asf/tez/blob/e8791286/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/ShuffleInputEventHandlerOrderedGrouped.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/ShuffleInputEventHandlerOrderedGrouped.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/ShuffleInputEventHandlerOrderedGrouped.java
index 32ac766..9481e65 100644
--- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/ShuffleInputEventHandlerOrderedGrouped.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/ShuffleInputEventHandlerOrderedGrouped.java
@@ -127,7 +127,7 @@ public class ShuffleInputEventHandlerOrderedGrouped {
   @VisibleForTesting
   URI getBaseURI(String host, int port, int partitionId) {
     StringBuilder sb = ShuffleUtils.constructBaseURIForShuffleHandler(host, port,
-      partitionId, inputContext.getApplicationId().toString(), sslShuffle);
+      partitionId, inputContext.getApplicationId().toString(), inputContext.getDagIdentifier(), sslShuffle);
     URI u = URI.create(sb.toString());
     return u;
   }

http://git-wip-us.apache.org/repos/asf/tez/blob/e8791286/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/TestFetcher.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/TestFetcher.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/TestFetcher.java
index e6f0c4a..081efb2 100644
--- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/TestFetcher.java
+++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/TestFetcher.java
@@ -64,7 +64,7 @@ public class TestFetcher {
     FetcherCallback fetcherCallback = mock(FetcherCallback.class);
 
     Fetcher.FetcherBuilder builder = new Fetcher.FetcherBuilder(fetcherCallback, null, null,
-        ApplicationId.newInstance(0, 1), null, "fetcherTest", conf, true, HOST);
+        ApplicationId.newInstance(0, 1), 1, null, "fetcherTest", conf, true, HOST);
     builder.assignWork(HOST, PORT, 0, Arrays.asList(srcAttempts));
     Fetcher fetcher = spy(builder.build());
 
@@ -82,7 +82,7 @@ public class TestFetcher {
     // When disabled use http fetch
     conf.set(TezRuntimeConfiguration.TEZ_RUNTIME_OPTIMIZE_LOCAL_FETCH, "false");
     builder = new Fetcher.FetcherBuilder(fetcherCallback, null, null,
-        ApplicationId.newInstance(0, 1), null, "fetcherTest", conf, false, HOST);
+        ApplicationId.newInstance(0, 1), 1,  null, "fetcherTest", conf, false, HOST);
     builder.assignWork(HOST, PORT, 0, Arrays.asList(srcAttempts));
     fetcher = spy(builder.build());
 
@@ -115,7 +115,7 @@ public class TestFetcher {
     int partition = 42;
     FetcherCallback callback = mock(FetcherCallback.class);
     Fetcher.FetcherBuilder builder = new Fetcher.FetcherBuilder(callback, null, null,
-        ApplicationId.newInstance(0, 1), null, "fetcherTest", conf, true, HOST);
+        ApplicationId.newInstance(0, 1), 1, null, "fetcherTest", conf, true, HOST);
     builder.assignWork(HOST, PORT, partition, Arrays.asList(srcAttempts));
     Fetcher fetcher = spy(builder.build());
 


[47/50] [abbrv] tez git commit: TEZ-2241. Miscellaneous fixes after last reabse. (sseth)

Posted by ss...@apache.org.
TEZ-2241. Miscellaneous fixes after last reabse. (sseth)


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

Branch: refs/heads/TEZ-2003
Commit: 8c9d3cf36d4f03b41789506bddf67bd6daabfa86
Parents: c115a4a
Author: Siddharth Seth <ss...@apache.org>
Authored: Thu Mar 26 17:57:55 2015 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Wed May 6 00:18:08 2015 -0700

----------------------------------------------------------------------
 TEZ-2003-CHANGES.txt                                          | 1 +
 .../org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java  | 2 ++
 .../org/apache/tez/dag/app/TezLocalTaskCommunicatorImpl.java  | 6 +++---
 .../java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java  | 6 +++---
 .../apache/tez/dag/app/launcher/ContainerLauncherRouter.java  | 6 +++---
 .../test/java/org/apache/tez/dag/app/MockDAGAppMaster.java    | 2 +-
 tez-ext-service-tests/pom.xml                                 | 4 ++--
 .../tez/dag/app/launcher/TezTestServiceContainerLauncher.java | 6 +++---
 .../dag/app/launcher/TezTestServiceNoOpContainerLauncher.java | 6 +++---
 .../tez/dag/app/rm/TezTestServiceTaskSchedulerService.java    | 7 ++++---
 .../dag/app/taskcomm/TezTestServiceTaskCommunicatorImpl.java  | 7 ++++---
 .../org/apache/tez/service/MiniTezTestServiceCluster.java     | 6 +++---
 .../java/org/apache/tez/service/impl/ContainerRunnerImpl.java | 5 +++--
 .../test/java/org/apache/tez/service/impl/TezTestService.java | 4 ++--
 .../tez/service/impl/TezTestServiceProtocolServerImpl.java    | 6 +++---
 .../org/apache/tez/shufflehandler/FadvisedChunkedFile.java    | 6 +++---
 .../org/apache/tez/shufflehandler/FadvisedFileRegion.java     | 6 +++---
 .../test/java/org/apache/tez/shufflehandler/IndexCache.java   | 6 +++---
 .../java/org/apache/tez/shufflehandler/ShuffleHandler.java    | 6 +++---
 .../java/org/apache/tez/tests/TestExternalTezServices.java    | 6 +++---
 20 files changed, 55 insertions(+), 49 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/8c9d3cf3/TEZ-2003-CHANGES.txt
----------------------------------------------------------------------
diff --git a/TEZ-2003-CHANGES.txt b/TEZ-2003-CHANGES.txt
index 774a685..d51686d 100644
--- a/TEZ-2003-CHANGES.txt
+++ b/TEZ-2003-CHANGES.txt
@@ -11,5 +11,6 @@ ALL CHANGES:
   TEZ-2139. Update tez version to 0.7.0-TEZ-2003-SNAPSHOT.
   TEZ-2175. Task priority should be available to the TaskCommunicator plugin.
   TEZ-2187. Allow TaskCommunicators to report failed / killed attempts.
+  TEZ-2241. Miscellaneous fixes after last reabse.
 
 INCOMPATIBLE CHANGES:

http://git-wip-us.apache.org/repos/asf/tez/blob/8c9d3cf3/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 c6bc79a..aaaa01c 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
@@ -67,6 +67,8 @@ import org.apache.tez.dag.records.TezTaskAttemptID;
 import org.apache.tez.dag.records.TezVertexID;
 import org.apache.tez.runtime.api.impl.TezEvent;
 import org.apache.tez.common.security.JobTokenSecretManager;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 
 @SuppressWarnings("unchecked")

http://git-wip-us.apache.org/repos/asf/tez/blob/8c9d3cf3/tez-dag/src/main/java/org/apache/tez/dag/app/TezLocalTaskCommunicatorImpl.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/TezLocalTaskCommunicatorImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/TezLocalTaskCommunicatorImpl.java
index 3704cc4..47688d1 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/TezLocalTaskCommunicatorImpl.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/TezLocalTaskCommunicatorImpl.java
@@ -18,14 +18,14 @@ import java.net.InetAddress;
 import java.net.InetSocketAddress;
 import java.net.UnknownHostException;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.tez.dag.api.TaskCommunicatorContext;
 import org.apache.tez.dag.api.TezUncheckedException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 public class TezLocalTaskCommunicatorImpl extends TezTaskCommunicatorImpl {
 
-  private static final Log LOG = LogFactory.getLog(TezLocalTaskCommunicatorImpl.class);
+  private static final Logger LOG = LoggerFactory.getLogger(TezLocalTaskCommunicatorImpl.class);
 
   public TezLocalTaskCommunicatorImpl(
       TaskCommunicatorContext taskCommunicatorContext) {

http://git-wip-us.apache.org/repos/asf/tez/blob/8c9d3cf3/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java
index f288748..9d0c031 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java
@@ -23,8 +23,6 @@ import java.util.concurrent.ConcurrentMap;
 
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Maps;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
@@ -55,11 +53,13 @@ import org.apache.tez.dag.records.TezTaskAttemptID;
 import org.apache.tez.runtime.api.impl.TaskSpec;
 import org.apache.tez.runtime.api.impl.TezHeartbeatRequest;
 import org.apache.tez.runtime.api.impl.TezHeartbeatResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 @InterfaceAudience.Private
 public class TezTaskCommunicatorImpl extends TaskCommunicator {
 
-  private static final Log LOG = LogFactory.getLog(TezTaskCommunicatorImpl.class);
+  private static final Logger LOG = LoggerFactory.getLogger(TezTaskCommunicatorImpl.class);
 
   private static final ContainerTask TASK_FOR_INVALID_JVM = new ContainerTask(
       null, true, null, null, false);

http://git-wip-us.apache.org/repos/asf/tez/blob/8c9d3cf3/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherRouter.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherRouter.java b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherRouter.java
index 70b0cbc..dd3571e 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherRouter.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherRouter.java
@@ -19,8 +19,6 @@ import java.lang.reflect.InvocationTargetException;
 import java.net.UnknownHostException;
 
 import com.google.common.annotations.VisibleForTesting;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.service.AbstractService;
 import org.apache.hadoop.yarn.event.EventHandler;
@@ -30,11 +28,13 @@ 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.app.rm.NMCommunicatorEvent;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 public class ContainerLauncherRouter extends AbstractService
     implements EventHandler<NMCommunicatorEvent> {
 
-  static final Log LOG = LogFactory.getLog(ContainerLauncherImpl.class);
+  static final Logger LOG = LoggerFactory.getLogger(ContainerLauncherImpl.class);
 
   private final ContainerLauncher containerLaunchers[];
 

http://git-wip-us.apache.org/repos/asf/tez/blob/8c9d3cf3/tez-dag/src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java b/tez-dag/src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java
index 73e93c5..d40b78d 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java
@@ -336,7 +336,7 @@ public class MockDAGAppMaster extends DAGAppMaster {
       long startTime = System.nanoTime();
       long startCpuTime = threadMxBean.getCurrentThreadCpuTime();
       TaskHeartbeatResponse response = taListener.heartbeat(request);
-      if (response.shouldDie()) {
+      if (response.isShouldDie()) {
         cData.remove();
       } else {
         cData.nextFromEventId += response.getEvents().size();

http://git-wip-us.apache.org/repos/asf/tez/blob/8c9d3cf3/tez-ext-service-tests/pom.xml
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/pom.xml b/tez-ext-service-tests/pom.xml
index 1113341..907e129 100644
--- a/tez-ext-service-tests/pom.xml
+++ b/tez-ext-service-tests/pom.xml
@@ -28,8 +28,8 @@
 
   <dependencies>
     <dependency>
-      <groupId>log4j</groupId>
-      <artifactId>log4j</artifactId>
+      <groupId>org.slf4j</groupId>
+      <artifactId>slf4j-log4j12</artifactId>
     </dependency>
     <dependency>
       <groupId>com.google.guava</groupId>

http://git-wip-us.apache.org/repos/asf/tez/blob/8c9d3cf3/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/launcher/TezTestServiceContainerLauncher.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/launcher/TezTestServiceContainerLauncher.java b/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/launcher/TezTestServiceContainerLauncher.java
index cb6308c..9d22196 100644
--- a/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/launcher/TezTestServiceContainerLauncher.java
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/launcher/TezTestServiceContainerLauncher.java
@@ -18,8 +18,6 @@ import java.net.InetSocketAddress;
 
 import com.google.common.base.Preconditions;
 import com.google.protobuf.ByteString;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.service.AbstractService;
 import org.apache.hadoop.yarn.api.ApplicationConstants;
@@ -40,6 +38,8 @@ import org.apache.tez.dag.history.events.ContainerLaunchedEvent;
 import org.apache.tez.service.TezTestServiceConfConstants;
 import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos;
 import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos.RunContainerRequestProto;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 public class TezTestServiceContainerLauncher extends AbstractService implements ContainerLauncher {
 
@@ -47,7 +47,7 @@ public class TezTestServiceContainerLauncher extends AbstractService implements
 
   // TODO May need multiple connections per target machine, depending upon how synchronization is handled in the RPC layer
 
-  static final Log LOG = LogFactory.getLog(TezTestServiceContainerLauncher.class);
+  static final Logger LOG = LoggerFactory.getLogger(TezTestServiceContainerLauncher.class);
 
   private final AppContext context;
   private final String tokenIdentifier;

http://git-wip-us.apache.org/repos/asf/tez/blob/8c9d3cf3/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/launcher/TezTestServiceNoOpContainerLauncher.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/launcher/TezTestServiceNoOpContainerLauncher.java b/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/launcher/TezTestServiceNoOpContainerLauncher.java
index 8c8e486..977d0d3 100644
--- a/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/launcher/TezTestServiceNoOpContainerLauncher.java
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/launcher/TezTestServiceNoOpContainerLauncher.java
@@ -14,8 +14,6 @@
 
 package org.apache.tez.dag.app.launcher;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.service.AbstractService;
 import org.apache.hadoop.yarn.util.Clock;
@@ -28,10 +26,12 @@ import org.apache.tez.dag.app.rm.container.AMContainerEventLaunched;
 import org.apache.tez.dag.app.rm.container.AMContainerEventType;
 import org.apache.tez.dag.history.DAGHistoryEvent;
 import org.apache.tez.dag.history.events.ContainerLaunchedEvent;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 public class TezTestServiceNoOpContainerLauncher extends AbstractService implements ContainerLauncher {
 
-  static final Log LOG = LogFactory.getLog(TezTestServiceNoOpContainerLauncher.class);
+  static final Logger LOG = LoggerFactory.getLogger(TezTestServiceNoOpContainerLauncher.class);
 
   private final AppContext context;
   private final Clock clock;

http://git-wip-us.apache.org/repos/asf/tez/blob/8c9d3cf3/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/rm/TezTestServiceTaskSchedulerService.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/rm/TezTestServiceTaskSchedulerService.java b/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/rm/TezTestServiceTaskSchedulerService.java
index 872d592..50dfb24 100644
--- a/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/rm/TezTestServiceTaskSchedulerService.java
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/rm/TezTestServiceTaskSchedulerService.java
@@ -28,8 +28,6 @@ import java.util.concurrent.atomic.AtomicInteger;
 import com.google.common.base.Preconditions;
 import com.google.common.primitives.Ints;
 import com.google.common.util.concurrent.ThreadFactoryBuilder;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
@@ -41,11 +39,14 @@ import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.tez.dag.api.TaskAttemptEndReason;
 import org.apache.tez.dag.app.AppContext;
 import org.apache.tez.service.TezTestServiceConfConstants;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 
 public class TezTestServiceTaskSchedulerService extends TaskSchedulerService {
 
-  private static final Log LOG = LogFactory.getLog(TezTestServiceTaskSchedulerService.class);
+  private static final Logger
+      LOG = LoggerFactory.getLogger(TezTestServiceTaskSchedulerService.class);
 
   private final ExecutorService appCallbackExecutor;
   private final TaskSchedulerAppCallback appClientDelegate;

http://git-wip-us.apache.org/repos/asf/tez/blob/8c9d3cf3/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/taskcomm/TezTestServiceTaskCommunicatorImpl.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/taskcomm/TezTestServiceTaskCommunicatorImpl.java b/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/taskcomm/TezTestServiceTaskCommunicatorImpl.java
index e3385a2..ef983c2 100644
--- a/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/taskcomm/TezTestServiceTaskCommunicatorImpl.java
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/taskcomm/TezTestServiceTaskCommunicatorImpl.java
@@ -23,8 +23,6 @@ import java.util.concurrent.RejectedExecutionException;
 
 import com.google.protobuf.ByteString;
 import com.google.protobuf.ServiceException;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.DataOutputBuffer;
 import org.apache.hadoop.ipc.RemoteException;
@@ -41,11 +39,14 @@ import org.apache.tez.runtime.api.impl.TaskSpec;
 import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos.SubmitWorkRequestProto;
 import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos.SubmitWorkResponseProto;
 import org.apache.tez.util.ProtoConverters;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 
 public class TezTestServiceTaskCommunicatorImpl extends TezTaskCommunicatorImpl {
 
-  private static final Log LOG = LogFactory.getLog(TezTestServiceTaskCommunicatorImpl.class);
+  private static final Logger
+      LOG = LoggerFactory.getLogger(TezTestServiceTaskCommunicatorImpl.class);
 
   private final TezTestServiceCommunicator communicator;
   private final SubmitWorkRequestProto BASE_SUBMIT_WORK_REQUEST;

http://git-wip-us.apache.org/repos/asf/tez/blob/8c9d3cf3/tez-ext-service-tests/src/test/java/org/apache/tez/service/MiniTezTestServiceCluster.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/service/MiniTezTestServiceCluster.java b/tez-ext-service-tests/src/test/java/org/apache/tez/service/MiniTezTestServiceCluster.java
index 0ac0b33..c5ff02d 100644
--- a/tez-ext-service-tests/src/test/java/org/apache/tez/service/MiniTezTestServiceCluster.java
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/service/MiniTezTestServiceCluster.java
@@ -19,8 +19,6 @@ import java.io.IOException;
 import java.net.InetSocketAddress;
 
 import com.google.common.base.Preconditions;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileContext;
 import org.apache.hadoop.fs.Path;
@@ -28,10 +26,12 @@ import org.apache.hadoop.service.AbstractService;
 import org.apache.hadoop.util.Shell;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.tez.service.impl.TezTestService;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 public class MiniTezTestServiceCluster extends AbstractService {
 
-  private static final Log LOG = LogFactory.getLog(MiniTezTestServiceCluster.class);
+  private static final Logger LOG = LoggerFactory.getLogger(MiniTezTestServiceCluster.class);
 
   private final File testWorkDir;
   private final long availableMemory;

http://git-wip-us.apache.org/repos/asf/tez/blob/8c9d3cf3/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/ContainerRunnerImpl.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/ContainerRunnerImpl.java b/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/ContainerRunnerImpl.java
index 379d952..e7c02c8 100644
--- a/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/ContainerRunnerImpl.java
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/ContainerRunnerImpl.java
@@ -52,7 +52,6 @@ import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.service.AbstractService;
 import org.apache.hadoop.yarn.api.ApplicationConstants;
 import org.apache.hadoop.yarn.util.AuxiliaryServiceHelper;
-import org.apache.log4j.Logger;
 import org.apache.tez.common.TezCommonUtils;
 import org.apache.tez.common.TezTaskUmbilicalProtocol;
 import org.apache.tez.common.security.JobTokenIdentifier;
@@ -75,10 +74,12 @@ import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos.RunContainer
 import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos.SubmitWorkRequestProto;
 import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos.TaskSpecProto;
 import org.apache.tez.util.ProtoConverters;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 public class ContainerRunnerImpl extends AbstractService implements ContainerRunner {
 
-  private static final Logger LOG = Logger.getLogger(ContainerRunnerImpl.class);
+  private static final Logger LOG = LoggerFactory.getLogger(ContainerRunnerImpl.class);
 
   public static final String DAG_NAME_INSTRUMENTED_FAILURES = "InstrumentedFailures";
 

http://git-wip-us.apache.org/repos/asf/tez/blob/8c9d3cf3/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/TezTestService.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/TezTestService.java b/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/TezTestService.java
index 855f1b0..322be00 100644
--- a/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/TezTestService.java
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/TezTestService.java
@@ -23,16 +23,16 @@ import com.google.common.base.Preconditions;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.service.AbstractService;
 import org.apache.hadoop.util.StringUtils;
-import org.apache.log4j.Logger;
 import org.apache.tez.dag.api.TezException;
 import org.apache.tez.service.ContainerRunner;
 import org.apache.tez.shufflehandler.ShuffleHandler;
 import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos;
 import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos.RunContainerRequestProto;
+import org.slf4j.LoggerFactory;
 
 public class TezTestService extends AbstractService implements ContainerRunner {
 
-  private static final Logger LOG = Logger.getLogger(TezTestService.class);
+  private static final org.slf4j.Logger LOG = LoggerFactory.getLogger(TezTestService.class);
 
   private final Configuration shuffleHandlerConf;
   private final int numExecutors;

http://git-wip-us.apache.org/repos/asf/tez/blob/8c9d3cf3/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/TezTestServiceProtocolServerImpl.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/TezTestServiceProtocolServerImpl.java b/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/TezTestServiceProtocolServerImpl.java
index 39d7156..b5d3f83 100644
--- a/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/TezTestServiceProtocolServerImpl.java
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/TezTestServiceProtocolServerImpl.java
@@ -22,8 +22,6 @@ import com.google.common.annotations.VisibleForTesting;
 import com.google.protobuf.BlockingService;
 import com.google.protobuf.RpcController;
 import com.google.protobuf.ServiceException;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.ipc.ProtobufRpcEngine;
@@ -38,11 +36,13 @@ import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos.RunContainer
 import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos.RunContainerResponseProto;
 import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos.SubmitWorkRequestProto;
 import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos.SubmitWorkResponseProto;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 public class TezTestServiceProtocolServerImpl extends AbstractService
     implements TezTestServiceProtocolBlockingPB {
 
-  private static final Log LOG = LogFactory.getLog(TezTestServiceProtocolServerImpl.class);
+  private static final Logger LOG = LoggerFactory.getLogger(TezTestServiceProtocolServerImpl.class);
 
   private final ContainerRunner containerRunner;
   private RPC.Server server;

http://git-wip-us.apache.org/repos/asf/tez/blob/8c9d3cf3/tez-ext-service-tests/src/test/java/org/apache/tez/shufflehandler/FadvisedChunkedFile.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/shufflehandler/FadvisedChunkedFile.java b/tez-ext-service-tests/src/test/java/org/apache/tez/shufflehandler/FadvisedChunkedFile.java
index 65588fe..294add6 100644
--- a/tez-ext-service-tests/src/test/java/org/apache/tez/shufflehandler/FadvisedChunkedFile.java
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/shufflehandler/FadvisedChunkedFile.java
@@ -18,16 +18,16 @@ import java.io.FileDescriptor;
 import java.io.IOException;
 import java.io.RandomAccessFile;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.io.ReadaheadPool;
 import org.apache.hadoop.io.ReadaheadPool.ReadaheadRequest;
 import org.apache.hadoop.io.nativeio.NativeIO;
 import org.jboss.netty.handler.stream.ChunkedFile;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 public class FadvisedChunkedFile extends ChunkedFile {
 
-  private static final Log LOG = LogFactory.getLog(FadvisedChunkedFile.class);
+  private static final Logger LOG = LoggerFactory.getLogger(FadvisedChunkedFile.class);
 
   private final boolean manageOsCache;
   private final int readaheadLength;

http://git-wip-us.apache.org/repos/asf/tez/blob/8c9d3cf3/tez-ext-service-tests/src/test/java/org/apache/tez/shufflehandler/FadvisedFileRegion.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/shufflehandler/FadvisedFileRegion.java b/tez-ext-service-tests/src/test/java/org/apache/tez/shufflehandler/FadvisedFileRegion.java
index bdffe52..e5392d3 100644
--- a/tez-ext-service-tests/src/test/java/org/apache/tez/shufflehandler/FadvisedFileRegion.java
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/shufflehandler/FadvisedFileRegion.java
@@ -22,16 +22,16 @@ import java.nio.channels.FileChannel;
 import java.nio.channels.WritableByteChannel;
 
 import com.google.common.annotations.VisibleForTesting;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.io.ReadaheadPool;
 import org.apache.hadoop.io.ReadaheadPool.ReadaheadRequest;
 import org.apache.hadoop.io.nativeio.NativeIO;
 import org.jboss.netty.channel.DefaultFileRegion;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 public class FadvisedFileRegion extends DefaultFileRegion {
 
-  private static final Log LOG = LogFactory.getLog(FadvisedFileRegion.class);
+  private static final Logger LOG = LoggerFactory.getLogger(FadvisedFileRegion.class);
 
   private final boolean manageOsCache;
   private final int readaheadLength;

http://git-wip-us.apache.org/repos/asf/tez/blob/8c9d3cf3/tez-ext-service-tests/src/test/java/org/apache/tez/shufflehandler/IndexCache.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/shufflehandler/IndexCache.java b/tez-ext-service-tests/src/test/java/org/apache/tez/shufflehandler/IndexCache.java
index 9a51ca0..5a45917 100644
--- a/tez-ext-service-tests/src/test/java/org/apache/tez/shufflehandler/IndexCache.java
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/shufflehandler/IndexCache.java
@@ -22,20 +22,20 @@ import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.LinkedBlockingQueue;
 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.Path;
 import org.apache.tez.runtime.library.common.Constants;
 import org.apache.tez.runtime.library.common.sort.impl.TezIndexRecord;
 import org.apache.tez.runtime.library.common.sort.impl.TezSpillRecord;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 class IndexCache {
 
   private final Configuration conf;
   private final int totalMemoryAllowed;
   private AtomicInteger totalMemoryUsed = new AtomicInteger();
-  private static final Log LOG = LogFactory.getLog(IndexCache.class);
+  private static final Logger LOG = LoggerFactory.getLogger(IndexCache.class);
 
   private final ConcurrentHashMap<String,IndexInformation> cache =
       new ConcurrentHashMap<String,IndexInformation>();

http://git-wip-us.apache.org/repos/asf/tez/blob/8c9d3cf3/tez-ext-service-tests/src/test/java/org/apache/tez/shufflehandler/ShuffleHandler.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/shufflehandler/ShuffleHandler.java b/tez-ext-service-tests/src/test/java/org/apache/tez/shufflehandler/ShuffleHandler.java
index cc82d74..8cbb8c7 100644
--- a/tez-ext-service-tests/src/test/java/org/apache/tez/shufflehandler/ShuffleHandler.java
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/shufflehandler/ShuffleHandler.java
@@ -51,8 +51,6 @@ import java.util.regex.Pattern;
 import com.google.common.base.Charsets;
 import com.google.common.base.Preconditions;
 import com.google.common.util.concurrent.ThreadFactoryBuilder;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.LocalDirAllocator;
 import org.apache.hadoop.fs.Path;
@@ -105,10 +103,12 @@ import org.jboss.netty.handler.codec.http.QueryStringDecoder;
 import org.jboss.netty.handler.ssl.SslHandler;
 import org.jboss.netty.handler.stream.ChunkedWriteHandler;
 import org.jboss.netty.util.CharsetUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 public class ShuffleHandler {
 
-  private static final Log LOG = LogFactory.getLog(ShuffleHandler.class);
+  private static final Logger LOG = LoggerFactory.getLogger(ShuffleHandler.class);
 
   public static final String SHUFFLE_HANDLER_LOCAL_DIRS = "tez.shuffle.handler.local-dirs";
 

http://git-wip-us.apache.org/repos/asf/tez/blob/8c9d3cf3/tez-ext-service-tests/src/test/java/org/apache/tez/tests/TestExternalTezServices.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/tests/TestExternalTezServices.java b/tez-ext-service-tests/src/test/java/org/apache/tez/tests/TestExternalTezServices.java
index b6a166d..4d0a610 100644
--- a/tez-ext-service-tests/src/test/java/org/apache/tez/tests/TestExternalTezServices.java
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/tests/TestExternalTezServices.java
@@ -20,8 +20,6 @@ import java.io.IOException;
 import java.util.Map;
 
 import com.google.common.collect.Maps;
-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;
@@ -48,10 +46,12 @@ import org.apache.tez.test.MiniTezCluster;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 public class TestExternalTezServices {
 
-  private static final Log LOG = LogFactory.getLog(TestExternalTezServices.class);
+  private static final Logger LOG = LoggerFactory.getLogger(TestExternalTezServices.class);
 
   private static final String EXT_PUSH_ENTITY_NAME = "ExtServiceTestPush";
 


[40/50] [abbrv] tez git commit: TEZ-2187. Allow TaskCommunicators to report failed / killed attempts. (sseth)

Posted by ss...@apache.org.
TEZ-2187. Allow TaskCommunicators to report failed / killed attempts.  (sseth)


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

Branch: refs/heads/TEZ-2003
Commit: c115a4ac9400c46da0a7dc54e7eee44259331286
Parents: 25eed21
Author: Siddharth Seth <ss...@apache.org>
Authored: Tue Mar 10 01:25:39 2015 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Wed May 6 00:18:08 2015 -0700

----------------------------------------------------------------------
 TEZ-2003-CHANGES.txt                            |   1 +
 .../org/apache/tez/common/TezUtilsInternal.java |  60 +++++++++
 .../tez/dag/api/TaskAttemptEndReason.java       |  24 ++++
 .../records/TaskAttemptTerminationCause.java    |   7 +-
 .../apache/tez/dag/api/TaskCommunicator.java    |   2 +
 .../tez/dag/api/TaskCommunicatorContext.java    |  13 +-
 .../dag/app/TaskAttemptListenerImpTezDag.java   |  33 +++++
 .../event/TaskAttemptEventAttemptFailed.java    |   2 +
 .../event/TaskAttemptEventAttemptKilled.java    |  47 +++++++
 .../dag/app/dag/event/TaskAttemptEventType.java |   5 +-
 .../tez/dag/app/dag/impl/TaskAttemptImpl.java   |  33 ++++-
 .../tez/dag/app/rm/AMSchedulerEventTAEnded.java |   9 +-
 .../dag/app/rm/LocalTaskSchedulerService.java   |   3 +-
 .../dag/app/rm/TaskSchedulerEventHandler.java   |   7 +-
 .../tez/dag/app/rm/TaskSchedulerService.java    |   6 +-
 .../dag/app/rm/YarnTaskSchedulerService.java    |   8 +-
 .../app/TestTaskAttemptListenerImplTezDag.java  |   1 +
 .../app/TestTaskAttemptListenerImplTezDag2.java | 126 +++++++++++++++++++
 .../tez/dag/app/rm/TestContainerReuse.java      |  65 +++++-----
 .../app/rm/TestLocalTaskSchedulerService.java   |   5 +-
 .../tez/dag/app/rm/TestTaskScheduler.java       |  18 +--
 .../rm/TezTestServiceTaskSchedulerService.java  |   3 +-
 .../TezTestServiceTaskCommunicatorImpl.java     |  36 +++++-
 .../org/apache/tez/service/ContainerRunner.java |   5 +-
 .../tez/service/MiniTezTestServiceCluster.java  |   5 +-
 .../tez/service/impl/ContainerRunnerImpl.java   |  60 +++++++--
 .../apache/tez/service/impl/TezTestService.java |   6 +-
 .../impl/TezTestServiceProtocolServerImpl.java  |  10 +-
 .../tez/tests/TestExternalTezServices.java      |  29 +++++
 29 files changed, 548 insertions(+), 81 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/c115a4ac/TEZ-2003-CHANGES.txt
----------------------------------------------------------------------
diff --git a/TEZ-2003-CHANGES.txt b/TEZ-2003-CHANGES.txt
index 7726815..774a685 100644
--- a/TEZ-2003-CHANGES.txt
+++ b/TEZ-2003-CHANGES.txt
@@ -10,5 +10,6 @@ ALL CHANGES:
   TEZ-2138. Fix minor bugs in adding default scheduler, getting launchers.
   TEZ-2139. Update tez version to 0.7.0-TEZ-2003-SNAPSHOT.
   TEZ-2175. Task priority should be available to the TaskCommunicator plugin.
+  TEZ-2187. Allow TaskCommunicators to report failed / killed attempts.
 
 INCOMPATIBLE CHANGES:

http://git-wip-us.apache.org/repos/asf/tez/blob/c115a4ac/tez-common/src/main/java/org/apache/tez/common/TezUtilsInternal.java
----------------------------------------------------------------------
diff --git a/tez-common/src/main/java/org/apache/tez/common/TezUtilsInternal.java b/tez-common/src/main/java/org/apache/tez/common/TezUtilsInternal.java
index 9c78377..347a4f6 100644
--- a/tez-common/src/main/java/org/apache/tez/common/TezUtilsInternal.java
+++ b/tez-common/src/main/java/org/apache/tez/common/TezUtilsInternal.java
@@ -41,6 +41,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.security.Credentials;
 import org.apache.log4j.Appender;
 import org.apache.tez.dag.api.DagTypeConverters;
+import org.apache.tez.dag.api.TaskAttemptEndReason;
 import org.apache.tez.dag.api.TezConstants;
 import org.apache.tez.dag.api.records.DAGProtos;
 import org.apache.tez.dag.api.records.DAGProtos.ConfigurationProto;
@@ -49,6 +50,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import com.google.common.base.Stopwatch;
+import org.apache.tez.dag.records.TaskAttemptTerminationCause;
 
 @Private
 public class TezUtilsInternal {
@@ -234,4 +236,62 @@ public class TezUtilsInternal {
     return sb.toString();
   }
 
+  public static TaskAttemptTerminationCause fromTaskAttemptEndReason(
+      TaskAttemptEndReason taskAttemptEndReason) {
+    if (taskAttemptEndReason == null) {
+      return null;
+    }
+    switch (taskAttemptEndReason) {
+      case COMMUNICATION_ERROR:
+        return TaskAttemptTerminationCause.COMMUNICATION_ERROR;
+      case SERVICE_BUSY:
+        return TaskAttemptTerminationCause.SERVICE_BUSY;
+      case INTERRUPTED_BY_SYSTEM:
+        return TaskAttemptTerminationCause.INTERRUPTED_BY_SYSTEM;
+      case INTERRUPTED_BY_USER:
+        return TaskAttemptTerminationCause.INTERRUPTED_BY_USER;
+      case OTHER:
+        return TaskAttemptTerminationCause.UNKNOWN_ERROR;
+      default:
+        return TaskAttemptTerminationCause.UNKNOWN_ERROR;
+    }
+  }
+
+  public static TaskAttemptEndReason toTaskAttemptEndReason(TaskAttemptTerminationCause cause) {
+    // TODO Post TEZ-2003. Consolidate these states, and mappings.
+    if (cause == null) {
+      return null;
+    }
+    switch (cause) {
+      case COMMUNICATION_ERROR:
+        return TaskAttemptEndReason.COMMUNICATION_ERROR;
+      case SERVICE_BUSY:
+        return TaskAttemptEndReason.SERVICE_BUSY;
+      case INTERRUPTED_BY_SYSTEM:
+        return TaskAttemptEndReason.INTERRUPTED_BY_SYSTEM;
+      case INTERRUPTED_BY_USER:
+        return TaskAttemptEndReason.INTERRUPTED_BY_USER;
+      case UNKNOWN_ERROR:
+      case TERMINATED_BY_CLIENT:
+      case TERMINATED_AT_SHUTDOWN:
+      case INTERNAL_PREEMPTION:
+      case EXTERNAL_PREEMPTION:
+      case TERMINATED_INEFFECTIVE_SPECULATION:
+      case TERMINATED_EFFECTIVE_SPECULATION:
+      case TERMINATED_ORPHANED:
+      case APPLICATION_ERROR:
+      case FRAMEWORK_ERROR:
+      case INPUT_READ_ERROR:
+      case OUTPUT_WRITE_ERROR:
+      case OUTPUT_LOST:
+      case TASK_HEARTBEAT_ERROR:
+      case CONTAINER_LAUNCH_FAILED:
+      case CONTAINER_EXITED:
+      case CONTAINER_STOPPED:
+      case NODE_FAILED:
+      case NODE_DISK_ERROR:
+      default:
+        return TaskAttemptEndReason.OTHER;
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/c115a4ac/tez-common/src/main/java/org/apache/tez/dag/api/TaskAttemptEndReason.java
----------------------------------------------------------------------
diff --git a/tez-common/src/main/java/org/apache/tez/dag/api/TaskAttemptEndReason.java b/tez-common/src/main/java/org/apache/tez/dag/api/TaskAttemptEndReason.java
new file mode 100644
index 0000000..96a4768
--- /dev/null
+++ b/tez-common/src/main/java/org/apache/tez/dag/api/TaskAttemptEndReason.java
@@ -0,0 +1,24 @@
+/*
+ * 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.
+ */
+
+package org.apache.tez.dag.api;
+
+// TODO TEZ-2003 Expose as a public API
+public enum TaskAttemptEndReason {
+  COMMUNICATION_ERROR,
+  SERVICE_BUSY,
+  INTERRUPTED_BY_SYSTEM,
+  INTERRUPTED_BY_USER,
+  OTHER
+}

http://git-wip-us.apache.org/repos/asf/tez/blob/c115a4ac/tez-common/src/main/java/org/apache/tez/dag/records/TaskAttemptTerminationCause.java
----------------------------------------------------------------------
diff --git a/tez-common/src/main/java/org/apache/tez/dag/records/TaskAttemptTerminationCause.java b/tez-common/src/main/java/org/apache/tez/dag/records/TaskAttemptTerminationCause.java
index ef0bb33..7112d9e 100644
--- a/tez-common/src/main/java/org/apache/tez/dag/records/TaskAttemptTerminationCause.java
+++ b/tez-common/src/main/java/org/apache/tez/dag/records/TaskAttemptTerminationCause.java
@@ -41,5 +41,10 @@ public enum TaskAttemptTerminationCause {
   CONTAINER_STOPPED, // Container stopped or released by Tez
   NODE_FAILED, // Node for the container failed
   NODE_DISK_ERROR, // Disk failed on the node runnign the task
-  
+
+  COMMUNICATION_ERROR, // Equivalent to a launch failure
+  SERVICE_BUSY, // Service rejected the task
+  INTERRUPTED_BY_SYSTEM, // Interrupted by the system. e.g. Pre-emption
+  INTERRUPTED_BY_USER, // Interrupted by the user
+
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/c115a4ac/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicator.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicator.java b/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicator.java
index 82eed20..945091e 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicator.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicator.java
@@ -45,6 +45,8 @@ public abstract class TaskCommunicator extends AbstractService {
                                                   Credentials credentials,
                                                   boolean credentialsChanged, int priority);
 
+  // TODO TEZ-2003. Are additional APIs required to mark a container as completed ? - for completeness.
+
   // TODO TEZ-2003 Remove reference to TaskAttemptID
   public abstract void unregisterRunningTaskAttempt(TezTaskAttemptID taskAttemptID);
 

http://git-wip-us.apache.org/repos/asf/tez/blob/c115a4ac/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicatorContext.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicatorContext.java b/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicatorContext.java
index 41675fe..a85fb7f 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicatorContext.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicatorContext.java
@@ -14,6 +14,7 @@
 
 package org.apache.tez.dag.api;
 
+import javax.annotation.Nullable;
 import java.io.IOException;
 
 import org.apache.hadoop.security.Credentials;
@@ -37,15 +38,21 @@ public interface TaskCommunicatorContext {
   // TODO TEZ-2003 Move to vertex, taskIndex, version
   boolean canCommit(TezTaskAttemptID taskAttemptId) throws IOException;
 
+  // TODO TEZ-2003 Split the heartbeat API to a liveness check and a status update
   TaskHeartbeatResponse heartbeat(TaskHeartbeatRequest request) throws IOException, TezException;
 
   boolean isKnownContainer(ContainerId containerId);
 
-  // TODO TEZ-2003 Move to vertex, taskIndex, version
+  // TODO TEZ-2003 Move to vertex, taskIndex, version. Rename to taskAttempt*
   void taskStartedRemotely(TezTaskAttemptID taskAttemptID, ContainerId containerId);
 
-  // TODO TEZ-2003 Add an API to register task failure - for example, a communication failure.
-  // This will have to take into consideration the TA_FAILED event
+  // TODO TEZ-2003 Move to vertex, taskIndex, version. Rename to taskAttempt*
+  void taskKilled(TezTaskAttemptID taskAttemptId, TaskAttemptEndReason taskAttemptEndReason, @Nullable String diagnostics);
+
+  // TODO TEZ-2003 Move to vertex, taskIndex, version. Rename to taskAttempt*
+  void taskFailed(TezTaskAttemptID taskAttemptId, TaskAttemptEndReason taskAttemptEndReason, @Nullable String diagnostics);
+
+  // TODO TEZ-2003 API. Should a method exist for task succeeded.
 
   // TODO Eventually Add methods to report availability stats to the scheduler.
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/c115a4ac/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 a24a654..c6bc79a 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
@@ -37,14 +37,17 @@ import org.apache.tez.runtime.api.events.TaskStatusUpdateEvent;
 import org.apache.tez.runtime.api.impl.EventType;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+import com.google.common.annotations.VisibleForTesting;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.service.AbstractService;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.tez.common.ReflectionUtils;
+import org.apache.tez.common.TezUtilsInternal;
 import org.apache.tez.dag.api.TaskCommunicator;
 import org.apache.tez.dag.api.TaskCommunicatorContext;
+import org.apache.tez.dag.api.TaskAttemptEndReason;
 import org.apache.tez.dag.api.TaskHeartbeatResponse;
 import org.apache.tez.dag.api.TezConstants;
 import org.apache.tez.dag.api.TezException;
@@ -54,7 +57,10 @@ import org.apache.hadoop.yarn.util.ConverterUtils;
 import org.apache.tez.dag.api.TaskHeartbeatRequest;
 import org.apache.tez.dag.app.dag.DAG;
 import org.apache.tez.dag.app.dag.Task;
+import org.apache.tez.dag.app.dag.event.TaskAttemptEventAttemptFailed;
+import org.apache.tez.dag.app.dag.event.TaskAttemptEventAttemptKilled;
 import org.apache.tez.dag.app.dag.event.TaskAttemptEventStartedRemotely;
+import org.apache.tez.dag.app.dag.event.TaskAttemptEventType;
 import org.apache.tez.dag.app.dag.event.VertexEventRouteEvent;
 import org.apache.tez.dag.app.rm.container.AMContainerTask;
 import org.apache.tez.dag.records.TezTaskAttemptID;
@@ -257,6 +263,33 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
     pingContainerHeartbeatHandler(containerId);
   }
 
+  @Override
+  public void taskKilled(TezTaskAttemptID taskAttemptId, TaskAttemptEndReason taskAttemptEndReason,
+                         String diagnostics) {
+    // Regular flow via TaskAttempt will take care of un-registering from the heartbeat handler,
+    // and messages from the scheduler will release the container.
+    // TODO TEZ-2003 Maybe consider un-registering here itself, since the task is not active anymore,
+    // instead of waiting for the unregister to flow through the Container.
+    // Fix along the same lines as TEZ-2124 by introducing an explict context.
+    context.getEventHandler().handle(new TaskAttemptEventAttemptKilled(taskAttemptId,
+        diagnostics, TezUtilsInternal.fromTaskAttemptEndReason(
+        taskAttemptEndReason)));
+  }
+
+  @Override
+  public void taskFailed(TezTaskAttemptID taskAttemptId, TaskAttemptEndReason taskAttemptEndReason,
+                         String diagnostics) {
+    // Regular flow via TaskAttempt will take care of un-registering from the heartbeat handler,
+    // and messages from the scheduler will release the container.
+    // TODO TEZ-2003 Maybe consider un-registering here itself, since the task is not active anymore,
+    // instead of waiting for the unregister to flow through the Container.
+    // Fix along the same lines as TEZ-2124 by introducing an explict context.
+    context.getEventHandler().handle(new TaskAttemptEventAttemptFailed(taskAttemptId,
+        TaskAttemptEventType.TA_FAILED, diagnostics, TezUtilsInternal.fromTaskAttemptEndReason(
+        taskAttemptEndReason)));
+  }
+
+
   /**
    * Child checking whether it can commit.
    * <p/>

http://git-wip-us.apache.org/repos/asf/tez/blob/c115a4ac/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskAttemptEventAttemptFailed.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskAttemptEventAttemptFailed.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskAttemptEventAttemptFailed.java
index b9c1d09..7ec8921 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskAttemptEventAttemptFailed.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskAttemptEventAttemptFailed.java
@@ -26,6 +26,8 @@ public class TaskAttemptEventAttemptFailed extends TaskAttemptEvent
 
   private final String diagnostics;
   private final TaskAttemptTerminationCause errorCause;
+
+  /* Accepted Types - FAILED, TIMED_OUT */
   public TaskAttemptEventAttemptFailed(TezTaskAttemptID id,
       TaskAttemptEventType type, String diagnostics, TaskAttemptTerminationCause errorCause) {
     super(id, type);

http://git-wip-us.apache.org/repos/asf/tez/blob/c115a4ac/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskAttemptEventAttemptKilled.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskAttemptEventAttemptKilled.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskAttemptEventAttemptKilled.java
new file mode 100644
index 0000000..72e6b07
--- /dev/null
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskAttemptEventAttemptKilled.java
@@ -0,0 +1,47 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.TaskAttemptTerminationCause;
+import org.apache.tez.dag.records.TezTaskAttemptID;
+
+public class TaskAttemptEventAttemptKilled extends TaskAttemptEvent
+    implements DiagnosableEvent, TaskAttemptEventTerminationCauseEvent {
+
+  private final String diagnostics;
+  private final TaskAttemptTerminationCause errorCause;
+  public TaskAttemptEventAttemptKilled(TezTaskAttemptID id,
+                                       String diagnostics,
+                                       TaskAttemptTerminationCause errorCause) {
+    super(id, TaskAttemptEventType.TA_KILLED);
+    this.diagnostics = diagnostics;
+    this.errorCause = errorCause;
+  }
+
+  @Override
+  public String getDiagnosticInfo() {
+    return diagnostics;
+  }
+
+  @Override
+  public TaskAttemptTerminationCause getTerminationCause() {
+    return errorCause;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/tez/blob/c115a4ac/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 b7aca36..6d20368 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
@@ -26,14 +26,15 @@ public enum TaskAttemptEventType {
 //Producer:Task, Speculator
   TA_SCHEDULE,
 
-//Producer: TaskAttemptListener
+//Producer: TaskAttemptListener | Vertex after routing events
   TA_STARTED_REMOTELY,
   TA_STATUS_UPDATE,
   TA_DIAGNOSTICS_UPDATE, // REMOVE THIS - UNUSED
   TA_DONE,
   TA_FAILED,
+  TA_KILLED, // Generated by TaskCommunicators
   TA_TIMED_OUT,
-  
+
 //Producer: Client, Scheduler, On speculation.
   TA_KILL_REQUEST,
 

http://git-wip-us.apache.org/repos/asf/tez/blob/c115a4ac/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 c80571d..11d4df9 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
@@ -51,6 +51,7 @@ import org.apache.hadoop.yarn.state.StateMachineFactory;
 import org.apache.hadoop.yarn.util.Clock;
 import org.apache.hadoop.yarn.util.RackResolver;
 import org.apache.hadoop.yarn.util.Records;
+import org.apache.tez.common.TezUtilsInternal;
 import org.apache.tez.common.counters.DAGCounter;
 import org.apache.tez.common.counters.TezCounters;
 import org.apache.tez.dag.api.ProcessorDescriptor;
@@ -185,6 +186,11 @@ public class TaskAttemptImpl implements TaskAttempt,
 
   private final StateMachine<TaskAttemptStateInternal, TaskAttemptEventType, TaskAttemptEvent> stateMachine;
 
+  // TODO TEZ-2003 We may need some additional state management for STATUS_UPDATES, FAILED, KILLED coming in before
+  // TASK_STARTED_REMOTELY. In case of a PUSH it's more intuitive to send TASK_STARTED_REMOTELY after communicating
+  // with the listening service and getting a response, which in turn can trigger STATUS_UPDATES / FAILED / KILLED
+
+  // TA_KILLED handled the same as TA_KILL_REQUEST. Just a different name indicating a request / already killed.
   private static StateMachineFactory
   <TaskAttemptImpl, TaskAttemptStateInternal, TaskAttemptEventType, TaskAttemptEvent>
   stateMachineFactory
@@ -225,6 +231,10 @@ public class TaskAttemptImpl implements TaskAttempt,
           new TerminatedBeforeRunningTransition(KILLED_HELPER))
       .addTransition(TaskAttemptStateInternal.START_WAIT,
           TaskAttemptStateInternal.KILL_IN_PROGRESS,
+          TaskAttemptEventType.TA_KILLED,
+          new TerminatedBeforeRunningTransition(KILLED_HELPER))
+      .addTransition(TaskAttemptStateInternal.START_WAIT,
+          TaskAttemptStateInternal.KILL_IN_PROGRESS,
           TaskAttemptEventType.TA_NODE_FAILED,
           new NodeFailedBeforeRunningTransition())
       .addTransition(TaskAttemptStateInternal.START_WAIT,
@@ -265,6 +275,10 @@ public class TaskAttemptImpl implements TaskAttempt,
           new TerminatedWhileRunningTransition(KILLED_HELPER))
       .addTransition(TaskAttemptStateInternal.RUNNING,
           TaskAttemptStateInternal.KILL_IN_PROGRESS,
+          TaskAttemptEventType.TA_KILLED,
+          new TerminatedWhileRunningTransition(KILLED_HELPER))
+      .addTransition(TaskAttemptStateInternal.RUNNING,
+          TaskAttemptStateInternal.KILL_IN_PROGRESS,
           TaskAttemptEventType.TA_NODE_FAILED,
           new TerminatedWhileRunningTransition(KILLED_HELPER))
       .addTransition(TaskAttemptStateInternal.RUNNING,
@@ -303,6 +317,7 @@ public class TaskAttemptImpl implements TaskAttempt,
               TaskAttemptEventType.TA_DONE, TaskAttemptEventType.TA_FAILED,
               TaskAttemptEventType.TA_TIMED_OUT,
               TaskAttemptEventType.TA_KILL_REQUEST,
+              TaskAttemptEventType.TA_KILLED,
               TaskAttemptEventType.TA_NODE_FAILED,
               TaskAttemptEventType.TA_CONTAINER_TERMINATING,
               TaskAttemptEventType.TA_OUTPUT_FAILED))
@@ -324,6 +339,7 @@ public class TaskAttemptImpl implements TaskAttempt,
               TaskAttemptEventType.TA_DONE, TaskAttemptEventType.TA_FAILED,
               TaskAttemptEventType.TA_TIMED_OUT,
               TaskAttemptEventType.TA_KILL_REQUEST,
+              TaskAttemptEventType.TA_KILLED,
               TaskAttemptEventType.TA_NODE_FAILED,
               TaskAttemptEventType.TA_CONTAINER_TERMINATING,
               TaskAttemptEventType.TA_OUTPUT_FAILED))
@@ -342,6 +358,7 @@ public class TaskAttemptImpl implements TaskAttempt,
               TaskAttemptEventType.TA_DONE, TaskAttemptEventType.TA_FAILED,
               TaskAttemptEventType.TA_TIMED_OUT,
               TaskAttemptEventType.TA_KILL_REQUEST,
+              TaskAttemptEventType.TA_KILLED,
               TaskAttemptEventType.TA_NODE_FAILED,
               TaskAttemptEventType.TA_CONTAINER_TERMINATING,
               TaskAttemptEventType.TA_CONTAINER_TERMINATED,
@@ -361,6 +378,7 @@ public class TaskAttemptImpl implements TaskAttempt,
               TaskAttemptEventType.TA_DONE, TaskAttemptEventType.TA_FAILED,
               TaskAttemptEventType.TA_TIMED_OUT,
               TaskAttemptEventType.TA_KILL_REQUEST,
+              TaskAttemptEventType.TA_KILLED,
               TaskAttemptEventType.TA_NODE_FAILED,
               TaskAttemptEventType.TA_CONTAINER_TERMINATING,
               TaskAttemptEventType.TA_CONTAINER_TERMINATED,
@@ -383,6 +401,12 @@ public class TaskAttemptImpl implements TaskAttempt,
           TaskAttemptStateInternal.SUCCEEDED,
           EnumSet.of(TaskAttemptStateInternal.KILLED,
               TaskAttemptStateInternal.SUCCEEDED),
+          TaskAttemptEventType.TA_KILLED,
+          new TerminatedAfterSuccessTransition())
+      .addTransition(
+          TaskAttemptStateInternal.SUCCEEDED,
+          EnumSet.of(TaskAttemptStateInternal.KILLED,
+              TaskAttemptStateInternal.SUCCEEDED),
           TaskAttemptEventType.TA_NODE_FAILED,
           new TerminatedAfterSuccessTransition())
       .addTransition(
@@ -434,7 +458,6 @@ public class TaskAttemptImpl implements TaskAttempt,
     this.leafVertex = leafVertex;
   }
 
-
   @Override
   public TezTaskAttemptID getID() {
     return attemptId;
@@ -1030,6 +1053,7 @@ public class TaskAttemptImpl implements TaskAttempt,
 
       // Compute node/rack location request even if re-scheduled.
       Set<String> racks = new HashSet<String>();
+      // TODO Post TEZ-2003. Allow for a policy in the VMPlugin to define localicty for different attempts.
       TaskLocationHint locationHint = ta.getTaskLocationHint();
       if (locationHint != null) {
         if (locationHint.getRacks() != null) {
@@ -1104,6 +1128,8 @@ public class TaskAttemptImpl implements TaskAttempt,
 
     @Override
     public void transition(TaskAttemptImpl ta, TaskAttemptEvent event) {
+      // This transition should not be invoked directly, if a scheduler event has already been sent out.
+      // Sub-classes should be used if a scheduler request has been sent.
       ta.setFinishTime();
 
       if (event instanceof DiagnosableEvent) {
@@ -1218,7 +1244,8 @@ public class TaskAttemptImpl implements TaskAttempt,
       // Inform the scheduler
       if (sendSchedulerEvent()) {
         ta.sendEvent(new AMSchedulerEventTAEnded(ta, ta.containerId, helper
-            .getTaskAttemptState(), ta.getVertex().getTaskSchedulerIdentifier()));
+            .getTaskAttemptState(), TezUtilsInternal.toTaskAttemptEndReason(ta.terminationCause),
+            ta.getVertex().getTaskSchedulerIdentifier()));
       }
     }
   }
@@ -1300,7 +1327,7 @@ public class TaskAttemptImpl implements TaskAttempt,
 
       // Inform the Scheduler.
       ta.sendEvent(new AMSchedulerEventTAEnded(ta, ta.containerId,
-          TaskAttemptState.SUCCEEDED, ta.getVertex().getTaskSchedulerIdentifier()));
+          TaskAttemptState.SUCCEEDED, null, ta.getVertex().getTaskSchedulerIdentifier()));
 
       // Inform the task.
       ta.sendEvent(new TaskEventTAUpdate(ta.attemptId,

http://git-wip-us.apache.org/repos/asf/tez/blob/c115a4ac/tez-dag/src/main/java/org/apache/tez/dag/app/rm/AMSchedulerEventTAEnded.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/AMSchedulerEventTAEnded.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/AMSchedulerEventTAEnded.java
index 2ace642..a775948 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/AMSchedulerEventTAEnded.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/AMSchedulerEventTAEnded.java
@@ -18,6 +18,7 @@
 package org.apache.tez.dag.app.rm;
 
 import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.tez.dag.api.TaskAttemptEndReason;
 import org.apache.tez.dag.api.oldrecords.TaskAttemptState;
 import org.apache.tez.dag.app.dag.TaskAttempt;
 import org.apache.tez.dag.records.TezTaskAttemptID;
@@ -27,14 +28,16 @@ public class AMSchedulerEventTAEnded extends AMSchedulerEvent {
   private final TaskAttempt attempt;
   private final ContainerId containerId;
   private final TaskAttemptState state;
+  private final TaskAttemptEndReason taskAttemptEndReason;
   private final int schedulerId;
 
   public AMSchedulerEventTAEnded(TaskAttempt attempt, ContainerId containerId,
-      TaskAttemptState state, int schedulerId) {
+      TaskAttemptState state, TaskAttemptEndReason taskAttemptEndReason, int schedulerId) {
     super(AMSchedulerEventType.S_TA_ENDED);
     this.attempt = attempt;
     this.containerId = containerId;
     this.state = state;
+    this.taskAttemptEndReason = taskAttemptEndReason;
     this.schedulerId = schedulerId;
   }
 
@@ -57,4 +60,8 @@ public class AMSchedulerEventTAEnded extends AMSchedulerEvent {
   public int getSchedulerId() {
     return schedulerId;
   }
+
+  public TaskAttemptEndReason getTaskAttemptEndReason() {
+    return taskAttemptEndReason;
+  }
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/c115a4ac/tez-dag/src/main/java/org/apache/tez/dag/app/rm/LocalTaskSchedulerService.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/LocalTaskSchedulerService.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/LocalTaskSchedulerService.java
index 72a074f..a234e07 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/LocalTaskSchedulerService.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/LocalTaskSchedulerService.java
@@ -40,6 +40,7 @@ import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.tez.dag.api.TaskAttemptEndReason;
 import org.apache.tez.dag.api.TezConfiguration;
 import org.apache.tez.dag.api.TezUncheckedException;
 import org.apache.tez.dag.app.AppContext;
@@ -149,7 +150,7 @@ public class LocalTaskSchedulerService extends TaskSchedulerService {
   }
   
   @Override
-  public boolean deallocateTask(Object task, boolean taskSucceeded) {
+  public boolean deallocateTask(Object task, boolean taskSucceeded, TaskAttemptEndReason endReason) {
     return taskRequestHandler.addDeallocateTaskRequest(task);
   }
 

http://git-wip-us.apache.org/repos/asf/tez/blob/c115a4ac/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerEventHandler.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerEventHandler.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerEventHandler.java
index 8e5fc71..9f09f68 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerEventHandler.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerEventHandler.java
@@ -289,7 +289,9 @@ public class TaskSchedulerEventHandler extends AbstractService
 
   private void handleTAUnsuccessfulEnd(AMSchedulerEventTAEnded event) {
     TaskAttempt attempt = event.getAttempt();
-    boolean wasContainerAllocated = taskSchedulers[event.getSchedulerId()].deallocateTask(attempt, false);
+    // Propagate state and failure cause (if any) when informing the scheduler about the de-allocation.
+    boolean wasContainerAllocated = taskSchedulers[event.getSchedulerId()]
+        .deallocateTask(attempt, false, event.getTaskAttemptEndReason());
     // use stored value of container id in case the scheduler has removed this
     // assignment because the task has been deallocated earlier.
     // retroactive case
@@ -311,6 +313,7 @@ public class TaskSchedulerEventHandler extends AbstractService
       sendEvent(new AMContainerEventStopRequest(attemptContainerId));
       // Inform the Node - the task has asked to be STOPPED / has already
       // stopped.
+      // AMNodeImpl blacklisting logic does not account for KILLED attempts.
       sendEvent(new AMNodeEventTaskAttemptEnded(appContext.getAllContainers().
           get(attemptContainerId).getContainer().getNodeId(), attemptContainerId,
           attempt.getID(), event.getState() == TaskAttemptState.FAILED));
@@ -332,7 +335,7 @@ public class TaskSchedulerEventHandler extends AbstractService
     }
 
     boolean wasContainerAllocated = taskSchedulers[event.getSchedulerId()].deallocateTask(attempt,
-        true);
+        true, null);
     if (!wasContainerAllocated) {
       LOG.error("De-allocated successful task: " + attempt.getID()
           + ", but TaskScheduler reported no container assigned to task");

http://git-wip-us.apache.org/repos/asf/tez/blob/c115a4ac/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerService.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerService.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerService.java
index 48d5455..07dfcd6 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerService.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerService.java
@@ -32,6 +32,7 @@ import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.NodeReport;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.tez.dag.api.TaskAttemptEndReason;
 
 public abstract class TaskSchedulerService extends AbstractService{
 
@@ -61,8 +62,9 @@ public abstract class TaskSchedulerService extends AbstractService{
   public abstract void allocateTask(Object task, Resource capability,
       ContainerId containerId, Priority priority, Object containerSignature,
       Object clientCookie);
-  
-  public abstract boolean deallocateTask(Object task, boolean taskSucceeded);
+
+  /** Plugin writers must ensure to de-allocate a container once it's done, so that it can be collected. */
+  public abstract boolean deallocateTask(Object task, boolean taskSucceeded, TaskAttemptEndReason endReason);
 
   public abstract Object deallocateContainer(ContainerId containerId);
 

http://git-wip-us.apache.org/repos/asf/tez/blob/c115a4ac/tez-dag/src/main/java/org/apache/tez/dag/app/rm/YarnTaskSchedulerService.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/YarnTaskSchedulerService.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/YarnTaskSchedulerService.java
index 44f5484..1fc9ac2 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/YarnTaskSchedulerService.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/YarnTaskSchedulerService.java
@@ -56,6 +56,7 @@ import org.apache.hadoop.yarn.client.api.async.AMRMClientAsync;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.util.RackResolver;
 import org.apache.hadoop.yarn.util.resource.Resources;
+import org.apache.tez.dag.api.TaskAttemptEndReason;
 import org.apache.tez.dag.api.TezConfiguration;
 import org.apache.tez.dag.api.TezUncheckedException;
 import org.apache.tez.dag.app.AppContext;
@@ -987,10 +988,13 @@ public class YarnTaskSchedulerService extends TaskSchedulerService
    *          the task to de-allocate.
    * @param taskSucceeded
    *          specify whether the task succeeded or failed.
+   * @param endReason
+   *          reason for the task ending
    * @return true if a container is assigned to this task.
    */
   @Override
-  public boolean deallocateTask(Object task, boolean taskSucceeded) {
+  public boolean deallocateTask(Object task, boolean taskSucceeded,
+                                TaskAttemptEndReason endReason) {
     Map<CookieContainerRequest, Container> assignedContainers = null;
 
     synchronized (this) {
@@ -1180,7 +1184,7 @@ public class YarnTaskSchedulerService extends TaskSchedulerService
             CookieContainerRequest request = entry.getValue();
             if (request.getPriority().equals(lowestPriNewContainer.getPriority())) {
               LOG.info("Resending request for task again: " + task);
-              deallocateTask(task, true);
+              deallocateTask(task, true, null);
               allocateTask(task, request.getCapability(), 
                   (request.getNodes() == null ? null : 
                     request.getNodes().toArray(new String[request.getNodes().size()])), 

http://git-wip-us.apache.org/repos/asf/tez/blob/c115a4ac/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag.java b/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag.java
index 7a365bd..fa62c11 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag.java
@@ -74,6 +74,7 @@ import org.junit.Test;
 import org.mockito.ArgumentCaptor;
 
 @SuppressWarnings("unchecked")
+// TODO TEZ-2003 Rename to TestTezTaskCommunicator
 public class TestTaskAttemptListenerImplTezDag {
   private ApplicationId appId;
   private AppContext appContext;

http://git-wip-us.apache.org/repos/asf/tez/blob/c115a4ac/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag2.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag2.java b/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag2.java
new file mode 100644
index 0000000..934543f
--- /dev/null
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag2.java
@@ -0,0 +1,126 @@
+/*
+ * 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.
+ */
+
+package org.apache.tez.dag.app;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Matchers.any;
+import static org.mockito.Mockito.doReturn;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.hadoop.security.Credentials;
+import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.hadoop.yarn.event.Event;
+import org.apache.hadoop.yarn.event.EventHandler;
+import org.apache.tez.dag.api.TaskAttemptEndReason;
+import org.apache.tez.dag.app.dag.DAG;
+import org.apache.tez.dag.app.dag.event.TaskAttemptEventAttemptFailed;
+import org.apache.tez.dag.app.dag.event.TaskAttemptEventAttemptKilled;
+import org.apache.tez.dag.app.rm.container.AMContainer;
+import org.apache.tez.dag.app.rm.container.AMContainerMap;
+import org.apache.tez.dag.app.rm.container.AMContainerTask;
+import org.apache.tez.dag.records.TaskAttemptTerminationCause;
+import org.apache.tez.dag.records.TezTaskAttemptID;
+import org.apache.tez.runtime.api.impl.TaskSpec;
+import org.junit.Test;
+import org.mockito.ArgumentCaptor;
+
+// TODO TEZ-2003. Rename to TestTaskAttemptListener | whatever TaskAttemptListener is renamed to.
+public class TestTaskAttemptListenerImplTezDag2 {
+
+  @Test(timeout = 5000)
+  public void testTaskAttemptFailedKilled() {
+    ApplicationId appId = ApplicationId.newInstance(1000, 1);
+    ApplicationAttemptId appAttemptId = ApplicationAttemptId.newInstance(appId, 1);
+    Credentials credentials = new Credentials();
+    AppContext appContext = mock(AppContext.class);
+    EventHandler eventHandler = mock(EventHandler.class);
+    DAG dag = mock(DAG.class);
+    AMContainerMap amContainerMap = mock(AMContainerMap.class);
+    Map<ApplicationAccessType, String> appAcls = new HashMap<ApplicationAccessType, String>();
+    doReturn(eventHandler).when(appContext).getEventHandler();
+    doReturn(dag).when(appContext).getCurrentDAG();
+    doReturn(appAttemptId).when(appContext).getApplicationAttemptId();
+    doReturn(credentials).when(appContext).getAppCredentials();
+    doReturn(appAcls).when(appContext).getApplicationACLs();
+    doReturn(amContainerMap).when(appContext).getAllContainers();
+    NodeId nodeId = NodeId.newInstance("localhost", 0);
+    AMContainer amContainer = mock(AMContainer.class);
+    Container container = mock(Container.class);
+    doReturn(nodeId).when(container).getNodeId();
+    doReturn(amContainer).when(amContainerMap).get(any(ContainerId.class));
+    doReturn(container).when(amContainer).getContainer();
+
+    TaskAttemptListenerImpTezDag taskAttemptListener =
+        new TaskAttemptListenerImpTezDag(appContext, mock(TaskHeartbeatHandler.class),
+            mock(ContainerHeartbeatHandler.class), null, null, false);
+
+    TaskSpec taskSpec1 = mock(TaskSpec.class);
+    TezTaskAttemptID taskAttemptId1 = mock(TezTaskAttemptID.class);
+    doReturn(taskAttemptId1).when(taskSpec1).getTaskAttemptID();
+    AMContainerTask amContainerTask1 = new AMContainerTask(taskSpec1, null, null, false, 10);
+
+    TaskSpec taskSpec2 = mock(TaskSpec.class);
+    TezTaskAttemptID taskAttemptId2 = mock(TezTaskAttemptID.class);
+    doReturn(taskAttemptId2).when(taskSpec2).getTaskAttemptID();
+    AMContainerTask amContainerTask2 = new AMContainerTask(taskSpec2, null, null, false, 10);
+
+    ContainerId containerId1 = createContainerId(appId, 1);
+    taskAttemptListener.registerRunningContainer(containerId1, 0);
+    taskAttemptListener.registerTaskAttempt(amContainerTask1, containerId1, 0);
+    ContainerId containerId2 = createContainerId(appId, 2);
+    taskAttemptListener.registerRunningContainer(containerId2, 0);
+    taskAttemptListener.registerTaskAttempt(amContainerTask2, containerId2, 0);
+
+
+    taskAttemptListener
+        .taskFailed(taskAttemptId1, TaskAttemptEndReason.COMMUNICATION_ERROR, "Diagnostics1");
+    taskAttemptListener
+        .taskKilled(taskAttemptId2, TaskAttemptEndReason.SERVICE_BUSY, "Diagnostics2");
+
+    ArgumentCaptor<Event> argumentCaptor = ArgumentCaptor.forClass(Event.class);
+    verify(eventHandler, times(2)).handle(argumentCaptor.capture());
+    assertTrue(argumentCaptor.getAllValues().get(0) instanceof TaskAttemptEventAttemptFailed);
+    assertTrue(argumentCaptor.getAllValues().get(1) instanceof TaskAttemptEventAttemptKilled);
+    TaskAttemptEventAttemptFailed failedEvent =
+        (TaskAttemptEventAttemptFailed) argumentCaptor.getAllValues().get(0);
+    TaskAttemptEventAttemptKilled killedEvent =
+        (TaskAttemptEventAttemptKilled) argumentCaptor.getAllValues().get(1);
+
+    assertEquals("Diagnostics1", failedEvent.getDiagnosticInfo());
+    assertEquals(TaskAttemptTerminationCause.COMMUNICATION_ERROR,
+        failedEvent.getTerminationCause());
+
+    assertEquals("Diagnostics2", killedEvent.getDiagnosticInfo());
+    assertEquals(TaskAttemptTerminationCause.SERVICE_BUSY, killedEvent.getTerminationCause());
+    // TODO TEZ-2003. Verify unregistration from the registered list
+  }
+
+  private ContainerId createContainerId(ApplicationId applicationId, int containerIdx) {
+    ApplicationAttemptId appAttemptId = ApplicationAttemptId.newInstance(applicationId, 1);
+    ContainerId containerId = ContainerId.newInstance(appAttemptId, containerIdx);
+    return containerId;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tez/blob/c115a4ac/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 c1169ef..d45346a 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
@@ -22,6 +22,7 @@ import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.assertEquals;
 import static org.mockito.Matchers.any;
 import static org.mockito.Matchers.eq;
+import static org.mockito.Matchers.isNull;
 import static org.mockito.Mockito.doReturn;
 import static org.mockito.Mockito.doAnswer;
 import static org.mockito.Mockito.mock;
@@ -57,6 +58,7 @@ import org.apache.tez.common.MockDNSToSwitchMapping;
 import org.apache.tez.dag.api.InputDescriptor;
 import org.apache.tez.dag.api.OutputDescriptor;
 import org.apache.tez.dag.api.ProcessorDescriptor;
+import org.apache.tez.dag.api.TaskAttemptEndReason;
 import org.apache.tez.dag.api.TezConfiguration;
 import org.apache.tez.dag.api.TaskLocationHint;
 import org.apache.tez.dag.api.oldrecords.TaskAttemptState;
@@ -92,6 +94,7 @@ import org.apache.tez.runtime.api.impl.OutputSpec;
 import org.apache.tez.runtime.api.impl.TaskSpec;
 import org.junit.BeforeClass;
 import org.junit.Test;
+import org.mockito.internal.matchers.Null;
 import org.mockito.invocation.InvocationOnMock;
 import org.mockito.stubbing.Answer;
 
@@ -223,9 +226,9 @@ public class TestContainerReuse {
 
     taskSchedulerEventHandler.handleEvent(
       new AMSchedulerEventTAEnded(
-        ta11, containerHost1.getId(), TaskAttemptState.SUCCEEDED, 0));
+        ta11, containerHost1.getId(), TaskAttemptState.SUCCEEDED, null, 0));
     drainableAppCallback.drain();
-    verify(taskScheduler).deallocateTask(eq(ta11), eq(true));
+    verify(taskScheduler).deallocateTask(eq(ta11), eq(true), eq((TaskAttemptEndReason)null));
     verify(taskSchedulerEventHandler, times(1)).taskAllocated(
       eq(ta31), any(Object.class), eq(containerHost1));
     verify(rmClient, times(0)).releaseAssignedContainer(
@@ -235,7 +238,7 @@ public class TestContainerReuse {
 
     taskSchedulerEventHandler.handleEvent(
       new AMSchedulerEventTAEnded(ta21, containerHost2.getId(),
-        TaskAttemptState.SUCCEEDED, 0));
+        TaskAttemptState.SUCCEEDED, null, 0));
 
     long currentTs = System.currentTimeMillis();
     Throwable exception = null;
@@ -356,9 +359,9 @@ public class TestContainerReuse {
 
     taskSchedulerEventHandler.handleEvent(
       new AMSchedulerEventTAEnded(ta21, containerHost2.getId(),
-        TaskAttemptState.SUCCEEDED, 0));
+        TaskAttemptState.SUCCEEDED, null, 0));
     drainableAppCallback.drain();
-    verify(taskScheduler).deallocateTask(eq(ta21), eq(true));
+    verify(taskScheduler).deallocateTask(eq(ta21), eq(true), eq((TaskAttemptEndReason)null));
     verify(taskSchedulerEventHandler, times(0)).taskAllocated(
       eq(ta31), any(Object.class), eq(containerHost2));
     verify(rmClient, times(1)).releaseAssignedContainer(
@@ -459,9 +462,9 @@ public class TestContainerReuse {
     verify(taskSchedulerEventHandler).taskAllocated(eq(ta11), any(Object.class), eq(container1));
 
     // Task assigned to container completed successfully. Container should be re-used.
-    taskSchedulerEventHandler.handleEvent(new AMSchedulerEventTAEnded(ta11, container1.getId(), TaskAttemptState.SUCCEEDED, 0));
+    taskSchedulerEventHandler.handleEvent(new AMSchedulerEventTAEnded(ta11, container1.getId(), TaskAttemptState.SUCCEEDED, null, 0));
     drainableAppCallback.drain();
-    verify(taskScheduler).deallocateTask(eq(ta11), eq(true));
+    verify(taskScheduler).deallocateTask(eq(ta11), eq(true), eq((TaskAttemptEndReason)null));
     verify(taskSchedulerEventHandler).taskAllocated(eq(ta12), any(Object.class), eq(container1));
     verify(rmClient, times(0)).releaseAssignedContainer(eq(container1.getId()));
     eventHandler.verifyNoInvocations(AMContainerEventStopRequest.class);
@@ -469,19 +472,19 @@ public class TestContainerReuse {
 
     // Task assigned to container completed successfully.
     // Verify reuse across hosts.
-    taskSchedulerEventHandler.handleEvent(new AMSchedulerEventTAEnded(ta12, container1.getId(), TaskAttemptState.SUCCEEDED, 0));
+    taskSchedulerEventHandler.handleEvent(new AMSchedulerEventTAEnded(ta12, container1.getId(), TaskAttemptState.SUCCEEDED, null, 0));
     drainableAppCallback.drain();
-    verify(taskScheduler).deallocateTask(eq(ta12), eq(true));
+    verify(taskScheduler).deallocateTask(eq(ta12), eq(true), eq((TaskAttemptEndReason)null));
     verify(taskSchedulerEventHandler).taskAllocated(eq(ta13), any(Object.class), eq(container1));
     verify(rmClient, times(0)).releaseAssignedContainer(eq(container1.getId()));
     eventHandler.verifyNoInvocations(AMContainerEventStopRequest.class);
     eventHandler.reset();
 
     // Verify no re-use if a previous task fails.
-    taskSchedulerEventHandler.handleEvent(new AMSchedulerEventTAEnded(ta13, container1.getId(), TaskAttemptState.FAILED, 0));
+    taskSchedulerEventHandler.handleEvent(new AMSchedulerEventTAEnded(ta13, container1.getId(), TaskAttemptState.FAILED, null, 0));
     drainableAppCallback.drain();
     verify(taskSchedulerEventHandler, times(0)).taskAllocated(eq(ta14), any(Object.class), eq(container1));
-    verify(taskScheduler).deallocateTask(eq(ta13), eq(false));
+    verify(taskScheduler).deallocateTask(eq(ta13), eq(false), eq((TaskAttemptEndReason)null));
     verify(rmClient).releaseAssignedContainer(eq(container1.getId()));
     eventHandler.verifyInvocation(AMContainerEventStopRequest.class);
     eventHandler.reset();
@@ -496,9 +499,9 @@ public class TestContainerReuse {
     verify(taskSchedulerEventHandler).taskAllocated(eq(ta14), any(Object.class), eq(container2));
 
     // Task assigned to container completed successfully. No pending requests. Container should be released.
-    taskSchedulerEventHandler.handleEvent(new AMSchedulerEventTAEnded(ta14, container2.getId(), TaskAttemptState.SUCCEEDED, 0));
+    taskSchedulerEventHandler.handleEvent(new AMSchedulerEventTAEnded(ta14, container2.getId(), TaskAttemptState.SUCCEEDED, null, 0));
     drainableAppCallback.drain();
-    verify(taskScheduler).deallocateTask(eq(ta14), eq(true));
+    verify(taskScheduler).deallocateTask(eq(ta14), eq(true), eq((TaskAttemptEndReason)null));
     verify(rmClient).releaseAssignedContainer(eq(container2.getId()));
     eventHandler.verifyInvocation(AMContainerEventStopRequest.class);
     eventHandler.reset();
@@ -607,9 +610,9 @@ public class TestContainerReuse {
 
     // First task had profiling on. This container can not be reused further.
     taskSchedulerEventHandler.handleEvent(
-        new AMSchedulerEventTAEnded(ta11, container1.getId(), TaskAttemptState.SUCCEEDED, 0));
+        new AMSchedulerEventTAEnded(ta11, container1.getId(), TaskAttemptState.SUCCEEDED, null, 0));
     drainableAppCallback.drain();
-    verify(taskScheduler).deallocateTask(eq(ta11), eq(true));
+    verify(taskScheduler).deallocateTask(eq(ta11), eq(true), eq((TaskAttemptEndReason)null));
     verify(taskSchedulerEventHandler, times(0)).taskAllocated(eq(ta12), any(Object.class),
         eq(container1));
     verify(rmClient, times(1)).releaseAssignedContainer(eq(container1.getId()));
@@ -653,9 +656,9 @@ public class TestContainerReuse {
 
     // Verify that the container can not be reused when profiling option is turned on
     // Even for 2 tasks having same profiling option can have container reusability.
-    taskSchedulerEventHandler.handleEvent(new AMSchedulerEventTAEnded(ta13, container2.getId(), TaskAttemptState.SUCCEEDED, 0));
+    taskSchedulerEventHandler.handleEvent(new AMSchedulerEventTAEnded(ta13, container2.getId(), TaskAttemptState.SUCCEEDED, null, 0));
     drainableAppCallback.drain();
-    verify(taskScheduler).deallocateTask(eq(ta13), eq(true));
+    verify(taskScheduler).deallocateTask(eq(ta13), eq(true), eq((TaskAttemptEndReason)null));
     verify(taskSchedulerEventHandler, times(0)).taskAllocated(eq(ta14), any(Object.class),
         eq(container2));
     verify(rmClient, times(1)).releaseAssignedContainer(eq(container2.getId()));
@@ -698,9 +701,9 @@ public class TestContainerReuse {
     verify(taskSchedulerEventHandler).taskAllocated(eq(ta15), any(Object.class), eq(container3));
 
     //Ensure task 6 (of vertex 1) is allocated to same container
-    taskSchedulerEventHandler.handleEvent(new AMSchedulerEventTAEnded(ta15, container3.getId(), TaskAttemptState.SUCCEEDED, 0));
+    taskSchedulerEventHandler.handleEvent(new AMSchedulerEventTAEnded(ta15, container3.getId(), TaskAttemptState.SUCCEEDED, null, 0));
     drainableAppCallback.drain();
-    verify(taskScheduler).deallocateTask(eq(ta15), eq(true));
+    verify(taskScheduler).deallocateTask(eq(ta15), eq(true), eq((TaskAttemptEndReason)null));
     verify(taskSchedulerEventHandler).taskAllocated(eq(ta16), any(Object.class), eq(container3));
     eventHandler.reset();
 
@@ -811,9 +814,9 @@ public class TestContainerReuse {
     // until delay expires.
     taskSchedulerEventHandler.handleEvent(
       new AMSchedulerEventTAEnded(ta11, container1.getId(),
-        TaskAttemptState.SUCCEEDED, 0));
+        TaskAttemptState.SUCCEEDED, null, 0));
     drainableAppCallback.drain();
-    verify(taskScheduler).deallocateTask(eq(ta11), eq(true));
+    verify(taskScheduler).deallocateTask(eq(ta11), eq(true), eq((TaskAttemptEndReason)null));
     verify(taskSchedulerEventHandler, times(0)).taskAllocated(
       eq(ta12), any(Object.class), eq(container1));
     verify(rmClient, times(0)).releaseAssignedContainer(eq(container1.getId()));
@@ -828,7 +831,7 @@ public class TestContainerReuse {
     // TA12 completed.
     taskSchedulerEventHandler.handleEvent(
       new AMSchedulerEventTAEnded(ta12, container1.getId(),
-        TaskAttemptState.SUCCEEDED, 0));
+        TaskAttemptState.SUCCEEDED, null, 0));
     drainableAppCallback.drain();
     LOG.info("Sleeping to ensure that the scheduling loop runs");
     Thread.sleep(3000l);
@@ -946,9 +949,9 @@ public class TestContainerReuse {
     // Container should  be assigned to task21.
     taskSchedulerEventHandler.handleEvent(
       new AMSchedulerEventTAEnded(ta11, container1.getId(),
-        TaskAttemptState.SUCCEEDED, 0));
+        TaskAttemptState.SUCCEEDED, null, 0));
     drainableAppCallback.drain();
-    verify(taskScheduler).deallocateTask(eq(ta11), eq(true));
+    verify(taskScheduler).deallocateTask(eq(ta11), eq(true), eq((TaskAttemptEndReason)null));
     verify(taskSchedulerEventHandler).taskAllocated(
       eq(ta21), any(Object.class), eq(container1));
     verify(rmClient, times(0)).releaseAssignedContainer(eq(container1.getId()));
@@ -956,7 +959,7 @@ public class TestContainerReuse {
     // Task 2 completes.
     taskSchedulerEventHandler.handleEvent(
       new AMSchedulerEventTAEnded(ta21, container1.getId(),
-        TaskAttemptState.SUCCEEDED, 0));
+        TaskAttemptState.SUCCEEDED, null, 0));
     verify(rmClient, times(0)).releaseAssignedContainer(eq(container1.getId()));
 
     LOG.info("Sleeping to ensure that the scheduling loop runs");
@@ -1065,9 +1068,9 @@ public class TestContainerReuse {
     assertEquals(1, assignEvent.getRemoteTaskLocalResources().size());
     
     // Task assigned to container completed successfully. Container should be re-used.
-    taskSchedulerEventHandler.handleEvent(new AMSchedulerEventTAEnded(ta111, container1.getId(), TaskAttemptState.SUCCEEDED, 0));
+    taskSchedulerEventHandler.handleEvent(new AMSchedulerEventTAEnded(ta111, container1.getId(), TaskAttemptState.SUCCEEDED, null, 0));
     drainableAppCallback.drain();
-    verify(taskScheduler).deallocateTask(eq(ta111), eq(true));
+    verify(taskScheduler).deallocateTask(eq(ta111), eq(true), eq((TaskAttemptEndReason)null));
     verify(taskSchedulerEventHandler).taskAllocated(eq(ta112), any(Object.class), eq(container1));
     verify(rmClient, times(0)).releaseAssignedContainer(eq(container1.getId()));
     eventHandler.verifyNoInvocations(AMContainerEventStopRequest.class);
@@ -1077,9 +1080,9 @@ public class TestContainerReuse {
 
     // Task assigned to container completed successfully.
     // Verify reuse across hosts.
-    taskSchedulerEventHandler.handleEvent(new AMSchedulerEventTAEnded(ta112, container1.getId(), TaskAttemptState.SUCCEEDED, 0));
+    taskSchedulerEventHandler.handleEvent(new AMSchedulerEventTAEnded(ta112, container1.getId(), TaskAttemptState.SUCCEEDED, null, 0));
     drainableAppCallback.drain();
-    verify(taskScheduler).deallocateTask(eq(ta112), eq(true));
+    verify(taskScheduler).deallocateTask(eq(ta112), eq(true), eq((TaskAttemptEndReason)null));
     verify(rmClient, times(0)).releaseAssignedContainer(eq(container1.getId()));
     eventHandler.verifyNoInvocations(AMContainerEventStopRequest.class);
     eventHandler.reset();
@@ -1118,9 +1121,9 @@ public class TestContainerReuse {
     assertEquals(2, assignEvent.getRemoteTaskLocalResources().size());
     eventHandler.reset();
 
-    taskSchedulerEventHandler.handleEvent(new AMSchedulerEventTAEnded(ta211, container1.getId(), TaskAttemptState.SUCCEEDED, 0));
+    taskSchedulerEventHandler.handleEvent(new AMSchedulerEventTAEnded(ta211, container1.getId(), TaskAttemptState.SUCCEEDED, null, 0));
     drainableAppCallback.drain();
-    verify(taskScheduler).deallocateTask(eq(ta211), eq(true));
+    verify(taskScheduler).deallocateTask(eq(ta211), eq(true), eq((TaskAttemptEndReason)null));
     verify(taskSchedulerEventHandler).taskAllocated(eq(ta212), any(Object.class), eq(container1));
     verify(rmClient, times(0)).releaseAssignedContainer(eq(container1.getId()));
     eventHandler.verifyNoInvocations(AMContainerEventStopRequest.class);

http://git-wip-us.apache.org/repos/asf/tez/blob/c115a4ac/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestLocalTaskSchedulerService.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestLocalTaskSchedulerService.java b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestLocalTaskSchedulerService.java
index 25cf4b5..0a642bb 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestLocalTaskSchedulerService.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestLocalTaskSchedulerService.java
@@ -27,6 +27,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.tez.dag.api.TaskAttemptEndReason;
 import org.apache.tez.dag.app.AppContext;
 import org.apache.tez.dag.app.dag.Task;
 import org.apache.tez.dag.app.rm.TaskSchedulerService.TaskSchedulerAppCallback;
@@ -94,7 +95,7 @@ public class TestLocalTaskSchedulerService {
 
     Task task = mock(Task.class);
     taskSchedulerService.allocateTask(task, Resource.newInstance(1024, 1), null, null, Priority.newInstance(1), null, null);
-    taskSchedulerService.deallocateTask(task, false);
+    taskSchedulerService.deallocateTask(task, false, null);
     // start the RequestHandler, DeallocateTaskRequest has higher priority, so will be processed first
     taskSchedulerService.startRequestHandlerThread();
 
@@ -126,7 +127,7 @@ public class TestLocalTaskSchedulerService {
 
     MockAsyncDelegateRequestHandler requestHandler = taskSchedulerService.getRequestHandler();
     requestHandler.drainRequest(1);
-    taskSchedulerService.deallocateTask(task, false);
+    taskSchedulerService.deallocateTask(task, false, null);
     requestHandler.drainRequest(2);
     assertEquals(1, requestHandler.deallocateCount);
     assertEquals(1, requestHandler.allocateCount);

http://git-wip-us.apache.org/repos/asf/tez/blob/c115a4ac/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskScheduler.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskScheduler.java b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskScheduler.java
index dabae67..807e772 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskScheduler.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskScheduler.java
@@ -172,7 +172,7 @@ public class TestTaskScheduler {
                            addContainerRequest((CookieContainerRequest) any());
 
     // returned from task requests before allocation happens
-    assertFalse(scheduler.deallocateTask(mockTask1, true));
+    assertFalse(scheduler.deallocateTask(mockTask1, true, null));
     verify(mockApp, times(0)).containerBeingReleased(any(ContainerId.class));
     verify(mockRMClient, times(1)).
                         removeContainerRequest((CookieContainerRequest) any());
@@ -180,7 +180,7 @@ public class TestTaskScheduler {
                                  releaseAssignedContainer((ContainerId) any());
 
     // deallocating unknown task
-    assertFalse(scheduler.deallocateTask(mockTask1, true));
+    assertFalse(scheduler.deallocateTask(mockTask1, true, null));
     verify(mockApp, times(0)).containerBeingReleased(any(ContainerId.class));
     verify(mockRMClient, times(1)).
                         removeContainerRequest((CookieContainerRequest) any());
@@ -325,7 +325,7 @@ public class TestTaskScheduler {
     verify(mockRMClient).releaseAssignedContainer(mockCId4);
 
     // deallocate allocated task
-    assertTrue(scheduler.deallocateTask(mockTask1, true));
+    assertTrue(scheduler.deallocateTask(mockTask1, true, null));
     drainableAppCallback.drain();
     verify(mockApp).containerBeingReleased(mockCId1);
     verify(mockRMClient).releaseAssignedContainer(mockCId1);
@@ -445,7 +445,7 @@ public class TestTaskScheduler {
     verify(mockApp, times(4)).taskAllocated(any(), any(), (Container) any());
     verify(mockApp).taskAllocated(mockTask4, mockCookie4, mockContainer6);
     // deallocate allocated task
-    assertTrue(scheduler.deallocateTask(mockTask4, true));
+    assertTrue(scheduler.deallocateTask(mockTask4, true, null));
     drainableAppCallback.drain();
     verify(mockApp).containerBeingReleased(mockCId6);
     verify(mockRMClient).releaseAssignedContainer(mockCId6);
@@ -475,7 +475,7 @@ public class TestTaskScheduler {
         removeContainerRequest((CookieContainerRequest) any());
     verify(mockRMClient, times(8)).addContainerRequest(
         (CookieContainerRequest) any());
-    assertFalse(scheduler.deallocateTask(mockTask1, true));
+    assertFalse(scheduler.deallocateTask(mockTask1, true, null));
 
     List<NodeReport> mockUpdatedNodes = mock(List.class);
     scheduler.onNodesUpdated(mockUpdatedNodes);
@@ -741,7 +741,7 @@ public class TestTaskScheduler {
     verify(mockRMClient).releaseAssignedContainer(mockCId4);
 
     // deallocate allocated task
-    assertTrue(scheduler.deallocateTask(mockTask1, true));
+    assertTrue(scheduler.deallocateTask(mockTask1, true, null));
     drainableAppCallback.drain();
     verify(mockApp).containerBeingReleased(mockCId1);
     verify(mockRMClient).releaseAssignedContainer(mockCId1);
@@ -871,7 +871,7 @@ public class TestTaskScheduler {
     verify(mockApp, times(4)).taskAllocated(any(), any(), (Container) any());
     verify(mockApp).taskAllocated(mockTask4, mockCookie4, mockContainer6);
     // deallocate allocated task
-    assertTrue(scheduler.deallocateTask(mockTask4, true));
+    assertTrue(scheduler.deallocateTask(mockTask4, true, null));
     drainableAppCallback.drain();
     verify(mockApp).containerBeingReleased(mockCId6);
     verify(mockRMClient).releaseAssignedContainer(mockCId6);
@@ -960,8 +960,8 @@ public class TestTaskScheduler {
     // container7 allocated to the task with affinity for it
     verify(mockApp).taskAllocated(mockTask6, mockCookie6, mockContainer7);
     // deallocate allocated task
-    assertTrue(scheduler.deallocateTask(mockTask5, true));
-    assertTrue(scheduler.deallocateTask(mockTask6, true));
+    assertTrue(scheduler.deallocateTask(mockTask5, true, null));
+    assertTrue(scheduler.deallocateTask(mockTask6, true, null));
     drainableAppCallback.drain();
     verify(mockApp).containerBeingReleased(mockCId7);
     verify(mockApp).containerBeingReleased(mockCId8);

http://git-wip-us.apache.org/repos/asf/tez/blob/c115a4ac/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/rm/TezTestServiceTaskSchedulerService.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/rm/TezTestServiceTaskSchedulerService.java b/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/rm/TezTestServiceTaskSchedulerService.java
index 5657f86..872d592 100644
--- a/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/rm/TezTestServiceTaskSchedulerService.java
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/rm/TezTestServiceTaskSchedulerService.java
@@ -38,6 +38,7 @@ import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.tez.dag.api.TaskAttemptEndReason;
 import org.apache.tez.dag.app.AppContext;
 import org.apache.tez.service.TezTestServiceConfConstants;
 
@@ -198,7 +199,7 @@ public class TezTestServiceTaskSchedulerService extends TaskSchedulerService {
   }
 
   @Override
-  public boolean deallocateTask(Object task, boolean taskSucceeded) {
+  public boolean deallocateTask(Object task, boolean taskSucceeded, TaskAttemptEndReason endReason) {
     ContainerId containerId = runningTasks.remove(task);
     if (containerId == null) {
       LOG.error("Could not determine ContainerId for task: " + task +

http://git-wip-us.apache.org/repos/asf/tez/blob/c115a4ac/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/taskcomm/TezTestServiceTaskCommunicatorImpl.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/taskcomm/TezTestServiceTaskCommunicatorImpl.java b/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/taskcomm/TezTestServiceTaskCommunicatorImpl.java
index a327caf..e3385a2 100644
--- a/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/taskcomm/TezTestServiceTaskCommunicatorImpl.java
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/taskcomm/TezTestServiceTaskCommunicatorImpl.java
@@ -19,16 +19,20 @@ import java.nio.ByteBuffer;
 import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.RejectedExecutionException;
 
 import com.google.protobuf.ByteString;
+import com.google.protobuf.ServiceException;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.DataOutputBuffer;
+import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.yarn.api.ApplicationConstants;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.LocalResource;
+import org.apache.tez.dag.api.TaskAttemptEndReason;
 import org.apache.tez.dag.api.TaskCommunicatorContext;
 import org.apache.tez.dag.app.TezTaskCommunicatorImpl;
 import org.apache.tez.dag.app.TezTestServiceCommunicator;
@@ -83,6 +87,7 @@ public class TezTestServiceTaskCommunicatorImpl extends TezTaskCommunicatorImpl
   @Override
   public void serviceStop() {
     super.serviceStop();
+    this.communicator.stop();
   }
 
 
@@ -123,13 +128,15 @@ public class TezTestServiceTaskCommunicatorImpl extends TezTaskCommunicatorImpl
       throw new RuntimeException("ContainerInfo not found for container: " + containerId +
           ", while trying to launch task: " + taskSpec.getTaskAttemptID());
     }
+    // Have to register this up front right now. Otherwise, it's possible for the task to start
+    // sending out status/DONE/KILLED/FAILED messages before TAImpl knows how to handle them.
+    getTaskCommunicatorContext()
+        .taskStartedRemotely(taskSpec.getTaskAttemptID(), containerId);
     communicator.submitWork(requestProto, host, port,
         new TezTestServiceCommunicator.ExecuteRequestCallback<SubmitWorkResponseProto>() {
           @Override
           public void setResponse(SubmitWorkResponseProto response) {
             LOG.info("Successfully launched task: " + taskSpec.getTaskAttemptID());
-            getTaskCommunicatorContext()
-                .taskStartedRemotely(taskSpec.getTaskAttemptID(), containerId);
           }
 
           @Override
@@ -137,6 +144,31 @@ public class TezTestServiceTaskCommunicatorImpl extends TezTaskCommunicatorImpl
             // TODO Handle this error. This is where an API on the context to indicate failure / rejection comes in.
             LOG.info("Failed to run task: " + taskSpec.getTaskAttemptID() + " on containerId: " +
                 containerId, t);
+            if (t instanceof ServiceException) {
+              ServiceException se = (ServiceException) t;
+              t = se.getCause();
+            }
+            if (t instanceof RemoteException) {
+              RemoteException re = (RemoteException)t;
+              String message = re.toString();
+              if (message.contains(RejectedExecutionException.class.getName())) {
+                getTaskCommunicatorContext().taskKilled(taskSpec.getTaskAttemptID(),
+                    TaskAttemptEndReason.SERVICE_BUSY, "Service Busy");
+              } else {
+                getTaskCommunicatorContext()
+                    .taskFailed(taskSpec.getTaskAttemptID(), TaskAttemptEndReason.OTHER,
+                        t.toString());
+              }
+            } else {
+              if (t instanceof IOException) {
+                getTaskCommunicatorContext().taskKilled(taskSpec.getTaskAttemptID(),
+                    TaskAttemptEndReason.COMMUNICATION_ERROR, "Communication Error");
+              } else {
+                getTaskCommunicatorContext()
+                    .taskFailed(taskSpec.getTaskAttemptID(), TaskAttemptEndReason.OTHER,
+                        t.getMessage());
+              }
+            }
           }
         });
   }

http://git-wip-us.apache.org/repos/asf/tez/blob/c115a4ac/tez-ext-service-tests/src/test/java/org/apache/tez/service/ContainerRunner.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/service/ContainerRunner.java b/tez-ext-service-tests/src/test/java/org/apache/tez/service/ContainerRunner.java
index 2bca4ed..28c2286 100644
--- a/tez-ext-service-tests/src/test/java/org/apache/tez/service/ContainerRunner.java
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/service/ContainerRunner.java
@@ -16,12 +16,13 @@ package org.apache.tez.service;
 
 import java.io.IOException;
 
+import org.apache.tez.dag.api.TezException;
 import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos;
 import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos.RunContainerRequestProto;
 import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos.SubmitWorkRequestProto;
 
 public interface ContainerRunner {
 
-  void queueContainer(RunContainerRequestProto request) throws IOException;
-  void submitWork(SubmitWorkRequestProto request) throws IOException;
+  void queueContainer(RunContainerRequestProto request) throws TezException;
+  void submitWork(SubmitWorkRequestProto request) throws TezException;
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/c115a4ac/tez-ext-service-tests/src/test/java/org/apache/tez/service/MiniTezTestServiceCluster.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/service/MiniTezTestServiceCluster.java b/tez-ext-service-tests/src/test/java/org/apache/tez/service/MiniTezTestServiceCluster.java
index f47bd67..0ac0b33 100644
--- a/tez-ext-service-tests/src/test/java/org/apache/tez/service/MiniTezTestServiceCluster.java
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/service/MiniTezTestServiceCluster.java
@@ -133,7 +133,10 @@ public class MiniTezTestServiceCluster extends AbstractService {
 
   @Override
   public void serviceStop() {
-    tezTestService.stop();
+    if (tezTestService != null) {
+      tezTestService.stop();
+      tezTestService = null;
+    }
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/tez/blob/c115a4ac/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/ContainerRunnerImpl.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/ContainerRunnerImpl.java b/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/ContainerRunnerImpl.java
index 25d6030..379d952 100644
--- a/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/ContainerRunnerImpl.java
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/ContainerRunnerImpl.java
@@ -25,6 +25,7 @@ import java.util.Map;
 import java.util.concurrent.Callable;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
+import java.util.concurrent.RejectedExecutionException;
 import java.util.concurrent.atomic.AtomicLong;
 import java.util.concurrent.atomic.AtomicReference;
 
@@ -58,6 +59,7 @@ import org.apache.tez.common.security.JobTokenIdentifier;
 import org.apache.tez.common.security.TokenCache;
 import org.apache.tez.dag.api.TezConfiguration;
 import org.apache.tez.dag.api.TezException;
+import org.apache.tez.runtime.api.impl.TaskSpec;
 import org.apache.tez.runtime.task.TaskReporter;
 import org.apache.tez.runtime.task.TezTaskRunner;
 import org.apache.tez.service.ContainerRunner;
@@ -68,14 +70,18 @@ import org.apache.tez.runtime.common.objectregistry.ObjectRegistryImpl;
 import org.apache.tez.runtime.task.TezChild;
 import org.apache.tez.runtime.task.TezChild.ContainerExecutionResult;
 import org.apache.tez.shufflehandler.ShuffleHandler;
+import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos;
 import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos.RunContainerRequestProto;
 import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos.SubmitWorkRequestProto;
+import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos.TaskSpecProto;
 import org.apache.tez.util.ProtoConverters;
 
 public class ContainerRunnerImpl extends AbstractService implements ContainerRunner {
 
   private static final Logger LOG = Logger.getLogger(ContainerRunnerImpl.class);
 
+  public static final String DAG_NAME_INSTRUMENTED_FAILURES = "InstrumentedFailures";
+
   private final ListeningExecutorService executorService;
   private final AtomicReference<InetSocketAddress> localAddress;
   private final String[] localDirsBase;
@@ -146,10 +152,10 @@ public class ContainerRunnerImpl extends AbstractService implements ContainerRun
    * Submit a container which is ready for running.
    * The regular pull mechanism will be used to fetch work from the AM
    * @param request
-   * @throws IOException
+   * @throws TezException
    */
   @Override
-  public void queueContainer(RunContainerRequestProto request) throws IOException {
+  public void queueContainer(RunContainerRequestProto request) throws TezException {
     LOG.info("Queuing container for execution: " + request);
 
     Map<String, String> env = new HashMap<String, String>();
@@ -162,7 +168,11 @@ public class ContainerRunnerImpl extends AbstractService implements ContainerRun
     for (int i = 0; i < localDirsBase.length; i++) {
       localDirs[i] = createAppSpecificLocalDir(localDirsBase[i], request.getApplicationIdString(),
           request.getUser());
-      localFs.mkdirs(new Path(localDirs[i]));
+      try {
+        localFs.mkdirs(new Path(localDirs[i]));
+      } catch (IOException e) {
+        throw new TezException(e);
+      }
     }
     LOG.info("DEBUG: Dirs are: " + Arrays.toString(localDirs));
 
@@ -175,7 +185,11 @@ public class ContainerRunnerImpl extends AbstractService implements ContainerRun
     DataInputBuffer dib = new DataInputBuffer();
     byte[] tokenBytes = request.getCredentialsBinary().toByteArray();
     dib.reset(tokenBytes, tokenBytes.length);
-    credentials.readTokenStorageStream(dib);
+    try {
+      credentials.readTokenStorageStream(dib);
+    } catch (IOException e) {
+      throw new TezException(e);
+    }
 
     Token<JobTokenIdentifier> jobToken = TokenCache.getSessionToken(credentials);
 
@@ -197,13 +211,14 @@ public class ContainerRunnerImpl extends AbstractService implements ContainerRun
    * This is intended for a task push from the AM
    *
    * @param request
-   * @throws IOException
+   * @throws org.apache.tez.dag.api.TezException
    */
   @Override
-  public void submitWork(SubmitWorkRequestProto request) throws
-      IOException {
+  public void submitWork(SubmitWorkRequestProto request) throws TezException {
     LOG.info("Queuing work for execution: " + request);
 
+    checkAndThrowExceptionForTests(request);
+
     Map<String, String> env = new HashMap<String, String>();
     env.putAll(localEnv);
     env.put(ApplicationConstants.Environment.USER.name(), request.getUser());
@@ -214,7 +229,11 @@ public class ContainerRunnerImpl extends AbstractService implements ContainerRun
     for (int i = 0; i < localDirsBase.length; i++) {
       localDirs[i] = createAppSpecificLocalDir(localDirsBase[i], request.getApplicationIdString(),
           request.getUser());
-      localFs.mkdirs(new Path(localDirs[i]));
+      try {
+        localFs.mkdirs(new Path(localDirs[i]));
+      } catch (IOException e) {
+        throw new TezException(e);
+      }
     }
     if (LOG.isDebugEnabled()) {
       LOG.debug("Dirs are: " + Arrays.toString(localDirs));
@@ -228,7 +247,11 @@ public class ContainerRunnerImpl extends AbstractService implements ContainerRun
     DataInputBuffer dib = new DataInputBuffer();
     byte[] tokenBytes = request.getCredentialsBinary().toByteArray();
     dib.reset(tokenBytes, tokenBytes.length);
-    credentials.readTokenStorageStream(dib);
+    try {
+      credentials.readTokenStorageStream(dib);
+    } catch (IOException e) {
+      throw new TezException(e);
+    }
 
     Token<JobTokenIdentifier> jobToken = TokenCache.getSessionToken(credentials);
 
@@ -509,4 +532,23 @@ public class ContainerRunnerImpl extends AbstractService implements ContainerRun
     }
   }
 
+
+  private void checkAndThrowExceptionForTests(SubmitWorkRequestProto request) throws TezException {
+    if (!request.getTaskSpec().getDagName().equals(DAG_NAME_INSTRUMENTED_FAILURES)) {
+      return;
+    }
+
+    TaskSpec taskSpec = ProtoConverters.getTaskSpecfromProto(request.getTaskSpec());
+    if (taskSpec.getTaskAttemptID().getTaskID().getId() == 0 &&
+        taskSpec.getTaskAttemptID().getId() == 0) {
+      LOG.info("Simulating Rejected work");
+      throw new RejectedExecutionException(
+          "Simulating Rejected work for taskAttemptId=" + taskSpec.getTaskAttemptID());
+    } else if (taskSpec.getTaskAttemptID().getTaskID().getId() == 1 &&
+        taskSpec.getTaskAttemptID().getId() == 0) {
+      LOG.info("Simulating Task Setup Failure during launch");
+      throw new TezException("Simulating Task Setup Failure during launch for taskAttemptId=" +
+          taskSpec.getTaskAttemptID());
+    }
+  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tez/blob/c115a4ac/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/TezTestService.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/TezTestService.java b/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/TezTestService.java
index 012e352..855f1b0 100644
--- a/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/TezTestService.java
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/TezTestService.java
@@ -14,7 +14,6 @@
 
 package org.apache.tez.service.impl;
 
-import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.util.Arrays;
 import java.util.concurrent.atomic.AtomicInteger;
@@ -25,6 +24,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.service.AbstractService;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.log4j.Logger;
+import org.apache.tez.dag.api.TezException;
 import org.apache.tez.service.ContainerRunner;
 import org.apache.tez.shufflehandler.ShuffleHandler;
 import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos;
@@ -108,14 +108,14 @@ public class TezTestService extends AbstractService implements ContainerRunner {
 
 
   @Override
-  public void queueContainer(RunContainerRequestProto request) throws IOException {
+  public void queueContainer(RunContainerRequestProto request) throws TezException {
     numSubmissions.incrementAndGet();
     containerRunner.queueContainer(request);
   }
 
   @Override
   public void submitWork(TezTestServiceProtocolProtos.SubmitWorkRequestProto request) throws
-      IOException {
+      TezException {
     numSubmissions.incrementAndGet();
     containerRunner.submitWork(request);
   }

http://git-wip-us.apache.org/repos/asf/tez/blob/c115a4ac/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/TezTestServiceProtocolServerImpl.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/TezTestServiceProtocolServerImpl.java b/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/TezTestServiceProtocolServerImpl.java
index d7f8444..39d7156 100644
--- a/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/TezTestServiceProtocolServerImpl.java
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/TezTestServiceProtocolServerImpl.java
@@ -30,11 +30,13 @@ import org.apache.hadoop.ipc.ProtobufRpcEngine;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.service.AbstractService;
+import org.apache.tez.dag.api.TezException;
 import org.apache.tez.service.ContainerRunner;
 import org.apache.tez.service.TezTestServiceProtocolBlockingPB;
 import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos;
 import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos.RunContainerRequestProto;
 import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos.RunContainerResponseProto;
+import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos.SubmitWorkRequestProto;
 import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos.SubmitWorkResponseProto;
 
 public class TezTestServiceProtocolServerImpl extends AbstractService
@@ -61,20 +63,20 @@ public class TezTestServiceProtocolServerImpl extends AbstractService
     LOG.info("Received request: " + request);
     try {
       containerRunner.queueContainer(request);
-    } catch (IOException e) {
+    } catch (TezException e) {
       throw new ServiceException(e);
     }
     return RunContainerResponseProto.getDefaultInstance();
   }
 
   @Override
-  public SubmitWorkResponseProto submitWork(RpcController controller, TezTestServiceProtocolProtos.SubmitWorkRequestProto request) throws
+  public SubmitWorkResponseProto submitWork(RpcController controller, SubmitWorkRequestProto request) throws
       ServiceException {
     LOG.info("Received submitWork request: " + request);
     try {
       containerRunner.submitWork(request);
-    } catch (IOException e) {
-      e.printStackTrace();
+    } catch (TezException e) {
+      throw new ServiceException(e);
     }
     return SubmitWorkResponseProto.getDefaultInstance();
   }

http://git-wip-us.apache.org/repos/asf/tez/blob/c115a4ac/tez-ext-service-tests/src/test/java/org/apache/tez/tests/TestExternalTezServices.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/tests/TestExternalTezServices.java b/tez-ext-service-tests/src/test/java/org/apache/tez/tests/TestExternalTezServices.java
index 0ec972b..b6a166d 100644
--- a/tez-ext-service-tests/src/test/java/org/apache/tez/tests/TestExternalTezServices.java
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/tests/TestExternalTezServices.java
@@ -27,16 +27,23 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.tez.client.TezClient;
+import org.apache.tez.dag.api.DAG;
+import org.apache.tez.dag.api.ProcessorDescriptor;
 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.Vertex;
+import org.apache.tez.dag.api.client.DAGClient;
+import org.apache.tez.dag.api.client.DAGStatus;
 import org.apache.tez.dag.app.launcher.TezTestServiceNoOpContainerLauncher;
 import org.apache.tez.dag.app.rm.TezTestServiceTaskSchedulerService;
 import org.apache.tez.dag.app.taskcomm.TezTestServiceTaskCommunicatorImpl;
 import org.apache.tez.examples.HashJoinExample;
 import org.apache.tez.examples.JoinDataGen;
 import org.apache.tez.examples.JoinValidateConfigured;
+import org.apache.tez.runtime.library.processor.SleepProcessor;
 import org.apache.tez.service.MiniTezTestServiceCluster;
+import org.apache.tez.service.impl.ContainerRunnerImpl;
 import org.apache.tez.test.MiniTezCluster;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
@@ -283,6 +290,28 @@ public class TestExternalTezServices {
         PROPS_IN_AM, PROPS_REGULAR_CONTAINERS);
   }
 
+  @Test(timeout = 60000)
+  public void testErrorPropagation() throws TezException, InterruptedException, IOException {
+    runExceptionSimulation();
+  }
+
+
+
+  private void runExceptionSimulation() throws IOException, TezException, InterruptedException {
+    DAG dag = DAG.create(ContainerRunnerImpl.DAG_NAME_INSTRUMENTED_FAILURES);
+    Vertex v =Vertex.create("Vertex1", ProcessorDescriptor.create(SleepProcessor.class.getName()),
+        3);
+    for (Map.Entry<String, String> prop : PROPS_EXT_SERVICE_PUSH.entrySet()) {
+      v.setConf(prop.getKey(), prop.getValue());
+    }
+    dag.addVertex(v);
+    DAGClient dagClient = sharedTezClient.submitDAG(dag);
+    DAGStatus dagStatus = dagClient.waitForCompletion();
+    assertEquals(DAGStatus.State.SUCCEEDED, dagStatus.getState());
+    assertEquals(1, dagStatus.getDAGProgress().getFailedTaskAttemptCount());
+    assertEquals(1, dagStatus.getDAGProgress().getKilledTaskAttemptCount());
+
+  }
 
   private void runJoinValidate(String name, int extExpectedCount, Map<String, String> lhsProps,
                                Map<String, String> rhsProps,


[13/50] [abbrv] tez git commit: TEZ-2405. PipelinedSorter can throw NPE with custom compartor (rbalamohan)

Posted by ss...@apache.org.
TEZ-2405. PipelinedSorter can throw NPE with custom compartor (rbalamohan)


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

Branch: refs/heads/TEZ-2003
Commit: c411e4edced690d111dac3cf2afcbb6cd39354f4
Parents: f6ea0fb
Author: Rajesh Balamohan <rb...@apache.org>
Authored: Mon May 4 12:38:34 2015 +0530
Committer: Rajesh Balamohan <rb...@apache.org>
Committed: Mon May 4 12:38:34 2015 +0530

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../common/sort/impl/PipelinedSorter.java       |  2 +-
 .../common/sort/impl/TestPipelinedSorter.java   | 53 +++++++++++++++++---
 3 files changed, 47 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/c411e4ed/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 8108ac8..6c19770 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -9,6 +9,7 @@ INCOMPATIBLE CHANGES
   TEZ-1993. Implement a pluggable InputSizeEstimator for grouping fairly
 
 ALL CHANGES:
+  TEZ-2405. PipelinedSorter can throw NPE with custom compartor.
   TEZ-1897. Create a concurrent version of AsyncDispatcher
   TEZ-2394. Issues when there is an error in VertexManager callbacks
   TEZ-2386. Tez UI: Inconsistent usage of icon colors

http://git-wip-us.apache.org/repos/asf/tez/blob/c411e4ed/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
index 65606bf..661f54c 100644
--- 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
@@ -749,7 +749,7 @@ public class PipelinedSorter extends ExternalSorter {
         cmp = comparator.compare(buf,
             keystart + off , (valstart - keystart),
             needle.getData(),
-            needle.getPosition(), needle.getLength());
+            needle.getPosition(), (needle.getLength() - needle.getPosition()));
       }
       return cmp;
     }

http://git-wip-us.apache.org/repos/asf/tez/blob/c411e4ed/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/sort/impl/TestPipelinedSorter.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/sort/impl/TestPipelinedSorter.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/sort/impl/TestPipelinedSorter.java
index 6e56567..5de96c9 100644
--- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/sort/impl/TestPipelinedSorter.java
+++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/sort/impl/TestPipelinedSorter.java
@@ -8,6 +8,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.DataInputBuffer;
 import org.apache.hadoop.io.DataOutputBuffer;
 import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.WritableComparator;
 import org.apache.hadoop.io.serializer.Deserializer;
 import org.apache.hadoop.io.serializer.SerializationFactory;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
@@ -22,6 +23,7 @@ import org.apache.tez.runtime.library.common.shuffle.ShuffleUtils;
 import org.apache.tez.runtime.library.conf.OrderedPartitionedKVOutputConfig.SorterImpl;
 import org.apache.tez.runtime.library.partitioner.HashPartitioner;
 import org.junit.After;
+import org.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
@@ -61,10 +63,10 @@ public class TestPipelinedSorter {
   private static final Configuration conf = new Configuration();
   private static FileSystem localFs = null;
   private static Path workDir = null;
+  private OutputContext outputContext;
 
   private int numOutputs;
   private long initialAvailableMem;
-  private OutputContext outputContext;
 
   //TODO: Need to make it nested structure so that multiple partition cases can be validated
   private static TreeMap<String, String> sortedDataMap = Maps.newTreeMap();
@@ -82,6 +84,11 @@ public class TestPipelinedSorter {
     }
   }
 
+  @AfterClass
+  public static void cleanup() throws IOException {
+    localFs.delete(workDir, true);
+  }
+
   @Before
   public void setup() throws IOException {
     ApplicationId appId = ApplicationId.newInstance(10000, 1);
@@ -89,12 +96,14 @@ public class TestPipelinedSorter {
     String uniqueId = UUID.randomUUID().toString();
     this.outputContext = createMockOutputContext(counters, appId, uniqueId);
 
-    //To enable PipelinedSorter, set 2 threads
+    //To enable PipelinedSorter
     conf.set(TezRuntimeConfiguration.TEZ_RUNTIME_SORTER_CLASS, SorterImpl.PIPELINED.name());
+
     conf.set(TezRuntimeConfiguration.TEZ_RUNTIME_KEY_CLASS, Text.class.getName());
     conf.set(TezRuntimeConfiguration.TEZ_RUNTIME_VALUE_CLASS, Text.class.getName());
-    conf.set(TezRuntimeConfiguration.TEZ_RUNTIME_PARTITIONER_CLASS,
-        HashPartitioner.class.getName());
+    conf.set(TezRuntimeConfiguration.TEZ_RUNTIME_PARTITIONER_CLASS, HashPartitioner.class.getName());
+
+    conf.setBoolean(TezRuntimeConfiguration.TEZ_RUNTIME_ENABLE_FINAL_MERGE_IN_OUTPUT, true);
 
     //Setup localdirs
     String localDirs = workDir.toString();
@@ -102,9 +111,8 @@ public class TestPipelinedSorter {
   }
 
   @After
-  public void cleanup() throws IOException {
-    localFs.delete(workDir, true);
-    sortedDataMap.clear();
+  public void reset() throws IOException {
+    cleanup();
     localFs.mkdirs(workDir);
   }
 
@@ -133,6 +141,13 @@ public class TestPipelinedSorter {
   }
 
   @Test
+  public void testWithCustomComparator() throws IOException {
+    //Test with custom comparator
+    conf.set(TezRuntimeConfiguration.TEZ_RUNTIME_KEY_COMPARATOR_CLASS, CustomComparator.class.getName());
+    basicTest(1, 100000, 100, (10 * 1024l * 1024l), 3 << 20);
+  }
+
+  @Test
   public void testWithPipelinedShuffle() throws IOException {
     this.numOutputs = 1;
     this.initialAvailableMem = 5 *1024 * 1024;
@@ -251,7 +266,7 @@ public class TestPipelinedSorter {
     Assert.assertTrue(numRecordsRead == sortedDataMap.size());
   }
 
-  private OutputContext createMockOutputContext(TezCounters counters, ApplicationId appId,
+  private static OutputContext createMockOutputContext(TezCounters counters, ApplicationId appId,
       String uniqueId) throws IOException {
     OutputContext outputContext = mock(OutputContext.class);
 
@@ -280,4 +295,26 @@ public class TestPipelinedSorter {
     doReturn(outDirs).when(outputContext).getWorkDirs();
     return outputContext;
   }
+
+  /**
+   * E.g Hive uses TezBytesComparator which internally makes use of WritableComparator's comparison.
+   * Any length mismatches are handled there.
+   *
+   * However, custom comparators can handle this differently and might throw
+   * IndexOutOfBoundsException in case of invalid lengths.
+   *
+   * This comparator (similar to comparator in BinInterSedes of pig) would thrown exception when
+   * wrong lengths are mentioned.
+   */
+  public static class CustomComparator extends WritableComparator {
+    @Override
+    public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2) {
+      //wrapping is done so that it would throw exceptions on wrong lengths
+      ByteBuffer bb1 = ByteBuffer.wrap(b1, s1, l1);
+      ByteBuffer bb2 = ByteBuffer.wrap(b2, s2, l2);
+
+      return bb1.compareTo(bb2);
+    }
+
+  }
 }


[50/50] [abbrv] tez git commit: TEZ-2414. LogicalIOProcessorRuntimeTask, RuntimeTask, TezTaskRunner should handle interrupts & carry out necessary cleanups. Contribtued by Rajesh Balamohan.

Posted by ss...@apache.org.
TEZ-2414. LogicalIOProcessorRuntimeTask, RuntimeTask, TezTaskRunner should handle interrupts & carry out necessary cleanups. Contribtued by Rajesh Balamohan.


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

Branch: refs/heads/TEZ-2003
Commit: 9338abfe9a12271e4247a7cf45f3f8fff30e6bb7
Parents: e879128
Author: Siddharth Seth <ss...@apache.org>
Authored: Wed May 6 00:39:46 2015 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Wed May 6 00:39:46 2015 -0700

----------------------------------------------------------------------
 TEZ-2003-CHANGES.txt                            |   1 +
 .../runtime/LogicalIOProcessorRuntimeTask.java  | 142 +++++++++++++++++++
 .../org/apache/tez/runtime/RuntimeTask.java     |   5 +
 .../apache/tez/runtime/task/TezTaskRunner.java  |  71 +++++++++-
 4 files changed, 217 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/9338abfe/TEZ-2003-CHANGES.txt
----------------------------------------------------------------------
diff --git a/TEZ-2003-CHANGES.txt b/TEZ-2003-CHANGES.txt
index 9fc9ed3..f8a71e8 100644
--- a/TEZ-2003-CHANGES.txt
+++ b/TEZ-2003-CHANGES.txt
@@ -20,5 +20,6 @@ ALL CHANGES:
   TEZ-2361. Propagate dag completion to TaskCommunicator.
   TEZ-2381. Fixes after rebase 04/28.
   TEZ-2388. Send dag identifier as part of the fetcher request string.
+  TEZ-2414. LogicalIOProcessorRuntimeTask, RuntimeTask, TezTaskRunner should handle interrupts & carry out necessary cleanups.
 
 INCOMPATIBLE CHANGES:

http://git-wip-us.apache.org/repos/asf/tez/blob/9338abfe/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
index f465d3c..7040598 100644
--- 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
@@ -20,10 +20,14 @@ package org.apache.tez.runtime;
 
 import java.io.Closeable;
 import java.io.IOException;
+import java.lang.management.ManagementFactory;
+import java.lang.management.ThreadInfo;
+import java.lang.management.ThreadMXBean;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
+import java.util.Iterator;
 import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
@@ -37,6 +41,7 @@ import java.util.concurrent.Executors;
 import java.util.concurrent.Future;
 import java.util.concurrent.LinkedBlockingQueue;
 
+import com.google.common.base.Throwables;
 import org.apache.commons.lang.exception.ExceptionUtils;
 import org.apache.tez.runtime.api.TaskContext;
 import org.apache.tez.runtime.api.impl.TezProcessorContextImpl;
@@ -109,6 +114,10 @@ public class LogicalIOProcessorRuntimeTask extends RuntimeTask {
   private final List<GroupInputSpec> groupInputSpecs;
   private ConcurrentHashMap<String, MergedLogicalInput> groupInputsMap;
 
+  private final ConcurrentHashMap<String, LogicalInput> initializedInputs;
+  private final ConcurrentHashMap<String, LogicalOutput> initializedOutputs;
+  private boolean processorClosed;
+
   private final ProcessorDescriptor processorDescriptor;
   private AbstractLogicalIOProcessor processor;
   private ProcessorContext processorContext;
@@ -160,6 +169,9 @@ public class LogicalIOProcessorRuntimeTask extends RuntimeTask {
     this.runInputMap = new LinkedHashMap<String, LogicalInput>();
     this.runOutputMap = new LinkedHashMap<String, LogicalOutput>();
 
+    this.initializedInputs = new ConcurrentHashMap<String, LogicalInput>();
+    this.initializedOutputs = new ConcurrentHashMap<String, LogicalOutput>();
+
     this.processorDescriptor = taskSpec.getProcessorDescriptor();
     this.serviceConsumerMetadata = serviceConsumerMetadata;
     this.envMap = envMap;
@@ -338,11 +350,13 @@ public class LogicalIOProcessorRuntimeTask extends RuntimeTask {
       this.state.set(State.CLOSED);
 
       // Close the Processor.
+      processorClosed = true;
       processor.close();
 
       // Close the Inputs.
       for (InputSpec inputSpec : inputSpecs) {
         String srcVertexName = inputSpec.getSourceVertexName();
+        initializedInputs.remove(srcVertexName);
         List<Event> closeInputEvents = ((InputFrameworkInterface)inputsMap.get(srcVertexName)).close();
         sendTaskGeneratedEvents(closeInputEvents,
             EventProducerConsumerType.INPUT, taskSpec.getVertexName(),
@@ -352,6 +366,7 @@ public class LogicalIOProcessorRuntimeTask extends RuntimeTask {
       // Close the Outputs.
       for (OutputSpec outputSpec : outputSpecs) {
         String destVertexName = outputSpec.getDestinationVertexName();
+        initializedOutputs.remove(destVertexName);
         List<Event> closeOutputEvents = ((LogicalOutputFrameworkInterface)outputsMap.get(destVertexName)).close();
         sendTaskGeneratedEvents(closeOutputEvents,
             EventProducerConsumerType.OUTPUT, taskSpec.getVertexName(),
@@ -391,6 +406,7 @@ public class LogicalIOProcessorRuntimeTask extends RuntimeTask {
           inputContext.getTaskVertexName(), inputContext.getSourceVertexName(),
           taskSpec.getTaskAttemptID());
       LOG.info("Initialized Input with src edge: " + edgeName);
+      initializedInputs.put(edgeName, input);
       return null;
     }
   }
@@ -439,6 +455,7 @@ public class LogicalIOProcessorRuntimeTask extends RuntimeTask {
           outputContext.getTaskVertexName(),
           outputContext.getDestinationVertexName(), taskSpec.getTaskAttemptID());
       LOG.info("Initialized Output with dest edge: " + edgeName);
+      initializedOutputs.put(edgeName, output);
       return null;
     }
   }
@@ -664,6 +681,13 @@ public class LogicalIOProcessorRuntimeTask extends RuntimeTask {
     eventsToBeProcessed.addAll(events);
   }
 
+  @Override
+  public synchronized void abortTask() throws Exception {
+    if (processor != null) {
+      processor.abort();
+    }
+  }
+
   private void startRouterThread() {
     eventRouterThread = new Thread(new RunnableWithNdc() {
       public void runInternal() {
@@ -683,6 +707,7 @@ public class LogicalIOProcessorRuntimeTask extends RuntimeTask {
             if (!isTaskDone()) {
               LOG.warn("Event Router thread interrupted. Returning.");
             }
+            Thread.currentThread().interrupt();
             return;
           }
         }
@@ -694,6 +719,12 @@ public class LogicalIOProcessorRuntimeTask extends RuntimeTask {
     eventRouterThread.start();
   }
 
+  private void maybeResetInterruptStatus() {
+    if (!Thread.currentThread().isInterrupted()) {
+      Thread.currentThread().interrupt();
+    }
+  }
+
   private void cleanupInputOutputs() {
     if (groupInputsMap != null) {
       groupInputsMap.clear();
@@ -726,6 +757,103 @@ public class LogicalIOProcessorRuntimeTask extends RuntimeTask {
   }
 
   public synchronized void cleanup() {
+
+    /**
+     * Cleanup IPO that are not closed.  In case, regular close() has happened in IPO, they
+     * would not be available in the IPOs to be cleaned. So this is safe.
+     *
+     * e.g whenever input gets closed() in normal way, it automatically removes it from
+     * initializedInputs map.
+     *
+     * In case any exception happens in processor close or IO close, it wouldn't be removed from
+     * the initialized IO data structures and here is the chance to close them and release
+     * resources.
+     *
+     */
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Processor closed={}", processorClosed);
+      LOG.debug("Num of inputs to be closed={}", initializedInputs.size());
+      LOG.debug("Num of outputs to be closed={}", initializedOutputs.size());
+    }
+    if (!processorClosed) {
+      try {
+        processorClosed = true;
+        processor.close();
+
+        LOG.info("Closed processor for vertex={}, index={}, interruptedStatus={}",
+            processor
+                .getContext().getTaskVertexName(),
+            processor.getContext().getTaskVertexIndex(),
+            Thread.currentThread().isInterrupted());
+
+        maybeResetInterruptStatus();
+      } catch (InterruptedException ie) {
+        //reset the status
+        LOG.info("Resetting interrupt for processor");
+        Thread.currentThread().interrupt();
+      } catch (Throwable e) {
+        LOG.warn("Exception when closing processor", e);
+      }
+    }
+    // Close the remaining inited Inputs.
+    Iterator<String> srcVertexItr = initializedInputs.keySet().iterator();
+    while (srcVertexItr.hasNext()) {
+      String srcVertexName = srcVertexItr.next();
+      try {
+        srcVertexItr.remove();
+
+        ((InputFrameworkInterface) initializedInputs.get(srcVertexName)).close();
+        initializedInputs.remove(srcVertexName);
+
+        maybeResetInterruptStatus();
+      } catch (InterruptedException ie) {
+        //reset the status
+        LOG.info("Resetting interrupt status for input with srcVertexName={}",
+            srcVertexName);
+        Thread.currentThread().interrupt();
+      } catch (Throwable e) {
+        LOG.warn("Exception when closing input in cleanup(interrupted)", e);
+      } finally {
+        LOG.info("Close input for vertex={}, sourceVertex={}, interruptedStatus={}", processor
+            .getContext().getTaskVertexName(), srcVertexName, Thread.currentThread()
+            .isInterrupted());
+      }
+    }
+
+    // Close the remaining inited Outputs.
+    try {
+      Iterator<String> outVertexItr = initializedOutputs.keySet().iterator();
+      while (outVertexItr.hasNext()) {
+        String destVertexName = outVertexItr.next();
+        try {
+          outVertexItr.remove();
+
+          ((OutputFrameworkInterface) initializedOutputs.get(destVertexName)).close();
+          initializedOutputs.remove(destVertexName);
+
+          maybeResetInterruptStatus();
+        } catch (InterruptedException ie) {
+          //reset the status
+          LOG.info("Resetting interrupt status for output with destVertexName={}",
+              destVertexName);
+          Thread.currentThread().interrupt();
+        } catch (Throwable e) {
+          LOG.warn("Exception when closing output in cleanup(interrupted)", e);
+        } finally {
+          LOG.info("Close input for vertex={}, sourceVertex={}, interruptedStatus={}", processor
+              .getContext().getTaskVertexName(), destVertexName, Thread.currentThread()
+              .isInterrupted());
+        }
+      }
+    } catch (Throwable e) {
+      LOG.warn(Throwables.getStackTraceAsString(e));
+    }
+
+    if (LOG.isDebugEnabled()) {
+      printThreads();
+    }
+
+
     try {
       cleanupInputOutputs();
       closeContexts();
@@ -739,6 +867,20 @@ public class LogicalIOProcessorRuntimeTask extends RuntimeTask {
       eventRouterThread.interrupt();
     }
   }
+
+
+  /**
+   * Print all threads in JVM (only for debugging)
+   */
+  void printThreads() {
+    //Print the status of all threads in JVM
+    ThreadMXBean threadMXBean = ManagementFactory.getThreadMXBean();
+    long[] threadIds = threadMXBean.getAllThreadIds();
+    for (Long id : threadIds) {
+      ThreadInfo threadInfo = threadMXBean.getThreadInfo(id);
+      LOG.info("ThreadId : " + id + ", name=" + threadInfo.getThreadName());
+    }
+  }
   
   @Private
   @VisibleForTesting

http://git-wip-us.apache.org/repos/asf/tez/blob/9338abfe/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
index f8b8621..162caf0 100644
--- 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
@@ -72,6 +72,10 @@ public abstract class RuntimeTask {
 
   protected final AtomicReference<State> state = new AtomicReference<State>();
 
+  public boolean isRunning() {
+    return (state.get() == State.RUNNING);
+  }
+
   public TezCounters addAndGetTezCounter(String name) {
     TezCounters counter = new TezCounters();
     counterMap.put(name, counter);
@@ -143,4 +147,5 @@ public abstract class RuntimeTask {
     taskDone.set(true);
   }
 
+  public abstract void abortTask() throws Exception;
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/9338abfe/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezTaskRunner.java
----------------------------------------------------------------------
diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezTaskRunner.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezTaskRunner.java
index 33a7f4a..7238d5e 100644
--- a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezTaskRunner.java
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezTaskRunner.java
@@ -25,8 +25,13 @@ import java.security.PrivilegedExceptionAction;
 import java.util.Collection;
 import java.util.Map;
 import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
 
+import com.google.common.base.Throwables;
 import org.apache.commons.lang.exception.ExceptionUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSError;
@@ -35,6 +40,7 @@ import org.apache.tez.common.CallableWithNdc;
 import org.apache.tez.dag.api.TezException;
 import org.apache.tez.dag.records.TezTaskAttemptID;
 import org.apache.tez.runtime.LogicalIOProcessorRuntimeTask;
+import org.apache.tez.runtime.RuntimeTask;
 import org.apache.tez.runtime.api.ExecutionContext;
 import org.apache.tez.runtime.api.ObjectRegistry;
 import org.apache.tez.runtime.api.impl.EventMetaData;
@@ -61,6 +67,7 @@ public class TezTaskRunner implements TezUmbilical, ErrorReporter {
   private final ListeningExecutorService executor;
   private volatile ListenableFuture<Void> taskFuture;
   private volatile Thread waitingThread;
+  private volatile Thread taskRunner;
   private volatile Throwable firstException;
 
   // Effectively a duplicate check, since hadFatalError does the same thing.
@@ -96,7 +103,10 @@ public class TezTaskRunner implements TezUmbilical, ErrorReporter {
     taskReporter.registerTask(task, this);
     TaskRunnerCallable callable = new TaskRunnerCallable();
     Throwable failureCause = null;
-    taskFuture = executor.submit(callable);
+    if (!Thread.currentThread().isInterrupted()) {
+      taskFuture = executor.submit(callable);
+      return isShutdownRequested();
+    }
     try {
       taskFuture.get();
 
@@ -158,6 +168,10 @@ public class TezTaskRunner implements TezUmbilical, ErrorReporter {
         }
       }
     }
+    return isShutdownRequested();
+  }
+
+  private boolean isShutdownRequested() {
     if (shutdownRequested.get()) {
       LOG.info("Shutdown requested... returning");
       return false;
@@ -173,11 +187,14 @@ public class TezTaskRunner implements TezUmbilical, ErrorReporter {
           @Override
           public Void run() throws Exception {
             try {
+              taskRunner = Thread.currentThread();
               LOG.info("Initializing task" + ", taskAttemptId=" + task.getTaskAttemptID());
               task.initialize();
               if (!Thread.currentThread().isInterrupted() && firstException == null) {
                 LOG.info("Running task, taskAttemptId=" + task.getTaskAttemptID());
                 task.run();
+                maybeInterruptWaitingThread();
+
                 LOG.info("Closing task, taskAttemptId=" + task.getTaskAttemptID());
                 task.close();
                 task.setFrameworkCounters();
@@ -199,6 +216,12 @@ public class TezTaskRunner implements TezUmbilical, ErrorReporter {
               }
               return null;
             } catch (Throwable cause) {
+              if (Thread.currentThread().isInterrupted()) {
+                LOG.info("TaskRunnerCallable interrupted=" + Thread.currentThread().isInterrupted()
+                    + ", shutdownRequest=" + shutdownRequested.get());
+                Thread.currentThread().interrupt();
+                return null;
+              }
               if (cause instanceof FSError) {
                 // Not immediately fatal, this is an error reported by Hadoop FileSystem
                 maybeRegisterFirstException(cause);
@@ -255,6 +278,17 @@ public class TezTaskRunner implements TezUmbilical, ErrorReporter {
         taskRunning.set(false);
       }
     }
+
+    private void maybeInterruptWaitingThread() {
+      /**
+       * Possible that the processor is swallowing InterruptException of taskRunner.interrupt().
+       * In such case, interrupt the waitingThread based on the shutdownRequested flag, so that
+       * entire task gets cancelled.
+       */
+      if (shutdownRequested.get()) {
+        waitingThread.interrupt();
+      }
+    }
   }
 
   // should wait until all messages are sent to AM before TezChild shutdown
@@ -353,10 +387,43 @@ public class TezTaskRunner implements TezUmbilical, ErrorReporter {
     }
   }
 
+  private void abortRunningTask() {
+    if (!taskRunning.get()) {
+      LOG.info("Task is not running");
+      waitingThread.interrupt();
+      return;
+    }
+
+    if (taskRunning.get()) {
+      try {
+        task.abortTask();
+      } catch (Exception e) {
+        LOG.warn("Error when aborting the task", e);
+        try {
+          sendFailure(e, "Error when aborting the task");
+        } catch (Exception ignored) {
+          // Ignored.
+        }
+      }
+    }
+    //Interrupt the relevant threads.  TaskRunner should be interrupted preferably.
+    if (isTaskRunning()) {
+      LOG.info("Interrupting taskRunner=" + taskRunner.getName());
+      taskRunner.interrupt();
+    } else {
+      LOG.info("Interrupting waitingThread=" + waitingThread.getName());
+      waitingThread.interrupt();
+    }
+  }
+
+  private boolean isTaskRunning() {
+    return (taskRunning.get() && task.isRunning());
+  }
+
   @Override
   public void shutdownRequested() {
     shutdownRequested.set(true);
-    waitingThread.interrupt();
+    abortRunningTask();
   }
 
   private String getTaskDiagnosticsString(Throwable t, String message) {


[17/50] [abbrv] tez git commit: TEZ-2408. TestTaskAttempt fails to compile against hadoop-2.4 and hadoop-2.2. (hitesh)

Posted by ss...@apache.org.
TEZ-2408. TestTaskAttempt fails to compile against hadoop-2.4 and hadoop-2.2. (hitesh)


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

Branch: refs/heads/TEZ-2003
Commit: 210619a56905941d8551f169a0848b4754c2809a
Parents: e762a35
Author: Hitesh Shah <hi...@apache.org>
Authored: Mon May 4 18:01:15 2015 -0700
Committer: Hitesh Shah <hi...@apache.org>
Committed: Mon May 4 18:01:15 2015 -0700

----------------------------------------------------------------------
 CHANGES.txt                                               |  1 +
 .../org/apache/tez/dag/app/dag/impl/TestTaskAttempt.java  | 10 +++++-----
 2 files changed, 6 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/210619a5/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 5b18258..2ff7601 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -9,6 +9,7 @@ INCOMPATIBLE CHANGES
   TEZ-1993. Implement a pluggable InputSizeEstimator for grouping fairly
 
 ALL CHANGES:
+  TEZ-2408. TestTaskAttempt fails to compile against hadoop-2.4 and hadoop-2.2.
   TEZ-2405. PipelinedSorter can throw NPE with custom compartor.
   TEZ-1897. Create a concurrent version of AsyncDispatcher
   TEZ-2394. Issues when there is an error in VertexManager callbacks

http://git-wip-us.apache.org/repos/asf/tez/blob/210619a5/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 50bb68c..60c4c88 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
@@ -725,7 +725,7 @@ public class TestTaskAttempt {
     Resource resource = Resource.newInstance(1024, 1);
 
     NodeId nid = NodeId.newInstance("127.0.0.1", 0);
-    ContainerId contId = ContainerId.newContainerId(appAttemptId, 3);
+    ContainerId contId = ContainerId.newInstance(appAttemptId, 3);
     Container container = mock(Container.class);
     when(container.getId()).thenReturn(contId);
     when(container.getNodeId()).thenReturn(nid);
@@ -816,7 +816,7 @@ public class TestTaskAttempt {
     Resource resource = Resource.newInstance(1024, 1);
 
     NodeId nid = NodeId.newInstance("127.0.0.1", 0);
-    ContainerId contId = ContainerId.newContainerId(appAttemptId, 3);
+    ContainerId contId = ContainerId.newInstance(appAttemptId, 3);
     Container container = mock(Container.class);
     when(container.getId()).thenReturn(contId);
     when(container.getNodeId()).thenReturn(nid);
@@ -911,7 +911,7 @@ public class TestTaskAttempt {
     Resource resource = Resource.newInstance(1024, 1);
 
     NodeId nid = NodeId.newInstance("127.0.0.1", 0);
-    ContainerId contId = ContainerId.newContainerId(appAttemptId, 3);
+    ContainerId contId = ContainerId.newInstance(appAttemptId, 3);
     Container container = mock(Container.class);
     when(container.getId()).thenReturn(contId);
     when(container.getNodeId()).thenReturn(nid);
@@ -1014,7 +1014,7 @@ public class TestTaskAttempt {
     Resource resource = Resource.newInstance(1024, 1);
 
     NodeId nid = NodeId.newInstance("127.0.0.1", 0);
-    ContainerId contId = ContainerId.newContainerId(appAttemptId, 3);
+    ContainerId contId = ContainerId.newInstance(appAttemptId, 3);
     Container container = mock(Container.class);
     when(container.getId()).thenReturn(contId);
     when(container.getNodeId()).thenReturn(nid);
@@ -1114,7 +1114,7 @@ public class TestTaskAttempt {
     Resource resource = Resource.newInstance(1024, 1);
 
     NodeId nid = NodeId.newInstance("127.0.0.1", 0);
-    ContainerId contId = ContainerId.newContainerId(appAttemptId, 3);
+    ContainerId contId = ContainerId.newInstance(appAttemptId, 3);
     Container container = mock(Container.class);
     when(container.getId()).thenReturn(contId);
     when(container.getNodeId()).thenReturn(nid);


[48/50] [abbrv] tez git commit: TEZ-2361. Propagate dag completion to TaskCommunicator. (sseth)

Posted by ss...@apache.org.
TEZ-2361. Propagate dag completion to TaskCommunicator. (sseth)


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

Branch: refs/heads/TEZ-2003
Commit: e5886f8e329a31e700f794cfc7c669866eaafc27
Parents: 960d92b
Author: Siddharth Seth <ss...@apache.org>
Authored: Thu Apr 23 17:26:25 2015 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Wed May 6 00:18:08 2015 -0700

----------------------------------------------------------------------
 TEZ-2003-CHANGES.txt                            |  1 +
 .../apache/tez/dag/api/TaskCommunicator.java    | 12 +++-
 .../org/apache/tez/dag/app/DAGAppMaster.java    |  4 +-
 .../dag/app/TaskAttemptListenerImpTezDag.java   | 17 +++++-
 .../dag/app/TaskCommunicatorContextImpl.java    | 64 +++++++++++++++++---
 .../tez/dag/app/TezTaskCommunicatorImpl.java    |  5 ++
 .../tez/dag/app/launcher/ContainerLauncher.java |  3 -
 .../dag/app/launcher/ContainerLauncherImpl.java | 12 ----
 .../app/launcher/ContainerLauncherRouter.java   | 10 +++
 .../app/launcher/LocalContainerLauncher.java    |  9 ---
 .../apache/tez/dag/app/MockDAGAppMaster.java    | 11 ----
 .../rm/TezTestServiceTaskSchedulerService.java  |  2 +-
 .../TezTestServiceTaskCommunicatorImpl.java     |  2 +-
 .../tez/tests/TestExternalTezServices.java      |  2 +
 14 files changed, 103 insertions(+), 51 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/e5886f8e/TEZ-2003-CHANGES.txt
----------------------------------------------------------------------
diff --git a/TEZ-2003-CHANGES.txt b/TEZ-2003-CHANGES.txt
index 7c13110..f6bc8e7 100644
--- a/TEZ-2003-CHANGES.txt
+++ b/TEZ-2003-CHANGES.txt
@@ -17,5 +17,6 @@ ALL CHANGES:
   TEZ-2285. Allow TaskCommunicators to indicate task/container liveness.
   TEZ-2302. Allow TaskCommunicators to subscribe for Vertex updates.
   TEZ-2347. Expose additional information in TaskCommunicatorContext.
+  TEZ-2361. Propagate dag completion to TaskCommunicator.
 
 INCOMPATIBLE CHANGES:

http://git-wip-us.apache.org/repos/asf/tez/blob/e5886f8e/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicator.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicator.java b/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicator.java
index a2cd858..cadca0c 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicator.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicator.java
@@ -14,7 +14,6 @@
 
 package org.apache.tez.dag.api;
 
-import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.util.Map;
 
@@ -74,4 +73,15 @@ public abstract class TaskCommunicator extends AbstractService {
    * @throws Exception
    */
   public abstract void onVertexStateUpdated(VertexStateUpdate stateUpdate) throws Exception;
+
+  /**
+   * Indicates the current running dag is complete. The TaskCommunicatorContext can be used to
+   * query information about the current dag during the duration of the dagComplete invocation.
+   *
+   * After this, the contents returned from querying the context may change at any point - due to
+   * the next dag being submitted.
+   */
+  // TODO TEZ-2003 This is extremely difficult to use. Add the dagStarted notification, and potentially
+  // throw exceptions between a dagComplete and dagStart invocation.
+  public abstract void dagComplete(String dagName);
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/e5886f8e/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 1ea369e..568e929 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
@@ -759,7 +759,7 @@ public class DAGAppMaster extends AbstractService {
       DAGAppMasterEventDagCleanup cleanupEvent = (DAGAppMasterEventDagCleanup) event;
       LOG.info("Cleaning up DAG: name=" + cleanupEvent.getDag().getName() + ", with id=" +
           cleanupEvent.getDag().getID());
-      containerLauncher.dagComplete(cleanupEvent.getDag());
+      containerLauncherRouter.dagComplete(cleanupEvent.getDag());
       taskAttemptListener.dagComplete(cleanupEvent.getDag());
       nodes.dagComplete(cleanupEvent.getDag());
       containers.dagComplete(cleanupEvent.getDag());
@@ -773,7 +773,7 @@ public class DAGAppMaster extends AbstractService {
     case NEW_DAG_SUBMITTED:
       // Inform sub-components that a new DAG has been submitted.
       taskSchedulerEventHandler.dagSubmitted();
-      containerLauncher.dagSubmitted();
+      containerLauncherRouter.dagSubmitted();
       taskAttemptListener.dagSubmitted();
       break;
     default:

http://git-wip-us.apache.org/repos/asf/tez/blob/e5886f8e/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 a7bbba9..61bd4ca 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
@@ -81,6 +81,7 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
 
   private final AppContext context;
   private final TaskCommunicator[] taskCommunicators;
+  private final TaskCommunicatorContext[] taskCommunicatorContexts;
 
   protected final TaskHeartbeatHandler taskHeartbeatHandler;
   protected final ContainerHeartbeatHandler containerHeartbeatHandler;
@@ -123,7 +124,9 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
       }
     }
     this.taskCommunicators = new TaskCommunicator[taskCommunicatorClassIdentifiers.length];
+    this.taskCommunicatorContexts = new TaskCommunicatorContext[taskCommunicatorClassIdentifiers.length];
     for (int i = 0 ; i < taskCommunicatorClassIdentifiers.length ; i++) {
+      taskCommunicatorContexts[i] = new TaskCommunicatorContextImpl(context, this, i);
       taskCommunicators[i] = createTaskCommunicator(taskCommunicatorClassIdentifiers[i], i);
     }
     // TODO TEZ-2118 Start using taskCommunicator indices properly
@@ -148,10 +151,10 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
   private TaskCommunicator createTaskCommunicator(String taskCommClassIdentifier, int taskCommIndex) {
     if (taskCommClassIdentifier.equals(TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT)) {
       LOG.info("Using Default Task Communicator");
-      return new TezTaskCommunicatorImpl(new TaskCommunicatorContextImpl(context, this, taskCommIndex));
+      return new TezTaskCommunicatorImpl(taskCommunicatorContexts[taskCommIndex]);
     } else if (taskCommClassIdentifier.equals(TezConstants.TEZ_AM_SERVICE_PLUGINS_LOCAL_MODE_NAME_DEFAULT)) {
       LOG.info("Using Default Local Task Communicator");
-      return new TezLocalTaskCommunicatorImpl(new TaskCommunicatorContextImpl(context, this, taskCommIndex));
+      return new TezLocalTaskCommunicatorImpl(taskCommunicatorContexts[taskCommIndex]);
     } else {
       LOG.info("Using TaskCommunicator: " + taskCommClassIdentifier);
       Class<? extends TaskCommunicator> taskCommClazz = (Class<? extends TaskCommunicator>) ReflectionUtils
@@ -159,7 +162,7 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
       try {
         Constructor<? extends TaskCommunicator> ctor = taskCommClazz.getConstructor(TaskCommunicatorContext.class);
         ctor.setAccessible(true);
-        return ctor.newInstance(new TaskCommunicatorContextImpl(context, this, taskCommIndex));
+        return ctor.newInstance(taskCommunicatorContexts[taskCommIndex]);
       } catch (NoSuchMethodException e) {
         throw new TezUncheckedException(e);
       } catch (InvocationTargetException e) {
@@ -318,6 +321,14 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
     // This becomes more relevant when task kills without container kills are allowed.
 
     // TODO TEZ-2336. Send a signal to containers indicating DAG completion.
+
+    // Inform all communicators of the dagCompletion.
+    for (int i = 0 ; i < taskCommunicators.length ; i++) {
+      ((TaskCommunicatorContextImpl)taskCommunicatorContexts[i]).dagCompleteStart(dag);
+      taskCommunicators[i].dagComplete(dag.getName());
+      ((TaskCommunicatorContextImpl)taskCommunicatorContexts[i]).dagCompleteEnd();
+    }
+
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/tez/blob/e5886f8e/tez-dag/src/main/java/org/apache/tez/dag/app/TaskCommunicatorContextImpl.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/TaskCommunicatorContextImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/TaskCommunicatorContextImpl.java
index 4cb0c93..790066f 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/TaskCommunicatorContextImpl.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/TaskCommunicatorContextImpl.java
@@ -17,6 +17,11 @@ package org.apache.tez.dag.app;
 import javax.annotation.Nullable;
 import java.io.IOException;
 import java.util.Set;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
 
 import com.google.common.base.Function;
 import com.google.common.base.Preconditions;
@@ -33,6 +38,7 @@ import org.apache.tez.dag.api.TezException;
 import org.apache.tez.dag.api.TezUncheckedException;
 import org.apache.tez.dag.api.event.VertexState;
 import org.apache.tez.dag.api.event.VertexStateUpdate;
+import org.apache.tez.dag.app.dag.DAG;
 import org.apache.tez.dag.app.dag.Vertex;
 import org.apache.tez.dag.app.dag.VertexStateUpdateListener;
 import org.apache.tez.dag.records.TezTaskAttemptID;
@@ -44,6 +50,10 @@ public class TaskCommunicatorContextImpl implements TaskCommunicatorContext, Ver
   private final AppContext context;
   private final TaskAttemptListenerImpTezDag taskAttemptListener;
   private final int taskCommunicatorIndex;
+  private final ReentrantReadWriteLock.ReadLock dagChangedReadLock;
+  private final ReentrantReadWriteLock.WriteLock dagChangedWriteLock;
+
+  private DAG dag;
 
   public TaskCommunicatorContextImpl(AppContext appContext,
                                      TaskAttemptListenerImpTezDag taskAttemptListener,
@@ -51,6 +61,10 @@ public class TaskCommunicatorContextImpl implements TaskCommunicatorContext, Ver
     this.context = appContext;
     this.taskAttemptListener = taskAttemptListener;
     this.taskCommunicatorIndex = taskCommunicatorIndex;
+
+    ReentrantReadWriteLock dagChangedLock = new ReentrantReadWriteLock();
+    dagChangedReadLock = dagChangedLock.readLock();
+    dagChangedWriteLock = dagChangedLock.writeLock();
   }
 
   @Override
@@ -111,18 +125,19 @@ public class TaskCommunicatorContextImpl implements TaskCommunicatorContext, Ver
   public void registerForVertexStateUpdates(String vertexName,
                                             @Nullable Set<VertexState> stateSet) {
     Preconditions.checkNotNull(vertexName, "VertexName cannot be null: " + vertexName);
-    context.getCurrentDAG().getStateChangeNotifier().registerForVertexUpdates(vertexName, stateSet, this);
+    getDag().getStateChangeNotifier().registerForVertexUpdates(vertexName, stateSet,
+        this);
   }
 
   @Override
   public String getCurretnDagName() {
-    return context.getCurrentDAG().getName();
+    return getDag().getName();
   }
 
   @Override
   public Iterable<String> getInputVertexNames(String vertexName) {
     Preconditions.checkNotNull(vertexName, "VertexName cannot be null: " + vertexName);
-    Vertex vertex = context.getCurrentDAG().getVertex(vertexName);
+    Vertex vertex = getDag().getVertex(vertexName);
     Set<Vertex> sources = vertex.getInputVertices().keySet();
     return Iterables.transform(sources, new Function<Vertex, String>() {
       @Override
@@ -135,31 +150,32 @@ public class TaskCommunicatorContextImpl implements TaskCommunicatorContext, Ver
   @Override
   public int getVertexTotalTaskCount(String vertexName) {
     Preconditions.checkArgument(vertexName != null, "VertexName must be specified");
-    return context.getCurrentDAG().getVertex(vertexName).getTotalTasks();
+    return getDag().getVertex(vertexName).getTotalTasks();
   }
 
   @Override
   public int getVertexCompletedTaskCount(String vertexName) {
     Preconditions.checkArgument(vertexName != null, "VertexName must be specified");
-    return context.getCurrentDAG().getVertex(vertexName).getCompletedTasks();
+    return getDag().getVertex(vertexName).getCompletedTasks();
   }
 
   @Override
   public int getVertexRunningTaskCount(String vertexName) {
     Preconditions.checkArgument(vertexName != null, "VertexName must be specified");
-    return context.getCurrentDAG().getVertex(vertexName).getRunningTasks();
+    return getDag().getVertex(vertexName).getRunningTasks();
   }
 
   @Override
   public long getFirstAttemptStartTime(String vertexName, int taskIndex) {
     Preconditions.checkArgument(vertexName != null, "VertexName must be specified");
     Preconditions.checkArgument(taskIndex >=0, "TaskIndex must be > 0");
-    return context.getCurrentDAG().getVertex(vertexName).getTask(taskIndex).getFirstAttemptStartTime();
+    return getDag().getVertex(vertexName).getTask(
+        taskIndex).getFirstAttemptStartTime();
   }
 
   @Override
   public long getDagStartTime() {
-    return context.getCurrentDAG().getStartTime();
+    return getDag().getStartTime();
   }
 
   @Override
@@ -171,4 +187,36 @@ public class TaskCommunicatorContextImpl implements TaskCommunicatorContext, Ver
       throw new TezUncheckedException(e);
     }
   }
+
+  private DAG getDag() {
+    dagChangedReadLock.lock();
+    try {
+      if (dag != null) {
+        return dag;
+      } else {
+        return context.getCurrentDAG();
+      }
+    } finally {
+      dagChangedReadLock.unlock();
+    }
+  }
+
+  @InterfaceAudience.Private
+  public void dagCompleteStart(DAG dag) {
+    dagChangedWriteLock.lock();
+    try {
+      this.dag = dag;
+    } finally {
+      dagChangedWriteLock.unlock();
+    }
+  }
+
+  public void dagCompleteEnd() {
+    dagChangedWriteLock.lock();
+    try {
+      this.dag = null;
+    } finally {
+      dagChangedWriteLock.unlock();
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/e5886f8e/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java
index 1417a3b..825a4d2 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java
@@ -258,6 +258,11 @@ public class TezTaskCommunicatorImpl extends TaskCommunicator {
     // Empty. Not registering, or expecting any updates.
   }
 
+  @Override
+  public void dagComplete(String dagName) {
+    // Nothing to do at the moment. Some of the TODOs from TaskAttemptListener apply here.
+  }
+
   protected String getTokenIdentifier() {
     return tokenIdentifier;
   }

http://git-wip-us.apache.org/repos/asf/tez/blob/e5886f8e/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncher.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncher.java b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncher.java
index 8a8498f..ea07a1d 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncher.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncher.java
@@ -26,7 +26,4 @@ import org.apache.tez.dag.app.rm.NMCommunicatorEvent;
 public interface ContainerLauncher
     extends EventHandler<NMCommunicatorEvent> {
 
-    void dagComplete(DAG dag);
-
-    void dagSubmitted();
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/e5886f8e/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherImpl.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherImpl.java
index 94889a1..a1eb2a7 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherImpl.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherImpl.java
@@ -30,7 +30,6 @@ import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 
 import org.apache.commons.lang.exception.ExceptionUtils;
-import org.apache.tez.dag.app.dag.DAG;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -111,17 +110,6 @@ public class ContainerLauncherImpl extends AbstractService implements
     }
   }
 
-  @Override
-  public void dagComplete(DAG dag) {
-    // Nothing required at the moment. Containers are shared across DAGs
-  }
-
-  @Override
-  public void dagSubmitted() {
-    // Nothing to do right now. Indicates that a new DAG has been submitted and
-    // the context has updated information.
-  }
-
   private static enum ContainerState {
     PREP, FAILED, RUNNING, DONE, KILLED_BEFORE_LAUNCH
   }

http://git-wip-us.apache.org/repos/asf/tez/blob/e5886f8e/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherRouter.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherRouter.java b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherRouter.java
index dd3571e..db145f4 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherRouter.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherRouter.java
@@ -27,6 +27,7 @@ import org.apache.tez.dag.api.TezConstants;
 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.app.dag.DAG;
 import org.apache.tez.dag.app.rm.NMCommunicatorEvent;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -128,6 +129,15 @@ public class ContainerLauncherRouter extends AbstractService
     }
   }
 
+  public void dagComplete(DAG dag) {
+    // Nothing required at the moment. Containers are shared across DAGs
+  }
+
+  public void dagSubmitted() {
+    // Nothing to do right now. Indicates that a new DAG has been submitted and
+    // the context has updated information.
+  }
+
 
   @Override
   public void handle(NMCommunicatorEvent event) {

http://git-wip-us.apache.org/repos/asf/tez/blob/e5886f8e/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/LocalContainerLauncher.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/LocalContainerLauncher.java b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/LocalContainerLauncher.java
index 18b2e35..305f8b3 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/LocalContainerLauncher.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/LocalContainerLauncher.java
@@ -44,7 +44,6 @@ import com.google.common.util.concurrent.ListeningExecutorService;
 import com.google.common.util.concurrent.MoreExecutors;
 import com.google.common.util.concurrent.ThreadFactoryBuilder;
 
-import org.apache.tez.dag.app.dag.DAG;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -169,14 +168,6 @@ public class LocalContainerLauncher extends AbstractService implements
     callbackExecutor.shutdownNow();
   }
 
-  @Override
-  public void dagComplete(DAG dag) {
-  }
-
-  @Override
-  public void dagSubmitted() {
-  }
-
   // Thread to monitor the queue of incoming NMCommunicator events
   private class TezSubTaskRunner implements Runnable {
     @Override

http://git-wip-us.apache.org/repos/asf/tez/blob/e5886f8e/tez-dag/src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java b/tez-dag/src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java
index d40b78d..2a38096 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java
@@ -34,10 +34,8 @@ import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicLong;
 
-import org.apache.tez.dag.app.dag.DAG;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import com.google.common.annotations.VisibleForTesting;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.service.AbstractService;
@@ -54,7 +52,6 @@ import org.apache.tez.common.counters.TezCounters;
 import org.apache.tez.dag.api.TaskHeartbeatRequest;
 import org.apache.tez.dag.api.TaskHeartbeatResponse;
 import org.apache.tez.dag.api.TezConfiguration;
-import org.apache.tez.dag.api.TaskCommunicator;
 import org.apache.tez.dag.api.TezUncheckedException;
 import org.apache.tez.dag.app.launcher.ContainerLauncher;
 import org.apache.tez.dag.app.launcher.ContainerLauncherRouter;
@@ -146,14 +143,6 @@ public class MockDAGAppMaster extends DAGAppMaster {
       this.goFlag = goFlag;
     }
 
-    @Override
-    public void dagComplete(DAG dag) {
-    }
-
-    @Override
-    public void dagSubmitted() {
-    }
-
     public class ContainerData {
       ContainerId cId;
       TezTaskAttemptID taId;

http://git-wip-us.apache.org/repos/asf/tez/blob/e5886f8e/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/rm/TezTestServiceTaskSchedulerService.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/rm/TezTestServiceTaskSchedulerService.java b/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/rm/TezTestServiceTaskSchedulerService.java
index 50dfb24..073cb50 100644
--- a/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/rm/TezTestServiceTaskSchedulerService.java
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/rm/TezTestServiceTaskSchedulerService.java
@@ -158,7 +158,7 @@ public class TezTestServiceTaskSchedulerService extends TaskSchedulerService {
   }
 
   @Override
-  public void resetMatchLocalityForAllHeldContainers() {
+  public void dagComplete() {
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/tez/blob/e5886f8e/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/taskcomm/TezTestServiceTaskCommunicatorImpl.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/taskcomm/TezTestServiceTaskCommunicatorImpl.java b/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/taskcomm/TezTestServiceTaskCommunicatorImpl.java
index ef983c2..cf28b11 100644
--- a/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/taskcomm/TezTestServiceTaskCommunicatorImpl.java
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/taskcomm/TezTestServiceTaskCommunicatorImpl.java
@@ -150,7 +150,7 @@ public class TezTestServiceTaskCommunicatorImpl extends TezTaskCommunicatorImpl
               t = se.getCause();
             }
             if (t instanceof RemoteException) {
-              RemoteException re = (RemoteException)t;
+              RemoteException re = (RemoteException) t;
               String message = re.toString();
               if (message.contains(RejectedExecutionException.class.getName())) {
                 getTaskCommunicatorContext().taskKilled(taskSpec.getTaskAttemptID(),

http://git-wip-us.apache.org/repos/asf/tez/blob/e5886f8e/tez-ext-service-tests/src/test/java/org/apache/tez/tests/TestExternalTezServices.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/tests/TestExternalTezServices.java b/tez-ext-service-tests/src/test/java/org/apache/tez/tests/TestExternalTezServices.java
index 4d0a610..45c70f1 100644
--- a/tez-ext-service-tests/src/test/java/org/apache/tez/tests/TestExternalTezServices.java
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/tests/TestExternalTezServices.java
@@ -39,6 +39,7 @@ import org.apache.tez.dag.app.taskcomm.TezTestServiceTaskCommunicatorImpl;
 import org.apache.tez.examples.HashJoinExample;
 import org.apache.tez.examples.JoinDataGen;
 import org.apache.tez.examples.JoinValidateConfigured;
+import org.apache.tez.runtime.library.api.TezRuntimeConfiguration;
 import org.apache.tez.runtime.library.processor.SleepProcessor;
 import org.apache.tez.service.MiniTezTestServiceCluster;
 import org.apache.tez.service.impl.ContainerRunnerImpl;
@@ -124,6 +125,7 @@ public class TestExternalTezServices {
     remoteFs.mkdirs(stagingDirPath);
     // This is currently configured to push tasks into the Service, and then use the standard RPC
     confForJobs.set(TezConfiguration.TEZ_AM_STAGING_DIR, stagingDirPath.toString());
+    confForJobs.setBoolean(TezRuntimeConfiguration.TEZ_RUNTIME_OPTIMIZE_LOCAL_FETCH, false);
 
     confForJobs.setStrings(TezConfiguration.TEZ_AM_TASK_SCHEDULERS,
         TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT,


[21/50] [abbrv] tez git commit: TEZ-1752. Inputs / Outputs in the Runtime library should be interruptable (rbalamohan)

Posted by ss...@apache.org.
TEZ-1752. Inputs / Outputs in the Runtime library should be interruptable (rbalamohan)


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

Branch: refs/heads/TEZ-2003
Commit: 146ab0702a25cda7020de936e270e291ca567e3c
Parents: 8c44f24
Author: Rajesh Balamohan <rb...@apache.org>
Authored: Wed May 6 03:43:49 2015 +0530
Committer: Rajesh Balamohan <rb...@apache.org>
Committed: Wed May 6 03:43:49 2015 +0530

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 .../runtime/api/AbstractLogicalIOProcessor.java |   4 +
 .../api/ProcessorFrameworkInterface.java        |  11 +-
 .../library/api/TezRuntimeConfiguration.java    |  11 +
 .../common/readers/UnorderedKVReader.java       |   1 +
 .../runtime/library/common/shuffle/Fetcher.java |   1 +
 .../common/shuffle/impl/ShuffleManager.java     |   5 +
 .../orderedgrouped/FetcherOrderedGrouped.java   |   7 +-
 .../shuffle/orderedgrouped/MergeManager.java    | 124 +++++++--
 .../shuffle/orderedgrouped/MergeThread.java     |  18 +-
 .../common/shuffle/orderedgrouped/Shuffle.java  |  16 +-
 .../orderedgrouped/ShuffleScheduler.java        |   1 +
 .../common/sort/impl/ExternalSorter.java        |  38 +++
 .../common/sort/impl/PipelinedSorter.java       | 261 +++++++++++--------
 .../library/common/sort/impl/TezMerger.java     |  31 ++-
 .../common/sort/impl/dflt/DefaultSorter.java    |  70 +++--
 .../library/input/OrderedGroupedKVInput.java    |   1 +
 .../runtime/library/input/UnorderedKVInput.java |   1 +
 .../output/OrderedPartitionedKVOutput.java      |   1 +
 .../library/output/UnorderedKVOutput.java       |   1 +
 .../output/UnorderedPartitionedKVOutput.java    |   1 +
 .../library/common/TestValuesIterator.java      |  20 +-
 .../orderedgrouped/TestMergeManager.java        |  87 +++++--
 .../library/common/sort/impl/TestTezMerger.java |   3 +-
 24 files changed, 518 insertions(+), 197 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/146ab070/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index d7a1e1f..7ba8021 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -9,6 +9,7 @@ INCOMPATIBLE CHANGES
   TEZ-1993. Implement a pluggable InputSizeEstimator for grouping fairly
 
 ALL CHANGES:
+  TEZ-1752. Inputs / Outputs in the Runtime library should be interruptable.
   TEZ-2401. Tez UI: All-dag page has duration keep counting for KILLED dag.
   TEZ-2392. Have all readers throw an Exception on incorrect next() usage.
   TEZ-2408. TestTaskAttempt fails to compile against hadoop-2.4 and hadoop-2.2.

http://git-wip-us.apache.org/repos/asf/tez/blob/146ab070/tez-api/src/main/java/org/apache/tez/runtime/api/AbstractLogicalIOProcessor.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/runtime/api/AbstractLogicalIOProcessor.java b/tez-api/src/main/java/org/apache/tez/runtime/api/AbstractLogicalIOProcessor.java
index 7714321..5a4cbe8 100644
--- a/tez-api/src/main/java/org/apache/tez/runtime/api/AbstractLogicalIOProcessor.java
+++ b/tez-api/src/main/java/org/apache/tez/runtime/api/AbstractLogicalIOProcessor.java
@@ -49,4 +49,8 @@ public abstract class AbstractLogicalIOProcessor implements LogicalIOProcessor,
     return context;
   }
 
+  @Override
+  public void abort() {
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/146ab070/tez-api/src/main/java/org/apache/tez/runtime/api/ProcessorFrameworkInterface.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/runtime/api/ProcessorFrameworkInterface.java b/tez-api/src/main/java/org/apache/tez/runtime/api/ProcessorFrameworkInterface.java
index f0ba9c9..89d4e3c 100644
--- a/tez-api/src/main/java/org/apache/tez/runtime/api/ProcessorFrameworkInterface.java
+++ b/tez-api/src/main/java/org/apache/tez/runtime/api/ProcessorFrameworkInterface.java
@@ -18,9 +18,11 @@
 
 package org.apache.tez.runtime.api;
 
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+
 import java.util.List;
 
-import org.apache.hadoop.classification.InterfaceAudience.Public;
 
 /**
  * Represents the Tez framework part of an {@link org.apache.tez.runtime.api.Processor}.
@@ -56,4 +58,11 @@ public interface ProcessorFrameworkInterface {
    *           if an error occurs
    */
   public void close() throws Exception;
+
+  /**
+   * Indicates <code>Processor</code> to abort. Cleanup can be done.
+   *
+   */
+  @Unstable
+  public void abort();
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/146ab070/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/api/TezRuntimeConfiguration.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/api/TezRuntimeConfiguration.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/api/TezRuntimeConfiguration.java
index a818de8..3d9a701 100644
--- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/api/TezRuntimeConfiguration.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/api/TezRuntimeConfiguration.java
@@ -311,6 +311,16 @@ public class TezRuntimeConfiguration {
    */
   public static final boolean TEZ_RUNTIME_OPTIMIZE_SHARED_FETCH_DEFAULT = false;
 
+  /**
+   * Used only for internal testing. Strictly not recommended to be used elsewhere. This
+   * parameter could be changed/dropped later.
+   */
+  @Unstable
+  @Private
+  public static final String TEZ_RUNTIME_CLEANUP_FILES_ON_INTERRUPT = TEZ_RUNTIME_PREFIX
+      + "cleanup.files.on.interrupt";
+  public static final boolean TEZ_RUNTIME_CLEANUP_FILES_ON_INTERRUPT_DEFAULT = false;
+
   // TODO TEZ-1233 - allow this property to be set per vertex
   // TODO TEZ-1231 - move these properties out since they are not relevant for Inputs / Outputs
 
@@ -374,6 +384,7 @@ public class TezRuntimeConfiguration {
     tezRuntimeKeys.add(TEZ_RUNTIME_OPTIMIZE_SHARED_FETCH);
     tezRuntimeKeys.add(TEZ_RUNTIME_CONVERT_USER_PAYLOAD_TO_HISTORY_TEXT);
     tezRuntimeKeys.add(TEZ_RUNTIME_SORTER_CLASS);
+    tezRuntimeKeys.add(TEZ_RUNTIME_CLEANUP_FILES_ON_INTERRUPT);
 
     defaultConf.addResource("core-default.xml");
     defaultConf.addResource("core-site.xml");

http://git-wip-us.apache.org/repos/asf/tez/blob/146ab070/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/readers/UnorderedKVReader.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/readers/UnorderedKVReader.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/readers/UnorderedKVReader.java
index b14a461..fc2e312 100644
--- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/readers/UnorderedKVReader.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/readers/UnorderedKVReader.java
@@ -184,6 +184,7 @@ public class UnorderedKVReader<K, V> extends KeyValueReader {
       currentFetchedInput = shuffleManager.getNextInput();
     } catch (InterruptedException e) {
       LOG.warn("Interrupted while waiting for next available input", e);
+      Thread.currentThread().interrupt();
       throw new IOException(e);
     }
     if (currentFetchedInput == null) {

http://git-wip-us.apache.org/repos/asf/tez/blob/146ab070/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/Fetcher.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/Fetcher.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/Fetcher.java
index 3154943..48fe0f2 100644
--- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/Fetcher.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/Fetcher.java
@@ -376,6 +376,7 @@ public class Fetcher extends CallableWithNdc<FetchResult> {
       // fall back to HTTP fetch below
       LOG.warn("Double locking detected for " + host);
     } catch (InterruptedException sleepInterrupted) {
+      Thread.currentThread().interrupt();
       // fall back to HTTP fetch below
       LOG.warn("Lock was interrupted for " + host);
     } finally {

http://git-wip-us.apache.org/repos/asf/tez/blob/146ab070/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleManager.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleManager.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleManager.java
index 749143a..d47e652 100644
--- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleManager.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleManager.java
@@ -748,6 +748,11 @@ public class ShuffleManager implements FetcherCallback {
   /////////////////// End of Methods from FetcherCallbackHandler
 
   public void shutdown() throws InterruptedException {
+    if (Thread.currentThread().isInterrupted()) {
+      //TODO: need to cleanup all FetchedInput (DiskFetchedInput, LocalDisFetchedInput), lockFile
+      //As of now relying on job cleanup (when all directories would be cleared)
+      LOG.info("Thread interrupted. Need to cleanup the local dirs");
+    }
     if (!isShutdown.getAndSet(true)) {
       // Shut down any pending fetchers
       LOG.info("Shutting down pending fetchers on source" + srcNameTrimmed + ": "

http://git-wip-us.apache.org/repos/asf/tez/blob/146ab070/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/FetcherOrderedGrouped.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/FetcherOrderedGrouped.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/FetcherOrderedGrouped.java
index 8d20aa7..fbaabff 100644
--- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/FetcherOrderedGrouped.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/FetcherOrderedGrouped.java
@@ -177,6 +177,9 @@ class FetcherOrderedGrouped extends Thread {
         }
       }
     } catch (InterruptedException ie) {
+      //TODO: might not be respected when fetcher is in progress / server is busy.  TEZ-711
+      //Set the status back
+      Thread.currentThread().interrupt();
       return;
     } catch (Throwable t) {
       shuffle.reportException(t);
@@ -191,7 +194,9 @@ class FetcherOrderedGrouped extends Thread {
     try {
       join(5000);
     } catch (InterruptedException ie) {
-      LOG.warn("Got interrupt while joining " + getName(), ie);
+      //Reset the status
+      Thread.currentThread().interrupt();
+      LOG.warn("Got interrupt while joining " + getName());
     }
   }
 

http://git-wip-us.apache.org/repos/asf/tez/blob/146ab070/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/MergeManager.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/MergeManager.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/MergeManager.java
index 2e6ebd9..5a35f2f 100644
--- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/MergeManager.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/MergeManager.java
@@ -19,6 +19,7 @@ package org.apache.tez.runtime.library.common.shuffle.orderedgrouped;
 
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.Collection;
 import java.util.Collections;
 import java.util.Comparator;
 import java.util.List;
@@ -137,6 +138,8 @@ public class MergeManager {
 
   private AtomicInteger mergeFileSequenceId = new AtomicInteger(0);
 
+  private final boolean cleanup;
+
   /**
    * Construct the MergeManager. Must call start before it becomes usable.
    */
@@ -174,6 +177,9 @@ public class MergeManager {
     this.additionalBytesWritten = inputContext.getCounters().findCounter(TaskCounter.ADDITIONAL_SPILLS_BYTES_WRITTEN);
     this.additionalBytesRead = inputContext.getCounters().findCounter(TaskCounter.ADDITIONAL_SPILLS_BYTES_READ);
 
+    this.cleanup = conf.getBoolean(TezRuntimeConfiguration.TEZ_RUNTIME_CLEANUP_FILES_ON_INTERRUPT,
+        TezRuntimeConfiguration.TEZ_RUNTIME_CLEANUP_FILES_ON_INTERRUPT_DEFAULT);
+
     this.codec = codec;
     this.ifileReadAhead = ifileReadAheadEnabled;
     if (this.ifileReadAhead) {
@@ -514,27 +520,61 @@ public class MergeManager {
   public boolean isMergeComplete() {
     return finalMergeComplete;
   }
-  
+
   public TezRawKeyValueIterator close() throws Throwable {
     // Wait for on-going merges to complete
-    if (memToMemMerger != null) { 
+    if (memToMemMerger != null) {
       memToMemMerger.close();
     }
     inMemoryMerger.close();
     onDiskMerger.close();
-    
-    List<MapOutput> memory = 
+
+    List<MapOutput> memory =
       new ArrayList<MapOutput>(inMemoryMergedMapOutputs);
     inMemoryMergedMapOutputs.clear();
     memory.addAll(inMemoryMapOutputs);
     inMemoryMapOutputs.clear();
     List<FileChunk> disk = new ArrayList<FileChunk>(onDiskMapOutputs);
     onDiskMapOutputs.clear();
-    TezRawKeyValueIterator kvIter = finalMerge(conf, rfs, memory, disk);
-    this.finalMergeComplete = true;
-    return kvIter;
+    try {
+      TezRawKeyValueIterator kvIter = finalMerge(conf, rfs, memory, disk);
+      this.finalMergeComplete = true;
+      return kvIter;
+    } catch(InterruptedException e) {
+      //Cleanup the disk segments
+      if (cleanup) {
+        cleanup(localFS, disk);
+        cleanup(localFS, onDiskMapOutputs);
+      }
+      Thread.currentThread().interrupt(); //reset interrupt status
+      throw e;
+    }
+  }
+
+
+  static void cleanup(FileSystem fs, Collection<FileChunk> fileChunkList) {
+    for (FileChunk fileChunk : fileChunkList) {
+      cleanup(fs, fileChunk.getPath());
+    }
   }
-   
+
+  static void cleanup(FileSystem fs, Path path) {
+    if (path == null) {
+      return;
+    }
+
+    try {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Deleting " + path);
+      }
+      fs.delete(path, true);
+    } catch (IOException e) {
+      LOG.info("Error in deleting " + path);
+    }
+  }
+
+
+
   void runCombineProcessor(TezRawKeyValueIterator kvIter, Writer writer)
       throws IOException, InterruptedException {
     combiner.combine(kvIter, writer);
@@ -555,7 +595,7 @@ public class MergeManager {
     }
 
     @Override
-    public void merge(List<MapOutput> inputs) throws IOException {
+    public void merge(List<MapOutput> inputs) throws IOException, InterruptedException {
       if (inputs == null || inputs.size() == 0) {
         return;
       }
@@ -597,13 +637,28 @@ public class MergeManager {
       // Note the output of the merge
       closeInMemoryMergedFile(mergedMapOutputs);
     }
+
+    @Override
+    public void cleanup(List<MapOutput> inputs, boolean deleteData) throws IOException,
+        InterruptedException {
+      //No OP
+    }
   }
   
   /**
    * Merges multiple in-memory segment to a disk segment
    */
   private class InMemoryMerger extends MergeThread<MapOutput> {
-    
+
+    @VisibleForTesting
+    volatile InputAttemptIdentifier srcTaskIdentifier;
+
+    @VisibleForTesting
+    volatile Path outputPath;
+
+    @VisibleForTesting
+    volatile Path tmpDir;
+
     public InMemoryMerger(MergeManager manager) {
       super(manager, Integer.MAX_VALUE, exceptionReporter);
       setName("MemtoDiskMerger [" + TezUtilsInternal
@@ -628,7 +683,7 @@ public class MergeManager {
       //in the merge method)
 
       //figure out the mapId 
-      InputAttemptIdentifier srcTaskIdentifier = inputs.get(0).getAttemptIdentifier();
+      srcTaskIdentifier = inputs.get(0).getAttemptIdentifier();
 
       List<Segment> inMemorySegments = new ArrayList<Segment>();
       long mergeOutputSize = 
@@ -639,7 +694,7 @@ public class MergeManager {
       
       // All disk writes done by this merge are overhead - due to the lac of
       // adequate memory to keep all segments in memory.
-      Path outputPath = mapOutputFile.getInputFileForWrite(
+      outputPath = mapOutputFile.getInputFileForWrite(
           srcTaskIdentifier.getInputIdentifier().getInputIndex(), srcTaskIdentifier.getSpillEventId(),
           mergeOutputSize).suffix(Constants.MERGED_OUTPUT_PREFIX);
       LOG.info("Patch..InMemoryMerger outputPath: " + outputPath);
@@ -657,13 +712,13 @@ public class MergeManager {
         LOG.info("Initiating in-memory merge with " + noInMemorySegments + 
             " segments...");
 
+        tmpDir = new Path(inputContext.getUniqueIdentifier());
         // Nothing actually materialized to disk - controlled by setting sort-factor to #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),
+            tmpDir, (RawComparator)ConfigUtils.getIntermediateInputKeyComparator(conf),
             nullProgressable, spilledRecordsCounter, null, additionalBytesRead, null);
         // spilledRecordsCounter is tracking the number of keys that will be
         // read from each of the segments being merged - which is essentially
@@ -700,6 +755,18 @@ public class MergeManager {
       closeOnDiskFile(new FileChunk(outputPath, 0, outFileLen));
     }
 
+    @Override
+    public void cleanup(List<MapOutput> inputs, boolean deleteData)
+        throws IOException, InterruptedException {
+      if (deleteData) {
+        //Additional check at task level
+        if (cleanup) {
+          LOG.info("Try deleting stale data");
+          MergeManager.cleanup(localFS, outputPath);
+          MergeManager.cleanup(localFS, tmpDir);
+        }
+      }
+    }
   }
 
   /**
@@ -708,6 +775,11 @@ public class MergeManager {
   @VisibleForTesting
   class OnDiskMerger extends MergeThread<FileChunk> {
 
+    @VisibleForTesting
+    volatile Path outputPath;
+    @VisibleForTesting
+    volatile Path tmpDir;
+
     public OnDiskMerger(MergeManager manager) {
       super(manager, ioSortFactor, exceptionReporter);
       setName("DiskToDiskMerger [" + TezUtilsInternal
@@ -716,7 +788,7 @@ public class MergeManager {
     }
     
     @Override
-    public void merge(List<FileChunk> inputs) throws IOException {
+    public void merge(List<FileChunk> inputs) throws IOException, InterruptedException {
       // sanity check
       if (inputs == null || inputs.isEmpty()) {
         LOG.info("No ondisk files to merge...");
@@ -768,7 +840,7 @@ public class MergeManager {
 
       // namePart includes the suffix of the file. We need to remove it.
       namePart = FilenameUtils.removeExtension(namePart);
-      Path outputPath = localDirAllocator.getLocalPathForWrite(namePart, approxOutputSize, conf);
+      outputPath = localDirAllocator.getLocalPathForWrite(namePart, approxOutputSize, conf);
       outputPath = outputPath.suffix(Constants.MERGED_OUTPUT_PREFIX + mergeFileSequenceId.getAndIncrement());
 
       Writer writer =
@@ -776,7 +848,7 @@ public class MergeManager {
                         (Class)ConfigUtils.getIntermediateInputKeyClass(conf), 
                         (Class)ConfigUtils.getIntermediateInputValueClass(conf),
                         codec, null, null);
-      Path tmpDir = new Path(inputContext.getUniqueIdentifier());
+      tmpDir = new Path(inputContext.getUniqueIdentifier());
       try {
         TezRawKeyValueIterator iter = TezMerger.merge(conf, rfs,
             (Class)ConfigUtils.getIntermediateInputKeyClass(conf),
@@ -808,6 +880,20 @@ public class MergeManager {
           " Local output file is " + outputPath + " of size " +
           outputLen);
     }
+
+    @Override
+    public void cleanup(List<FileChunk> inputs, boolean deleteData) throws IOException,
+        InterruptedException {
+      if (deleteData) {
+        //Additional check at task level
+        if (cleanup) {
+          LOG.info("Try deleting stale data");
+          MergeManager.cleanup(localFS, inputs);
+          MergeManager.cleanup(localFS, outputPath);
+          MergeManager.cleanup(localFS, tmpDir);
+        }
+      }
+    }
   }
   
   private long createInMemorySegments(List<MapOutput> inMemoryMapOutputs,
@@ -821,7 +907,7 @@ public class MergeManager {
     for (MapOutput mo : inMemoryMapOutputs) {
       fullSize += mo.getMemory().length;
     }
-    while(fullSize > leaveBytes) {
+    while((fullSize > leaveBytes) && !Thread.currentThread().isInterrupted()) {
       MapOutput mo = inMemoryMapOutputs.remove(0);
       byte[] data = mo.getMemory();
       long size = data.length;
@@ -878,7 +964,7 @@ public class MergeManager {
   private TezRawKeyValueIterator finalMerge(Configuration job, FileSystem fs,
                                        List<MapOutput> inMemoryMapOutputs,
                                        List<FileChunk> onDiskMapOutputs
-                                       ) throws IOException {
+                                       ) throws IOException, InterruptedException {
     LOG.info("finalMerge called with " + 
              inMemoryMapOutputs.size() + " in-memory map-outputs and " + 
              onDiskMapOutputs.size() + " on-disk map-outputs");

http://git-wip-us.apache.org/repos/asf/tez/blob/146ab070/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/MergeThread.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/MergeThread.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/MergeThread.java
index d4faf51..52b4c5b 100644
--- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/MergeThread.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/MergeThread.java
@@ -46,8 +46,18 @@ abstract class MergeThread<T> extends Thread {
   
   public synchronized void close() throws InterruptedException {
     closed = true;
-    waitForMerge();
-    interrupt();
+    if (!Thread.currentThread().isInterrupted()) {
+      waitForMerge();
+      interrupt();
+    } else {
+      try {
+        interrupt();
+        cleanup(inputs, Thread.currentThread().isInterrupted());
+      } catch (IOException e) {
+        //ignore
+        LOG.warn("Error cleaning up", e);
+      }
+    }
   }
 
   public synchronized boolean isInProgress() {
@@ -89,6 +99,7 @@ abstract class MergeThread<T> extends Thread {
         merge(inputs);
       } catch (InterruptedException ie) {
         // Meant to handle a shutdown of the entire fetch/merge process
+        Thread.currentThread().interrupt();
         return;
       } catch(Throwable t) {
         reporter.reportException(t);
@@ -106,4 +117,7 @@ abstract class MergeThread<T> extends Thread {
 
   public abstract void merge(List<T> inputs) 
       throws IOException, InterruptedException;
+
+  public abstract void cleanup(List<T> inputs, boolean deleteData)
+      throws IOException, InterruptedException;
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/146ab070/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/Shuffle.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/Shuffle.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/Shuffle.java
index f98aa3a..442f032 100644
--- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/Shuffle.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/Shuffle.java
@@ -357,6 +357,7 @@ public class Shuffle implements ExceptionReporter {
       shufflePhaseTime.setValue(System.currentTimeMillis() - startTime);
 
       // Stop the map-output fetcher threads
+      LOG.info("Cleaning up fetchers");
       cleanupFetchers(false);
       
       // stop the scheduler
@@ -393,8 +394,7 @@ public class Shuffle implements ExceptionReporter {
         for (FetcherOrderedGrouped fetcher : fetchers) {
           try {
             fetcher.shutDown();
-            LOG.info("Shutdown.." + fetcher.getName() + ", status:" + fetcher.isAlive() + ", "
-                + "isInterrupted:" + fetcher.isInterrupted());
+            LOG.info("Shutdown.." + fetcher.getName());
           } catch (InterruptedException e) {
             if (ignoreErrors) {
               LOG.info("Interrupted while shutting down fetchers. Ignoring.");
@@ -425,6 +425,8 @@ public class Shuffle implements ExceptionReporter {
         scheduler.close();
       } catch (InterruptedException e) {
         if (ignoreErrors) {
+          //Reset the status
+          Thread.currentThread().interrupt();
           LOG.info("Interrupted while attempting to close the scheduler during cleanup. Ignoring");
         } else {
           throw e;
@@ -437,6 +439,14 @@ public class Shuffle implements ExceptionReporter {
     if (!mergerClosed.getAndSet(true)) {
       try {
         merger.close();
+      } catch (InterruptedException e) {
+        if (ignoreErrors) {
+          //Reset the status
+          Thread.currentThread().interrupt();
+          LOG.info("Interrupted while attempting to close the merger during cleanup. Ignoring");
+        } else {
+          throw e;
+        }
       } catch (Throwable e) {
         if (ignoreErrors) {
           LOG.info("Exception while trying to shutdown merger, Ignoring", e);
@@ -493,7 +503,7 @@ public class Shuffle implements ExceptionReporter {
     @Override
     public void onFailure(Throwable t) {
       if (isShutDown.get()) {
-        LOG.info("Already shutdown. Ignoring error: ",  t);
+        LOG.info("Already shutdown. Ignoring error");
       } else {
         LOG.error("ShuffleRunner failed with error", t);
         inputContext.fatalError(t, "Shuffle Runner Failed");

http://git-wip-us.apache.org/repos/asf/tez/blob/146ab070/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/ShuffleScheduler.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/ShuffleScheduler.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/ShuffleScheduler.java
index a3d79ae..c54b005 100644
--- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/ShuffleScheduler.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/ShuffleScheduler.java
@@ -765,6 +765,7 @@ class ShuffleScheduler {
           }
         }
       } catch (InterruptedException ie) {
+        Thread.currentThread().interrupt();
         // This handles shutdown of the entire fetch / merge process.
         return;
       } catch (Throwable t) {

http://git-wip-us.apache.org/repos/asf/tez/blob/146ab070/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
index c0445c9..ca4d889 100644
--- 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
@@ -115,6 +115,8 @@ public abstract class ExternalSorter {
   protected Path finalIndexFile;
   protected int numSpills;
 
+  protected final boolean cleanup;
+
   // Counters
   // MR compatilbity layer needs to rename counters back to what MR requries.
 
@@ -148,6 +150,9 @@ public abstract class ExternalSorter {
     this.conf = conf;
     this.partitions = numOutputs;
 
+    cleanup = conf.getBoolean(TezRuntimeConfiguration.TEZ_RUNTIME_CLEANUP_FILES_ON_INTERRUPT,
+        TezRuntimeConfiguration.TEZ_RUNTIME_CLEANUP_FILES_ON_INTERRUPT_DEFAULT);
+
     rfs = ((LocalFileSystem)FileSystem.getLocal(this.conf)).getRaw();
 
     LOG.info("Initial Mem : " + initialMemoryAvailable + ", assignedMb=" + ((initialMemoryAvailable >> 20)));
@@ -261,6 +266,7 @@ public abstract class ExternalSorter {
     try {
       combiner.combine(kvIter, writer);
     } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
       throw new IOException(e);
     }
   }
@@ -314,4 +320,36 @@ public abstract class ExternalSorter {
   public int getNumSpills() {
     return numSpills;
   }
+
+  protected synchronized void cleanup() throws IOException {
+    if (!cleanup) {
+      return;
+    }
+    cleanup(spillFilePaths);
+    cleanup(spillFileIndexPaths);
+    //TODO: What if when same volume rename happens (have to rely on job completion cleanup)
+    cleanup(finalOutputFile);
+    cleanup(finalIndexFile);
+  }
+
+  protected synchronized void cleanup(Path path) {
+    if (path == null || !cleanup) {
+      return;
+    }
+    try {
+      LOG.info("Deleting " + path);
+      rfs.delete(path, true);
+    } catch(IOException ioe) {
+      LOG.warn("Error in deleting "  + path);
+    }
+  }
+
+  protected synchronized void cleanup(Map<Integer, Path> spillMap) {
+    if (!cleanup) {
+      return;
+    }
+    for(Map.Entry<Integer, Path> entry : spillMap.entrySet()) {
+      cleanup(entry.getValue());
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/146ab070/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
index 661f54c..030440e 100644
--- 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
@@ -33,7 +33,6 @@ import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.Future;
 
-
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 import com.google.common.base.Stopwatch;
@@ -356,6 +355,9 @@ public class PipelinedSorter extends ExternalSorter {
       merger.ready(); // wait for all the future results from sort threads
       LOG.info("Spilling to " + filename.toString());
       for (int i = 0; i < partitions; ++i) {
+        if (isThreadInterrupted()) {
+          return;
+        }
         TezRawKeyValueIterator kvIter = merger.filter(i);
         //write merged output to disk
         long segmentStart = out.getPos();
@@ -391,147 +393,182 @@ public class PipelinedSorter extends ExternalSorter {
       ++numSpills;
     } catch(InterruptedException ie) {
       // TODO:the combiner has been interrupted
+      Thread.currentThread().interrupt();
     } finally {
       out.close();
     }
   }
 
+
+
+
+
+  private boolean isThreadInterrupted() throws IOException {
+    if (Thread.currentThread().isInterrupted()) {
+      if (cleanup) {
+        cleanup();
+      }
+      sortmaster.shutdownNow();
+      LOG.info("Thread interrupted, cleaned up stale data, sorter threads shutdown=" + sortmaster
+          .isShutdown() + ", terminated=" + sortmaster.isTerminated());
+      return true;
+    }
+    return false;
+  }
+
   @Override
   public void flush() throws IOException {
     final String uniqueIdentifier = outputContext.getUniqueIdentifier();
 
-    LOG.info("Starting flush of map output");
-    span.end();
-    merger.add(span.sort(sorter));
-    spill();
-    sortmaster.shutdown();
+    /**
+     * Possible that the thread got interrupted when flush was happening or when the flush was
+     * never invoked. As a part of cleanup activity in TezTaskRunner, it would invoke close()
+     * on all I/O. At that time, this is safe to cleanup
+     */
+    if (isThreadInterrupted()) {
+      return;
+    }
+
+    try {
+      LOG.info("Starting flush of map output");
+      span.end();
+      merger.add(span.sort(sorter));
+      spill();
+      sortmaster.shutdown();
 
-    //safe to clean up
-    bufferList.clear();
+      //safe to clean up
+      bufferList.clear();
 
-    numAdditionalSpills.increment(numSpills - 1);
+      numAdditionalSpills.increment(numSpills - 1);
 
-    if (!finalMergeEnabled) {
-      //Generate events for all spills
-      List<Event> events = Lists.newLinkedList();
+      if (!finalMergeEnabled) {
+        //Generate events for all spills
+        List<Event> events = Lists.newLinkedList();
 
-      //For pipelined shuffle, previous events are already sent. Just generate the last event alone
-      int startIndex = (pipelinedShuffle) ? (numSpills - 1) : 0;
-      int endIndex = numSpills;
+        //For pipelined shuffle, previous events are already sent. Just generate the last event alone
+        int startIndex = (pipelinedShuffle) ? (numSpills - 1) : 0;
+        int endIndex = numSpills;
 
-      for (int i = startIndex; i < endIndex; i++) {
-        boolean isLastEvent = (i == numSpills - 1);
+        for (int i = startIndex; i < endIndex; i++) {
+          boolean isLastEvent = (i == numSpills - 1);
 
-        String pathComponent = (outputContext.getUniqueIdentifier() + "_" + i);
-        ShuffleUtils.generateEventOnSpill(events, finalMergeEnabled, isLastEvent,
-            outputContext, i, indexCacheList.get(i), partitions,
-            sendEmptyPartitionDetails, pathComponent);
-        LOG.info("Adding spill event for spill (final update=" + isLastEvent + "), spillId=" + i);
+          String pathComponent = (outputContext.getUniqueIdentifier() + "_" + i);
+          ShuffleUtils.generateEventOnSpill(events, finalMergeEnabled, isLastEvent,
+              outputContext, i, indexCacheList.get(i), partitions,
+              sendEmptyPartitionDetails, pathComponent);
+          LOG.info("Adding spill event for spill (final update=" + isLastEvent + "), spillId=" + i);
+        }
+        outputContext.sendEvents(events);
+        //No need to generate final merge
+        return;
       }
-      outputContext.sendEvents(events);
-      //No need to generate final merge
-      return;
-    }
 
-    //In case final merge is required, the following code path is executed.
-    if(numSpills == 1) {
-      // someday be able to pass this directly to shuffle
-      // without writing to disk
-      final Path filename = spillFilePaths.get(0);
-      final Path indexFilename = spillFileIndexPaths.get(0);
-      finalOutputFile = mapOutputFile.getOutputFileForWriteInVolume(filename);
-      finalIndexFile = mapOutputFile.getOutputIndexFileForWriteInVolume(indexFilename);
-
-      sameVolRename(filename, finalOutputFile);
-      sameVolRename(indexFilename, finalIndexFile);
-      if (LOG.isInfoEnabled()) {
-        LOG.info("numSpills=" + numSpills + ", finalOutputFile=" + finalOutputFile + ", "
-            + "finalIndexFile=" + finalIndexFile + ", filename=" + filename + ", indexFilename=" +
-            indexFilename);
+      //In case final merge is required, the following code path is executed.
+      if (numSpills == 1) {
+        // someday be able to pass this directly to shuffle
+        // without writing to disk
+        final Path filename = spillFilePaths.get(0);
+        final Path indexFilename = spillFileIndexPaths.get(0);
+        finalOutputFile = mapOutputFile.getOutputFileForWriteInVolume(filename);
+        finalIndexFile = mapOutputFile.getOutputIndexFileForWriteInVolume(indexFilename);
+
+        sameVolRename(filename, finalOutputFile);
+        sameVolRename(indexFilename, finalIndexFile);
+        if (LOG.isInfoEnabled()) {
+          LOG.info("numSpills=" + numSpills + ", finalOutputFile=" + finalOutputFile + ", "
+              + "finalIndexFile=" + finalIndexFile + ", filename=" + filename + ", indexFilename=" +
+              indexFilename);
+        }
+        return;
       }
-      return;
-    }
 
-    finalOutputFile =
-        mapOutputFile.getOutputFileForWrite(0); //TODO
-    finalIndexFile =
-        mapOutputFile.getOutputIndexFileForWrite(0); //TODO
+      finalOutputFile =
+          mapOutputFile.getOutputFileForWrite(0); //TODO
+      finalIndexFile =
+          mapOutputFile.getOutputIndexFileForWrite(0); //TODO
 
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("numSpills: " + numSpills + ", finalOutputFile:" + finalOutputFile + ", finalIndexFile:"
-              + finalIndexFile);
-    }
+      if (LOG.isDebugEnabled()) {
+        LOG.debug(
+            "numSpills: " + numSpills + ", finalOutputFile:" + finalOutputFile + ", finalIndexFile:"
+                + finalIndexFile);
+      }
 
-    //The output stream for the final single output file
-    FSDataOutputStream finalOut = rfs.create(finalOutputFile, true, 4096);
+      //The output stream for the final single output file
+      FSDataOutputStream finalOut = rfs.create(finalOutputFile, true, 4096);
 
-    final TezSpillRecord spillRec = new TezSpillRecord(partitions);
+      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++) {
+          Path spillFilename = spillFilePaths.get(i);
+          TezIndexRecord indexRecord = indexCacheList.get(i).getIndex(parts);
 
-    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 = spillFilePaths.get(i);
-        TezIndexRecord indexRecord = indexCacheList.get(i).getIndex(parts);
+          Segment s =
+              new Segment(rfs, spillFilename, indexRecord.getStartOffset(),
+                  indexRecord.getPartLength(), codec, ifileReadAhead,
+                  ifileReadAheadLength, ifileBufferSize, true);
+          segmentList.add(i, s);
+        }
 
-        Segment s =
-            new Segment(rfs, spillFilename, indexRecord.getStartOffset(),
-                             indexRecord.getPartLength(), codec, ifileReadAhead,
-                             ifileReadAheadLength, ifileBufferSize, true);
-        segmentList.add(i, s);
-      }
+        int mergeFactor =
+            this.conf.getInt(TezRuntimeConfiguration.TEZ_RUNTIME_IO_SORT_FACTOR,
+                TezRuntimeConfiguration.TEZ_RUNTIME_IO_SORT_FACTOR_DEFAULT);
+        // 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, true,
+            null, spilledRecordsCounter, null,
+            null); // Not using any Progress in TezMerger. Should just work.
 
-      int mergeFactor = 
-              this.conf.getInt(TezRuntimeConfiguration.TEZ_RUNTIME_IO_SORT_FACTOR, 
-                  TezRuntimeConfiguration.TEZ_RUNTIME_IO_SORT_FACTOR_DEFAULT);
-      // 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, true,
-                     null, spilledRecordsCounter, null,
-                     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, null, merger.needsRLE());
-      if (combiner == null || numSpills < minSpillsForCombine) {
-        TezMerger.writeFile(kvIter, writer, nullProgressable, TezRuntimeConfiguration.TEZ_RUNTIME_RECORDS_BEFORE_PROGRESS_DEFAULT);
-      } else {
-        runCombineProcessor(kvIter, writer);
-      }
+        //write merged output to disk
+        long segmentStart = finalOut.getPos();
+        Writer writer =
+            new Writer(conf, finalOut, keyClass, valClass, codec,
+                spilledRecordsCounter, null, merger.needsRLE());
+        if (combiner == null || numSpills < minSpillsForCombine) {
+          TezMerger.writeFile(kvIter, writer, nullProgressable,
+              TezRuntimeConfiguration.TEZ_RUNTIME_RECORDS_BEFORE_PROGRESS_DEFAULT);
+        } else {
+          runCombineProcessor(kvIter, writer);
+        }
 
-      //close
-      writer.close();
+        //close
+        writer.close();
 
-      // record offsets
-      final TezIndexRecord rec = 
-          new TezIndexRecord(
-              segmentStart, 
-              writer.getRawLength(), 
-              writer.getCompressedLength());
-      spillRec.putIndex(rec, parts);
-    }
+        // 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 = spillFileIndexPaths.get(i);
-      Path spillFilename = spillFilePaths.get(i);
-      rfs.delete(indexFilename,true);
-      rfs.delete(spillFilename,true);
-    }
+      spillRec.writeToFile(finalIndexFile, conf);
+      finalOut.close();
+      for (int i = 0; i < numSpills; i++) {
+        Path indexFilename = spillFileIndexPaths.get(i);
+        Path spillFilename = spillFilePaths.get(i);
+        rfs.delete(indexFilename, true);
+        rfs.delete(spillFilename, true);
+      }
 
-    spillFileIndexPaths.clear();
-    spillFilePaths.clear();
+      spillFileIndexPaths.clear();
+      spillFilePaths.clear();
+    } catch(InterruptedException ie) {
+      if (cleanup) {
+        cleanup();
+      }
+      Thread.currentThread().interrupt();
+    }
   }
 
 

http://git-wip-us.apache.org/repos/asf/tez/blob/146ab070/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
index 758e9c7..3b7bf05 100644
--- 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
@@ -76,7 +76,7 @@ public class TezMerger {
                             TezCounter writesCounter,
                             TezCounter bytesReadCounter,
                             Progress mergePhase)
-  throws IOException {
+      throws IOException, InterruptedException {
     return 
       new MergeQueue(conf, fs, inputs, deleteInputs, codec, ifileReadAhead,
                            ifileReadAheadLength, ifileBufferSize, false, comparator, 
@@ -101,7 +101,7 @@ public class TezMerger {
                             TezCounter mergedMapOutputsCounter,
                             TezCounter bytesReadCounter,
                             Progress mergePhase)
-  throws IOException {
+      throws IOException, InterruptedException {
     return 
       new MergeQueue(conf, fs, inputs, deleteInputs, codec, ifileReadAhead,
                            ifileReadAheadLength, ifileBufferSize, false, comparator, 
@@ -124,7 +124,7 @@ public class TezMerger {
                             TezCounter writesCounter,
                             TezCounter bytesReadCounter,
                             Progress mergePhase)
-      throws IOException {
+      throws IOException, InterruptedException {
     // Get rid of this ?
     return merge(conf, fs, keyClass, valueClass, segments, mergeFactor, tmpDir,
                  comparator, reporter, false, readsCounter, writesCounter, bytesReadCounter,
@@ -142,7 +142,7 @@ public class TezMerger {
                             TezCounter writesCounter,
                             TezCounter bytesReadCounter,
                             Progress mergePhase)
-      throws IOException {
+      throws IOException, InterruptedException {
     return new MergeQueue(conf, fs, segments, comparator, reporter,
                            sortSegments, false).merge(keyClass, valueClass,
                                                mergeFactor, tmpDir,
@@ -163,7 +163,7 @@ public class TezMerger {
                             TezCounter writesCounter,
                             TezCounter bytesReadCounter,
                             Progress mergePhase)
-      throws IOException {
+      throws IOException, InterruptedException {
     return new MergeQueue(conf, fs, segments, comparator, reporter,
                            sortSegments, codec, considerFinalMergeForProgress).
                                          merge(keyClass, valueClass,
@@ -185,7 +185,7 @@ public class TezMerger {
                           TezCounter writesCounter,
                           TezCounter bytesReadCounter,
                           Progress mergePhase)
-    throws IOException {
+      throws IOException, InterruptedException {
   return new MergeQueue(conf, fs, segments, comparator, reporter,
                          sortSegments, codec, false).merge(keyClass, valueClass,
                                              mergeFactor, inMemSegments,
@@ -196,9 +196,9 @@ public class TezMerger {
 }
 
   public static <K extends Object, V extends Object>
-  void writeFile(TezRawKeyValueIterator records, Writer writer, 
-                 Progressable progressable, long recordsBeforeProgress) 
-  throws IOException {
+  void writeFile(TezRawKeyValueIterator records, Writer writer,
+      Progressable progressable, long recordsBeforeProgress)
+      throws IOException, InterruptedException {
     long recordCtr = 0;
     long count = 0;
     while(records.next()) {
@@ -211,6 +211,15 @@ public class TezMerger {
       
       if (((recordCtr++) % recordsBeforeProgress) == 0) {
         progressable.progress();
+        if (Thread.currentThread().isInterrupted()) {
+          /**
+           * Takes care DefaultSorter.mergeParts, MergeManager's merger threads,
+           * PipelinedSorter's flush(). This is not expensive check as it is carried out every
+           * 10000 records or so.
+           */
+          throw new InterruptedException("Current thread=" + Thread.currentThread().getName() + " got "
+              + "interrupted");
+        }
       }
     }
     if ((count > 0) && LOG.isDebugEnabled()) {
@@ -614,7 +623,7 @@ public class TezMerger {
                                      TezCounter writesCounter,
                                      TezCounter bytesReadCounter,
                                      Progress mergePhase)
-        throws IOException {
+        throws IOException, InterruptedException {
       return merge(keyClass, valueClass, factor, 0, tmpDir,
                    readsCounter, writesCounter, bytesReadCounter, mergePhase);
     }
@@ -625,7 +634,7 @@ public class TezMerger {
                                      TezCounter writesCounter,
                                      TezCounter bytesReadCounter,
                                      Progress mergePhase)
-        throws IOException {
+        throws IOException, InterruptedException {
       LOG.info("Merging " + segments.size() + " sorted segments");
       if (segments.size() == 0) {
         LOG.info("Nothing to merge. Returning an empty iterator");

http://git-wip-us.apache.org/repos/asf/tez/blob/146ab070/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
index 2cbb70a..9783c79 100644
--- 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
@@ -193,6 +193,9 @@ public class DefaultSorter extends ExternalSorter implements IndexedSortable {
         spillDone.await();
       }
     } catch (InterruptedException e) {
+      //interrupt spill thread
+      spillThread.interrupt();
+      Thread.currentThread().interrupt();
       throw new IOException("Spill thread failed to initialize", e);
     } finally {
       spillLock.unlock();
@@ -603,6 +606,7 @@ public class DefaultSorter extends ExternalSorter implements IndexedSortable {
                   spillDone.await();
                 }
               } catch (InterruptedException e) {
+                Thread.currentThread().interrupt();
                   throw new IOException(
                       "Buffer interrupted while waiting for the writer", e);
               }
@@ -625,9 +629,45 @@ public class DefaultSorter extends ExternalSorter implements IndexedSortable {
     }
   }
 
+  void interruptSpillThread() throws IOException {
+    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) {
+      LOG.info("Spill thread interrupted");
+      //Reset status
+      Thread.currentThread().interrupt();
+      throw new IOException("Spill failed", e);
+    }
+  }
+
   @Override
   public void flush() throws IOException {
     LOG.info("Starting flush of map output");
+    if (Thread.currentThread().isInterrupted()) {
+      /**
+       * Possible that the thread got interrupted when flush was happening or when the flush was
+       * never invoked. As a part of cleanup activity in TezTaskRunner, it would invoke close()
+       * on all I/O. At that time, this is safe to cleanup
+       */
+      if (cleanup) {
+        cleanup();
+      }
+      try {
+        interruptSpillThread();
+      } catch(IOException e) {
+        //safe to ignore
+      }
+      return;
+    }
+
     spillLock.lock();
     try {
       while (spillInProgress) {
@@ -656,28 +696,25 @@ public class DefaultSorter extends ExternalSorter implements IndexedSortable {
         sortAndSpill();
       }
     } catch (InterruptedException e) {
+      //Reset status
+      Thread.currentThread().interrupt();
+      interruptSpillThread();
       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
+
+    interruptSpillThread();
+    // release sort buffer before the mergecl
     //FIXME
     //kvbuffer = null;
 
-    mergeParts();
+    try {
+      mergeParts();
+    } catch (InterruptedException e) {
+      cleanup();
+      Thread.currentThread().interrupt();
+    }
     if (finalMergeEnabled) {
       fileOutputByteCounter.increment(rfs.getFileStatus(finalOutputFile).getLen());
     }
@@ -715,6 +752,7 @@ public class DefaultSorter extends ExternalSorter implements IndexedSortable {
           }
         }
       } catch (InterruptedException e) {
+        LOG.info("Spill thread interrupted");
         Thread.currentThread().interrupt();
       } finally {
         spillLock.unlock();
@@ -1085,7 +1123,7 @@ public class DefaultSorter extends ExternalSorter implements IndexedSortable {
     outputContext.sendEvents(events);
   }
 
-  private void mergeParts() throws IOException {
+  private void mergeParts() throws IOException, InterruptedException {
     // get the approximate size of the final output/index files
     long finalOutFileSize = 0;
     long finalIndexFileSize = 0;

http://git-wip-us.apache.org/repos/asf/tez/blob/146ab070/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/OrderedGroupedKVInput.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/OrderedGroupedKVInput.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/OrderedGroupedKVInput.java
index d784fcd..49cf102 100644
--- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/OrderedGroupedKVInput.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/OrderedGroupedKVInput.java
@@ -351,6 +351,7 @@ public class OrderedGroupedKVInput extends AbstractLogicalInput {
     confKeys.add(TezConfiguration.TEZ_COUNTERS_GROUP_NAME_MAX_LENGTH);
     confKeys.add(TezConfiguration.TEZ_COUNTERS_COUNTER_NAME_MAX_LENGTH);
     confKeys.add(TezConfiguration.TEZ_COUNTERS_MAX_GROUPS);
+    confKeys.add(TezRuntimeConfiguration.TEZ_RUNTIME_CLEANUP_FILES_ON_INTERRUPT);
   }
 
   // TODO Maybe add helper methods to extract keys

http://git-wip-us.apache.org/repos/asf/tez/blob/146ab070/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/UnorderedKVInput.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/UnorderedKVInput.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/UnorderedKVInput.java
index 62fa9a5..7fc9317 100644
--- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/UnorderedKVInput.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/UnorderedKVInput.java
@@ -267,6 +267,7 @@ public class UnorderedKVInput extends AbstractLogicalInput {
     confKeys.add(TezConfiguration.TEZ_COUNTERS_GROUP_NAME_MAX_LENGTH);
     confKeys.add(TezConfiguration.TEZ_COUNTERS_COUNTER_NAME_MAX_LENGTH);
     confKeys.add(TezConfiguration.TEZ_COUNTERS_MAX_GROUPS);
+    confKeys.add(TezRuntimeConfiguration.TEZ_RUNTIME_CLEANUP_FILES_ON_INTERRUPT);
   }
 
   // TODO Maybe add helper methods to extract keys

http://git-wip-us.apache.org/repos/asf/tez/blob/146ab070/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/OrderedPartitionedKVOutput.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/OrderedPartitionedKVOutput.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/OrderedPartitionedKVOutput.java
index 6227fb9..53abc17 100644
--- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/OrderedPartitionedKVOutput.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/OrderedPartitionedKVOutput.java
@@ -248,6 +248,7 @@ public class OrderedPartitionedKVOutput extends AbstractLogicalOutput {
     confKeys.add(TezConfiguration.TEZ_COUNTERS_COUNTER_NAME_MAX_LENGTH);
     confKeys.add(TezConfiguration.TEZ_COUNTERS_MAX_GROUPS);
     confKeys.add(TezRuntimeConfiguration.TEZ_RUNTIME_SORTER_CLASS);
+    confKeys.add(TezRuntimeConfiguration.TEZ_RUNTIME_CLEANUP_FILES_ON_INTERRUPT);
   }
 
   // TODO Maybe add helper methods to extract keys

http://git-wip-us.apache.org/repos/asf/tez/blob/146ab070/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/UnorderedKVOutput.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/UnorderedKVOutput.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/UnorderedKVOutput.java
index 08e6ec0..b50f17d 100644
--- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/UnorderedKVOutput.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/UnorderedKVOutput.java
@@ -171,6 +171,7 @@ public class UnorderedKVOutput extends AbstractLogicalOutput {
     confKeys.add(TezConfiguration.TEZ_COUNTERS_GROUP_NAME_MAX_LENGTH);
     confKeys.add(TezConfiguration.TEZ_COUNTERS_COUNTER_NAME_MAX_LENGTH);
     confKeys.add(TezConfiguration.TEZ_COUNTERS_MAX_GROUPS);
+    confKeys.add(TezRuntimeConfiguration.TEZ_RUNTIME_CLEANUP_FILES_ON_INTERRUPT);
   }
 
   // TODO Maybe add helper methods to extract keys

http://git-wip-us.apache.org/repos/asf/tez/blob/146ab070/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/UnorderedPartitionedKVOutput.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/UnorderedPartitionedKVOutput.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/UnorderedPartitionedKVOutput.java
index 38450ee..7498627 100644
--- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/UnorderedPartitionedKVOutput.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/UnorderedPartitionedKVOutput.java
@@ -144,6 +144,7 @@ public class UnorderedPartitionedKVOutput extends AbstractLogicalOutput {
     confKeys.add(TezConfiguration.TEZ_COUNTERS_GROUP_NAME_MAX_LENGTH);
     confKeys.add(TezConfiguration.TEZ_COUNTERS_COUNTER_NAME_MAX_LENGTH);
     confKeys.add(TezConfiguration.TEZ_COUNTERS_MAX_GROUPS);
+    confKeys.add(TezRuntimeConfiguration.TEZ_RUNTIME_CLEANUP_FILES_ON_INTERRUPT);
   }
 
   // TODO Maybe add helper methods to extract keys

http://git-wip-us.apache.org/repos/asf/tez/blob/146ab070/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/TestValuesIterator.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/TestValuesIterator.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/TestValuesIterator.java
index edb9b15..f62179a 100644
--- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/TestValuesIterator.java
+++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/TestValuesIterator.java
@@ -169,28 +169,28 @@ public class TestValuesIterator {
   }
 
   @Test(timeout = 20000)
-  public void testIteratorWithInMemoryReader() throws IOException {
+  public void testIteratorWithInMemoryReader() throws IOException, InterruptedException {
     ValuesIterator iterator = createIterator(true);
     verifyIteratorData(iterator);
   }
 
   @Test(timeout = 20000)
-  public void testIteratorWithIFileReader() throws IOException {
+  public void testIteratorWithIFileReader() throws IOException, InterruptedException {
     ValuesIterator iterator = createIterator(false);
     verifyIteratorData(iterator);
   }
 
   @Test(timeout = 20000)
-  public void testCountedIteratorWithInmemoryReader() throws IOException {
+  public void testCountedIteratorWithInmemoryReader() throws IOException, InterruptedException {
     verifyCountedIteratorReader(true);
   }
 
   @Test(timeout = 20000)
-  public void testCountedIteratorWithIFileReader() throws IOException {
+  public void testCountedIteratorWithIFileReader() throws IOException, InterruptedException {
     verifyCountedIteratorReader(false);
   }
 
-  private void verifyCountedIteratorReader(boolean inMemory) throws IOException {
+  private void verifyCountedIteratorReader(boolean inMemory) throws IOException, InterruptedException {
     TezCounter keyCounter = new GenericCounter("inputKeyCounter", "y3");
     TezCounter tupleCounter = new GenericCounter("inputValuesCounter", "y4");
     ValuesIterator iterator = createCountedIterator(inMemory, keyCounter,
@@ -207,7 +207,7 @@ public class TestValuesIterator {
   }
 
   @Test(timeout = 20000)
-  public void testIteratorWithIFileReaderEmptyPartitions() throws IOException {
+  public void testIteratorWithIFileReaderEmptyPartitions() throws IOException, InterruptedException {
     ValuesIterator iterator = createEmptyIterator(false);
     assertTrue(iterator.moveToNext() == false);
 
@@ -224,7 +224,8 @@ public class TestValuesIterator {
     }
   }
 
-  private ValuesIterator createEmptyIterator(boolean inMemory) throws IOException {
+  private ValuesIterator createEmptyIterator(boolean inMemory)
+      throws IOException, InterruptedException {
     if (!inMemory) {
       streamPaths = new Path[0];
       //This will return EmptyIterator
@@ -323,7 +324,7 @@ public class TestValuesIterator {
    * @return ValuesIterator
    * @throws IOException
    */
-  private ValuesIterator createIterator(boolean inMemory) throws IOException {
+  private ValuesIterator createIterator(boolean inMemory) throws IOException, InterruptedException {
     if (!inMemory) {
       streamPaths = createFiles();
       //Merge all files to get KeyValueIterator
@@ -353,7 +354,8 @@ public class TestValuesIterator {
    * @return ValuesIterator
    * @throws IOException
    */
-  private ValuesIterator createCountedIterator(boolean inMemory, TezCounter keyCounter, TezCounter tupleCounter) throws IOException {
+  private ValuesIterator createCountedIterator(boolean inMemory, TezCounter keyCounter, TezCounter tupleCounter)
+      throws IOException, InterruptedException {
     if (!inMemory) {
       streamPaths = createFiles();
       //Merge all files to get KeyValueIterator

http://git-wip-us.apache.org/repos/asf/tez/blob/146ab070/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/TestMergeManager.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/TestMergeManager.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/TestMergeManager.java
index 094237a..0faa22a 100644
--- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/TestMergeManager.java
+++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/TestMergeManager.java
@@ -28,6 +28,7 @@ import java.util.LinkedList;
 import java.util.List;
 import java.util.UUID;
 
+import com.google.common.collect.Sets;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -166,9 +167,32 @@ public class TestMergeManager {
     Assert.assertTrue(mergeManager.postMergeMemLimit == initialMemoryAvailable);
   }
 
+  class InterruptingThread implements Runnable {
+
+    MergeManager.OnDiskMerger mergeThread;
+
+    public InterruptingThread(MergeManager.OnDiskMerger mergeThread) {
+      this.mergeThread = mergeThread;
+    }
+
+    @Override public void run() {
+        while(this.mergeThread.tmpDir == null) {
+          //this is tight loop
+        }
+
+        this.mergeThread.interrupt();
+    }
+  }
+
   @Test(timeout = 10000)
-  public void testLocalDiskMergeMultipleTasks() throws IOException {
+  public void testLocalDiskMergeMultipleTasks() throws IOException, InterruptedException {
+    testLocalDiskMergeMultipleTasks(false);
+    testLocalDiskMergeMultipleTasks(true);
+  }
 
+
+  void testLocalDiskMergeMultipleTasks(boolean interruptInMiddle)
+      throws IOException, InterruptedException {
     Configuration conf = new TezConfiguration(defaultConf);
     conf.setBoolean(TezRuntimeConfiguration.TEZ_RUNTIME_COMPRESS, false);
     conf.set(TezRuntimeConfiguration.TEZ_RUNTIME_KEY_CLASS, IntWritable.class.getName());
@@ -194,6 +218,7 @@ public class TestMergeManager {
         new MergeManager(conf, localFs, localDirAllocator, t0inputContext, null, null, null, null,
             t0exceptionReporter, 2000000, null, false, -1);
     MergeManager t0mergeManager = spy(t0mergeManagerReal);
+    t0mergeManager.configureAndStart();
 
     MergeManager t1mergeManagerReal =
         new MergeManager(conf, localFs, localDirAllocator, t1inputContext, null, null, null, null,
@@ -249,30 +274,48 @@ public class TestMergeManager {
     List<FileChunk> t0MergeFiles = new LinkedList<FileChunk>();
     t0MergeFiles.addAll(t0mergeManager.onDiskMapOutputs);
     t0mergeManager.onDiskMapOutputs.clear();
-    t0mergeManager.onDiskMerger.merge(t0MergeFiles);
-    Assert.assertEquals(1, t0mergeManager.onDiskMapOutputs.size());
-
-
-    t1MapOutput0.commit();
-    t1MapOutput1.commit();
-    verify(t1mergeManager).closeOnDiskFile(t1MapOutput0.getOutputPath());
-    verify(t1mergeManager).closeOnDiskFile(t1MapOutput1.getOutputPath());
-    // Run the OnDiskMerge via MergeManager
-    // Simulate the thread invocation - remove files, and invoke merge
-    List<FileChunk> t1MergeFiles = new LinkedList<FileChunk>();
-    t1MergeFiles.addAll(t1mergeManager.onDiskMapOutputs);
-    t1mergeManager.onDiskMapOutputs.clear();
-    t1mergeManager.onDiskMerger.merge(t1MergeFiles);
-    Assert.assertEquals(1, t1mergeManager.onDiskMapOutputs.size());
 
-    Assert.assertNotEquals(t0mergeManager.onDiskMapOutputs.iterator().next().getPath(),
-        t1mergeManager.onDiskMapOutputs.iterator().next().getPath());
+    if (!interruptInMiddle) {
+      t0mergeManager.onDiskMerger.merge(t0MergeFiles);
+      Assert.assertEquals(1, t0mergeManager.onDiskMapOutputs.size());
+    } else {
+
+      //Start Interrupting thread
+      Thread interruptingThread = new Thread(new InterruptingThread(t0mergeManager.onDiskMerger));
+      interruptingThread.start();
+      try {
+        Thread.sleep(1000);
+      } catch (InterruptedException e) {
+        e.printStackTrace();
+      }
 
-    Assert.assertTrue(t0mergeManager.onDiskMapOutputs.iterator().next().getPath().toString()
-        .contains(t0inputContext.getUniqueIdentifier()));
-    Assert.assertTrue(t1mergeManager.onDiskMapOutputs.iterator().next().getPath().toString()
-        .contains(t1inputContext.getUniqueIdentifier()));
+      //Will be interrupted in the middle by interruptingThread.
+      t0mergeManager.onDiskMerger.startMerge(Sets.newHashSet(t0MergeFiles));
+      t0mergeManager.onDiskMerger.waitForMerge();
+      Assert.assertNotEquals(1, t0mergeManager.onDiskMapOutputs.size());
+    }
 
+    if (!interruptInMiddle) {
+      t1MapOutput0.commit();
+      t1MapOutput1.commit();
+      verify(t1mergeManager).closeOnDiskFile(t1MapOutput0.getOutputPath());
+      verify(t1mergeManager).closeOnDiskFile(t1MapOutput1.getOutputPath());
+      // Run the OnDiskMerge via MergeManager
+      // Simulate the thread invocation - remove files, and invoke merge
+      List<FileChunk> t1MergeFiles = new LinkedList<FileChunk>();
+      t1MergeFiles.addAll(t1mergeManager.onDiskMapOutputs);
+      t1mergeManager.onDiskMapOutputs.clear();
+      t1mergeManager.onDiskMerger.merge(t1MergeFiles);
+      Assert.assertEquals(1, t1mergeManager.onDiskMapOutputs.size());
+
+      Assert.assertNotEquals(t0mergeManager.onDiskMapOutputs.iterator().next().getPath(),
+          t1mergeManager.onDiskMapOutputs.iterator().next().getPath());
+
+      Assert.assertTrue(t0mergeManager.onDiskMapOutputs.iterator().next().getPath().toString()
+          .contains(t0inputContext.getUniqueIdentifier()));
+      Assert.assertTrue(t1mergeManager.onDiskMapOutputs.iterator().next().getPath().toString()
+          .contains(t1inputContext.getUniqueIdentifier()));
+    }
   }
 
   private InputContext createMockInputContext(String uniqueId) {

http://git-wip-us.apache.org/repos/asf/tez/blob/146ab070/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/sort/impl/TestTezMerger.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/sort/impl/TestTezMerger.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/sort/impl/TestTezMerger.java
index bb932f2..b86d054 100644
--- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/sort/impl/TestTezMerger.java
+++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/sort/impl/TestTezMerger.java
@@ -557,7 +557,8 @@ public class TestTezMerger {
    * @return
    * @throws IOException
    */
-  private TezRawKeyValueIterator merge(List<Path> pathList, RawComparator rc) throws IOException {
+  private TezRawKeyValueIterator merge(List<Path> pathList, RawComparator rc)
+      throws IOException, InterruptedException {
     TezMerger merger = new TezMerger();
     TezRawKeyValueIterator records = merger.merge(defaultConf, localFs, IntWritable.class,
         LongWritable.class, null, false, 0, 1024, pathList.toArray(new Path[pathList.size()]),


[44/50] [abbrv] tez git commit: TEZ-2347. Expose additional information in TaskCommunicatorContext. (sseth)

Posted by ss...@apache.org.
TEZ-2347. Expose additional information in TaskCommunicatorContext. (sseth)


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

Branch: refs/heads/TEZ-2003
Commit: 960d92bfe8fbd440f273f2920be25919e62a93e4
Parents: 3e03847
Author: Siddharth Seth <ss...@apache.org>
Authored: Mon Apr 20 13:17:31 2015 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Wed May 6 00:18:08 2015 -0700

----------------------------------------------------------------------
 TEZ-2003-CHANGES.txt                            |  1 +
 .../tez/dag/api/TaskCommunicatorContext.java    | 50 ++++++++++++++++++++
 .../dag/app/TaskCommunicatorContextImpl.java    | 50 ++++++++++++++++++++
 .../java/org/apache/tez/dag/app/dag/DAG.java    |  2 +
 .../java/org/apache/tez/dag/app/dag/Task.java   |  1 +
 .../org/apache/tez/dag/app/dag/TaskAttempt.java |  6 +++
 .../apache/tez/dag/app/dag/impl/DAGImpl.java    | 10 ++++
 .../tez/dag/app/dag/impl/TaskAttemptImpl.java   | 12 +++++
 .../apache/tez/dag/app/dag/impl/TaskImpl.java   | 13 ++++-
 9 files changed, 144 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/960d92bf/TEZ-2003-CHANGES.txt
----------------------------------------------------------------------
diff --git a/TEZ-2003-CHANGES.txt b/TEZ-2003-CHANGES.txt
index ca5225e..7c13110 100644
--- a/TEZ-2003-CHANGES.txt
+++ b/TEZ-2003-CHANGES.txt
@@ -16,5 +16,6 @@ ALL CHANGES:
   TEZ-2284. Separate TaskReporter into an interface.
   TEZ-2285. Allow TaskCommunicators to indicate task/container liveness.
   TEZ-2302. Allow TaskCommunicators to subscribe for Vertex updates.
+  TEZ-2347. Expose additional information in TaskCommunicatorContext.
 
 INCOMPATIBLE CHANGES:

http://git-wip-us.apache.org/repos/asf/tez/blob/960d92bf/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicatorContext.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicatorContext.java b/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicatorContext.java
index 19caed9..56345ab 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicatorContext.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicatorContext.java
@@ -16,6 +16,7 @@ package org.apache.tez.dag.api;
 
 import javax.annotation.Nullable;
 import java.io.IOException;
+import java.util.Collection;
 import java.util.Set;
 
 import org.apache.hadoop.security.Credentials;
@@ -71,4 +72,53 @@ public interface TaskCommunicatorContext {
   // TODO TEZ-2003 API. Should a method exist for task succeeded.
 
   // TODO Eventually Add methods to report availability stats to the scheduler.
+
+  /**
+   * Get the name of the currently executing dag
+   * @return the name of the currently executing dag
+   */
+  String getCurretnDagName();
+
+  /**
+   * Get the name of the Input vertices for the specified vertex.
+   * Root Inputs are not returned.
+   * @param vertexName the vertex for which source vertex names will be returned
+   * @return an Iterable containing the list of input vertices for the specified vertex
+   */
+  Iterable<String> getInputVertexNames(String vertexName);
+
+  /**
+   * Get the total number of tasks in the given vertex
+   * @param vertexName
+   * @return total number of tasks in this vertex
+   */
+  int getVertexTotalTaskCount(String vertexName);
+
+  /**
+   * Get the number of completed tasks for a given vertex
+   * @param vertexName the vertex name
+   * @return the number of completed tasks for the vertex
+   */
+  int getVertexCompletedTaskCount(String vertexName);
+
+  /**
+   * Get the number of running tasks for a given vertex
+   * @param vertexName the vertex name
+   * @return the number of running tasks for the vertex
+   */
+  int getVertexRunningTaskCount(String vertexName);
+
+  /**
+   * Get the start time for the first attempt of the specified task
+   * @param vertexName the vertex to which the task belongs
+   * @param taskIndex the index of the task
+   * @return the start time for the first attempt of the task
+   */
+  long getFirstAttemptStartTime(String vertexName, int taskIndex);
+
+  /**
+   * Get the start time for the currently executing DAG
+   * @return time when the current dag started executing
+   */
+  long getDagStartTime();
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/960d92bf/tez-dag/src/main/java/org/apache/tez/dag/app/TaskCommunicatorContextImpl.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/TaskCommunicatorContextImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/TaskCommunicatorContextImpl.java
index 3714c3c..4cb0c93 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/TaskCommunicatorContextImpl.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/TaskCommunicatorContextImpl.java
@@ -18,7 +18,9 @@ import javax.annotation.Nullable;
 import java.io.IOException;
 import java.util.Set;
 
+import com.google.common.base.Function;
 import com.google.common.base.Preconditions;
+import com.google.common.collect.Iterables;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
@@ -31,6 +33,7 @@ import org.apache.tez.dag.api.TezException;
 import org.apache.tez.dag.api.TezUncheckedException;
 import org.apache.tez.dag.api.event.VertexState;
 import org.apache.tez.dag.api.event.VertexStateUpdate;
+import org.apache.tez.dag.app.dag.Vertex;
 import org.apache.tez.dag.app.dag.VertexStateUpdateListener;
 import org.apache.tez.dag.records.TezTaskAttemptID;
 
@@ -111,6 +114,53 @@ public class TaskCommunicatorContextImpl implements TaskCommunicatorContext, Ver
     context.getCurrentDAG().getStateChangeNotifier().registerForVertexUpdates(vertexName, stateSet, this);
   }
 
+  @Override
+  public String getCurretnDagName() {
+    return context.getCurrentDAG().getName();
+  }
+
+  @Override
+  public Iterable<String> getInputVertexNames(String vertexName) {
+    Preconditions.checkNotNull(vertexName, "VertexName cannot be null: " + vertexName);
+    Vertex vertex = context.getCurrentDAG().getVertex(vertexName);
+    Set<Vertex> sources = vertex.getInputVertices().keySet();
+    return Iterables.transform(sources, new Function<Vertex, String>() {
+      @Override
+      public String apply(@Nullable Vertex input) {
+        return input.getName();
+      }
+    });
+  }
+
+  @Override
+  public int getVertexTotalTaskCount(String vertexName) {
+    Preconditions.checkArgument(vertexName != null, "VertexName must be specified");
+    return context.getCurrentDAG().getVertex(vertexName).getTotalTasks();
+  }
+
+  @Override
+  public int getVertexCompletedTaskCount(String vertexName) {
+    Preconditions.checkArgument(vertexName != null, "VertexName must be specified");
+    return context.getCurrentDAG().getVertex(vertexName).getCompletedTasks();
+  }
+
+  @Override
+  public int getVertexRunningTaskCount(String vertexName) {
+    Preconditions.checkArgument(vertexName != null, "VertexName must be specified");
+    return context.getCurrentDAG().getVertex(vertexName).getRunningTasks();
+  }
+
+  @Override
+  public long getFirstAttemptStartTime(String vertexName, int taskIndex) {
+    Preconditions.checkArgument(vertexName != null, "VertexName must be specified");
+    Preconditions.checkArgument(taskIndex >=0, "TaskIndex must be > 0");
+    return context.getCurrentDAG().getVertex(vertexName).getTask(taskIndex).getFirstAttemptStartTime();
+  }
+
+  @Override
+  public long getDagStartTime() {
+    return context.getCurrentDAG().getStartTime();
+  }
 
   @Override
   public void onStateUpdated(VertexStateUpdate event) {

http://git-wip-us.apache.org/repos/asf/tez/blob/960d92bf/tez-dag/src/main/java/org/apache/tez/dag/app/dag/DAG.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/DAG.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/DAG.java
index 6d6872b..458362f 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/DAG.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/DAG.java
@@ -94,6 +94,8 @@ public interface DAG {
 
   Map<String, TezVertexID> getVertexNameIDMapping();
 
+  long getStartTime();
+
   StateChangeNotifier getStateChangeNotifier();
 
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/960d92bf/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 b798fce..3af14b5 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
@@ -66,4 +66,5 @@ public interface Task {
 
   public void registerTezEvent(TezEvent tezEvent);
 
+  long getFirstAttemptStartTime();
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/960d92bf/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 6c85cc2..26613e9 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
@@ -122,6 +122,12 @@ public interface TaskAttempt {
    */
   long getLaunchTime();
 
+  /**
+   * Get the time at which this attempt was scheduled
+   * @return the time at which this attempt was scheduled, 0 if it hasn't been scheduled yet
+   */
+  long getScheduleTime();
+
   /** 
    * @return attempt's finish time. If attempt is not finished
    *  yet, returns 0.

http://git-wip-us.apache.org/repos/asf/tez/blob/960d92bf/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 998108b..64a184f 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
@@ -685,6 +685,16 @@ public class DAGImpl implements org.apache.tez.dag.app.dag.DAG,
   }
 
   @Override
+  public long getStartTime() {
+    readLock.lock();
+    try {
+      return this.startTime;
+    } finally {
+      readLock.unlock();
+    }
+  }
+
+  @Override
   public StateChangeNotifier getStateChangeNotifier() {
     return entityUpdateTracker;
   }

http://git-wip-us.apache.org/repos/asf/tez/blob/960d92bf/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 11d4df9..092520d 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
@@ -135,6 +135,7 @@ public class TaskAttemptImpl implements TaskAttempt,
   protected final AppContext appContext;
   private final TaskHeartbeatHandler taskHeartbeatHandler;
   private long launchTime = 0;
+  private long scheduleTime = 0;
   private long finishTime = 0;
   private String trackerName;
   private int httpPort;
@@ -671,6 +672,16 @@ public class TaskAttemptImpl implements TaskAttempt,
   }
 
   @Override
+  public long getScheduleTime() {
+    readLock.lock();
+    try {
+      return scheduleTime;
+    } finally {
+      readLock.unlock();
+    }
+  }
+
+  @Override
   public long getFinishTime() {
     readLock.lock();
     try {
@@ -1030,6 +1041,7 @@ public class TaskAttemptImpl implements TaskAttempt,
     public TaskAttemptStateInternal transition(TaskAttemptImpl ta, TaskAttemptEvent event) {
       TaskAttemptEventSchedule scheduleEvent = (TaskAttemptEventSchedule) event;
 
+      ta.scheduleTime = ta.clock.getTime();
       // TODO Creating the remote task here may not be required in case of
       // recovery.
 

http://git-wip-us.apache.org/repos/asf/tez/blob/960d92bf/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 15382a8..d4eabe6 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
@@ -1492,7 +1492,18 @@ public class TaskImpl implements Task, EventHandler<TaskEvent> {
       this.writeLock.unlock();
     }
   }
-  
+
+  @Override
+  public long getFirstAttemptStartTime() {
+    readLock.lock();
+    try {
+      // The first attempt will always have an index of 0.
+      return getAttempt(TezTaskAttemptID.getInstance(getTaskId(), 0)).getScheduleTime();
+    } finally {
+      readLock.unlock();
+    }
+  }
+
   @Private
   @VisibleForTesting
   public List<TezEvent> getTaskEvents() {


[03/50] [abbrv] tez git commit: TEZ-2390. tez-tools swimlane tool fails to parse large jobs >8K containers (jeagles)

Posted by ss...@apache.org.
TEZ-2390. tez-tools swimlane tool fails to parse large jobs >8K containers (jeagles)


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

Branch: refs/heads/TEZ-2003
Commit: a02a5ea9dd8af4e47114ef4145d8e6b75db6c119
Parents: 765afd2
Author: Jonathan Eagles <je...@gmail.com>
Authored: Thu Apr 30 16:16:10 2015 -0500
Committer: Jonathan Eagles <je...@gmail.com>
Committed: Thu Apr 30 16:16:10 2015 -0500

----------------------------------------------------------------------
 CHANGES.txt                        |  1 +
 tez-tools/swimlanes/amlogparser.py | 20 ++++++++++----------
 tez-tools/swimlanes/swimlane.py    |  6 ++----
 3 files changed, 13 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/a02a5ea9/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index aa72320..5a5c21f 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -155,6 +155,7 @@ Release 0.6.1: Unreleased
 INCOMPATIBLE CHANGES
 
 ALL CHANGES:
+  TEZ-2390. tez-tools swimlane tool fails to parse large jobs >8K containers
   TEZ-2256. Avoid use of BufferTooSmallException to signal end of buffer in UnorderedPartitionedKVWriter
   TEZ-2380. Disable fall back to reading from timeline if timeline disabled.
   TEZ-2226. Disable writing history to timeline if domain creation fails.

http://git-wip-us.apache.org/repos/asf/tez/blob/a02a5ea9/tez-tools/swimlanes/amlogparser.py
----------------------------------------------------------------------
diff --git a/tez-tools/swimlanes/amlogparser.py b/tez-tools/swimlanes/amlogparser.py
index 02f4892..8ab8e29 100644
--- a/tez-tools/swimlanes/amlogparser.py
+++ b/tez-tools/swimlanes/amlogparser.py
@@ -17,8 +17,7 @@
 # under the License.
 #
 
-import os,sys,re,math,os.path
-from collections import defaultdict
+import sys,re
 from itertools import groupby
 from bz2 import BZ2File
 from gzip import GzipFile as GZFile
@@ -142,10 +141,15 @@ class Attempt(object):
 	def __init__(self, pair):
 		start = first(filter(lambda a: a.event == "TASK_ATTEMPT_STARTED", pair))
 		finish = first(filter(lambda a: a.event == "TASK_ATTEMPT_FINISHED", pair))
+		if start is None or finish is None:
+			print [start, finish];
 		self.raw = finish
-		self.dag = finish.dag
 		self.kvs = csv_kv(start.args)
-		self.kvs.update(csv_kv(finish.args))
+		if finish is not None:
+			self.dag = finish.dag
+			self.kvs.update(csv_kv(finish.args))
+			self.finish = (int)(self.kvs["finishTime"])
+			self.duration = (int)(self.kvs["timeTaken"])
 		self.name = self.kvs["taskAttemptId"]
 		self.task = self.name[:self.name.rfind("_")].replace("attempt","task")
 		(_, _, amid, dagid, vertexid, taskid, attemptid) = self.name.split("_")
@@ -153,8 +157,6 @@ class Attempt(object):
 		self.attemptnum = int(attemptid)
 		self.vertex = self.kvs["vertexName"]
 		self.start = (int)(self.kvs["startTime"])
-		self.finish = (int)(self.kvs["finishTime"])
-		self.duration = (int)(self.kvs["timeTaken"])
 		self.container = self.kvs["containerId"]
 		self.node = self.kvs["nodeId"]
 	def __repr__(self):
@@ -243,6 +245,7 @@ class AMLog(object):
 	def parse(self, l):		
 		if(l.find("[HISTORY]") != -1):
 			m = self.MAIN_RE.match(l)
+			print(m);
 			ts = m.group("ts")
 			dag = m.group("dag")
 			event = m.group("event")
@@ -250,14 +253,11 @@ class AMLog(object):
 			return AMRawEvent(ts, dag, event, args)
 
 def main(argv):
-	f = argv[0]
 	tree = AMLog(argv[0]).structure()
 	# AM -> dag -> vertex -> task -> attempt
 	# AM -> container
-	containers = set(tree.containers.keys())
-	timeto = lambda a: (a - tree.zero)
 	for d in tree.dags:
-		for a in d.attempts():			
+		for a in d.attempts():
 			print [a.vertex, a.name, a.container, a.start, a.finish]
 
 if __name__ == "__main__":

http://git-wip-us.apache.org/repos/asf/tez/blob/a02a5ea9/tez-tools/swimlanes/swimlane.py
----------------------------------------------------------------------
diff --git a/tez-tools/swimlanes/swimlane.py b/tez-tools/swimlanes/swimlane.py
index b739b1e..dc8dc6f 100644
--- a/tez-tools/swimlanes/swimlane.py
+++ b/tez-tools/swimlanes/swimlane.py
@@ -17,10 +17,9 @@
 # under the License.
 #
 
-import os,sys,re,math,os.path
+import sys,math,os.path
 import StringIO
 from amlogparser import AMLog
-import random
 from getopt import getopt
 
 class ColourManager(object):
@@ -133,7 +132,7 @@ def main(argv):
 	svg = SVGHelper(x+2*marginRight+256, y+2*marginTop)
 	a = marginTop
 	svg.text(x/2, 32, log.name, style="font-size: 32px; text-anchor: middle")	
-	containerMap = dict(zip(list(lanes), xrange(8192)))
+	containerMap = dict(zip(list(lanes), xrange(len(lanes))))
 	svg.text(marginRight - 16, marginTop - 32, "Container ID", "text-anchor:end; font-size: 16px;")
 	# draw a grid
 	for l in lanes:
@@ -193,7 +192,6 @@ def main(argv):
 			percentX = finishes[int(len(finishes)*fraction)]
 			svg.line(marginRight+xdomain(percentX), marginTop, marginRight+xdomain(percentX), y+marginTop, style="stroke: red")
 			svg.text(marginRight+xdomain(percentX), y+marginTop+12, "%d%% (%0.1fs)" % (int(fraction*100), (percentX - dag.start)/1000.0), style="font-size:12px; text-anchor: middle")
-	prefix = lambda a: (a.find(".") == -1 and a) or (a[:a.find(".")])
 	out.write(svg.flush())
 	out.close()
 


[38/50] [abbrv] tez git commit: TEZ-2139. Update version to 0.7.0-TEZ-2003-SNAPSHOT. (sseth)

Posted by ss...@apache.org.
TEZ-2139. Update version to 0.7.0-TEZ-2003-SNAPSHOT. (sseth)


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

Branch: refs/heads/TEZ-2003
Commit: baf2f2486b274ce19ab0898e9bd1c568829aeb7d
Parents: 921bffc
Author: Siddharth Seth <ss...@apache.org>
Authored: Mon Feb 23 21:59:39 2015 -0800
Committer: Siddharth Seth <ss...@apache.org>
Committed: Wed May 6 00:18:07 2015 -0700

----------------------------------------------------------------------
 TEZ-2003-CHANGES.txt                                    | 1 +
 docs/pom.xml                                            | 2 +-
 pom.xml                                                 | 2 +-
 tez-api/pom.xml                                         | 2 +-
 tez-common/pom.xml                                      | 2 +-
 tez-dag/pom.xml                                         | 2 +-
 tez-dist/pom.xml                                        | 2 +-
 tez-examples/pom.xml                                    | 2 +-
 tez-ext-service-tests/pom.xml                           | 2 +-
 tez-mapreduce/pom.xml                                   | 2 +-
 tez-plugins/pom.xml                                     | 2 +-
 tez-plugins/tez-mbeans-resource-calculator/pom.xml      | 2 +-
 tez-plugins/tez-yarn-timeline-history-with-acls/pom.xml | 2 +-
 tez-plugins/tez-yarn-timeline-history/pom.xml           | 2 +-
 tez-runtime-internals/pom.xml                           | 2 +-
 tez-runtime-library/pom.xml                             | 2 +-
 tez-tests/pom.xml                                       | 2 +-
 tez-ui/pom.xml                                          | 2 +-
 18 files changed, 18 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/baf2f248/TEZ-2003-CHANGES.txt
----------------------------------------------------------------------
diff --git a/TEZ-2003-CHANGES.txt b/TEZ-2003-CHANGES.txt
index 4377f57..5c5fd8e 100644
--- a/TEZ-2003-CHANGES.txt
+++ b/TEZ-2003-CHANGES.txt
@@ -8,5 +8,6 @@ ALL CHANGES:
   TEZ-2125. Create a task communicator for local mode. Allow tasks to run in the AM.
   TEZ-2131. Add additional tests for tasks running in the AM.
   TEZ-2138. Fix minor bugs in adding default scheduler, getting launchers.
+  TEZ-2139. Update tez version to 0.7.0-TEZ-2003-SNAPSHOT.
 
 INCOMPATIBLE CHANGES:

http://git-wip-us.apache.org/repos/asf/tez/blob/baf2f248/docs/pom.xml
----------------------------------------------------------------------
diff --git a/docs/pom.xml b/docs/pom.xml
index a475c58..ded04a8 100644
--- a/docs/pom.xml
+++ b/docs/pom.xml
@@ -27,7 +27,7 @@
     <parent>
       <groupId>org.apache.tez</groupId>
       <artifactId>tez</artifactId>
-      <version>0.8.0-SNAPSHOT</version>
+      <version>0.8.0-TEZ-2003-SNAPSHOT</version>
     </parent>
     <artifactId>tez-docs</artifactId>
     <packaging>pom</packaging>

http://git-wip-us.apache.org/repos/asf/tez/blob/baf2f248/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index d1149fd..034041b 100644
--- a/pom.xml
+++ b/pom.xml
@@ -19,7 +19,7 @@
   <groupId>org.apache.tez</groupId>
   <artifactId>tez</artifactId>
   <packaging>pom</packaging>
-  <version>0.8.0-SNAPSHOT</version>
+  <version>0.8.0-TEZ-2003-SNAPSHOT</version>
   <name>tez</name>
 
   <licenses>

http://git-wip-us.apache.org/repos/asf/tez/blob/baf2f248/tez-api/pom.xml
----------------------------------------------------------------------
diff --git a/tez-api/pom.xml b/tez-api/pom.xml
index 0a5d570..46e8dd3 100644
--- a/tez-api/pom.xml
+++ b/tez-api/pom.xml
@@ -20,7 +20,7 @@
   <parent>
     <groupId>org.apache.tez</groupId>
     <artifactId>tez</artifactId>
-    <version>0.8.0-SNAPSHOT</version>
+    <version>0.8.0-TEZ-2003-SNAPSHOT</version>
   </parent>
   <artifactId>tez-api</artifactId>
 

http://git-wip-us.apache.org/repos/asf/tez/blob/baf2f248/tez-common/pom.xml
----------------------------------------------------------------------
diff --git a/tez-common/pom.xml b/tez-common/pom.xml
index c6922be..5e0ceef 100644
--- a/tez-common/pom.xml
+++ b/tez-common/pom.xml
@@ -20,7 +20,7 @@
   <parent>
     <groupId>org.apache.tez</groupId>
     <artifactId>tez</artifactId>
-    <version>0.8.0-SNAPSHOT</version>
+    <version>0.8.0-TEZ-2003-SNAPSHOT</version>
   </parent>
   <artifactId>tez-common</artifactId>
 

http://git-wip-us.apache.org/repos/asf/tez/blob/baf2f248/tez-dag/pom.xml
----------------------------------------------------------------------
diff --git a/tez-dag/pom.xml b/tez-dag/pom.xml
index cc005bb..4d2e9f5 100644
--- a/tez-dag/pom.xml
+++ b/tez-dag/pom.xml
@@ -19,7 +19,7 @@
   <parent>
     <groupId>org.apache.tez</groupId>
     <artifactId>tez</artifactId>
-    <version>0.8.0-SNAPSHOT</version>
+    <version>0.8.0-TEZ-2003-SNAPSHOT</version>
   </parent>
   <properties>
     <tez.component>tez-dag</tez.component>

http://git-wip-us.apache.org/repos/asf/tez/blob/baf2f248/tez-dist/pom.xml
----------------------------------------------------------------------
diff --git a/tez-dist/pom.xml b/tez-dist/pom.xml
index e78cb8e..e188fe2 100644
--- a/tez-dist/pom.xml
+++ b/tez-dist/pom.xml
@@ -21,7 +21,7 @@
   <parent>
     <groupId>org.apache.tez</groupId>
     <artifactId>tez</artifactId>
-    <version>0.8.0-SNAPSHOT</version>
+    <version>0.8.0-TEZ-2003-SNAPSHOT</version>
   </parent>
   <artifactId>tez-dist</artifactId>
 

http://git-wip-us.apache.org/repos/asf/tez/blob/baf2f248/tez-examples/pom.xml
----------------------------------------------------------------------
diff --git a/tez-examples/pom.xml b/tez-examples/pom.xml
index cd2df5c..d15e966 100644
--- a/tez-examples/pom.xml
+++ b/tez-examples/pom.xml
@@ -20,7 +20,7 @@
   <parent>
     <groupId>org.apache.tez</groupId>
     <artifactId>tez</artifactId>
-    <version>0.8.0-SNAPSHOT</version>
+    <version>0.8.0-TEZ-2003-SNAPSHOT</version>
   </parent>
 
   <artifactId>tez-examples</artifactId>

http://git-wip-us.apache.org/repos/asf/tez/blob/baf2f248/tez-ext-service-tests/pom.xml
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/pom.xml b/tez-ext-service-tests/pom.xml
index 37f68b1..1113341 100644
--- a/tez-ext-service-tests/pom.xml
+++ b/tez-ext-service-tests/pom.xml
@@ -20,7 +20,7 @@
   <parent>
     <artifactId>tez</artifactId>
     <groupId>org.apache.tez</groupId>
-    <version>0.7.0-SNAPSHOT</version>
+    <version>0.8.0-TEZ-2003-SNAPSHOT</version>
   </parent>
 
   <!-- TODO TEZ-2003 Merge this into the tez-tests module -->

http://git-wip-us.apache.org/repos/asf/tez/blob/baf2f248/tez-mapreduce/pom.xml
----------------------------------------------------------------------
diff --git a/tez-mapreduce/pom.xml b/tez-mapreduce/pom.xml
index 300f781..af8bc8b 100644
--- a/tez-mapreduce/pom.xml
+++ b/tez-mapreduce/pom.xml
@@ -20,7 +20,7 @@
   <parent>
     <groupId>org.apache.tez</groupId>
     <artifactId>tez</artifactId>
-    <version>0.8.0-SNAPSHOT</version>
+    <version>0.8.0-TEZ-2003-SNAPSHOT</version>
   </parent>
   <artifactId>tez-mapreduce</artifactId>
 

http://git-wip-us.apache.org/repos/asf/tez/blob/baf2f248/tez-plugins/pom.xml
----------------------------------------------------------------------
diff --git a/tez-plugins/pom.xml b/tez-plugins/pom.xml
index 9b2a4cb..0b148ec 100644
--- a/tez-plugins/pom.xml
+++ b/tez-plugins/pom.xml
@@ -21,7 +21,7 @@
   <parent>
     <groupId>org.apache.tez</groupId>
     <artifactId>tez</artifactId>
-    <version>0.8.0-SNAPSHOT</version>
+    <version>0.8.0-TEZ-2003-SNAPSHOT</version>
   </parent>
   <artifactId>tez-plugins</artifactId>
   <packaging>pom</packaging>

http://git-wip-us.apache.org/repos/asf/tez/blob/baf2f248/tez-plugins/tez-mbeans-resource-calculator/pom.xml
----------------------------------------------------------------------
diff --git a/tez-plugins/tez-mbeans-resource-calculator/pom.xml b/tez-plugins/tez-mbeans-resource-calculator/pom.xml
index 4be7f28..5c828a4 100644
--- a/tez-plugins/tez-mbeans-resource-calculator/pom.xml
+++ b/tez-plugins/tez-mbeans-resource-calculator/pom.xml
@@ -19,7 +19,7 @@
   <parent>
     <groupId>org.apache.tez</groupId>
     <artifactId>tez-plugins</artifactId>
-    <version>0.8.0-SNAPSHOT</version>
+    <version>0.8.0-TEZ-2003-SNAPSHOT</version>
   </parent>
   <artifactId>tez-mbeans-resource-calculator</artifactId>
 

http://git-wip-us.apache.org/repos/asf/tez/blob/baf2f248/tez-plugins/tez-yarn-timeline-history-with-acls/pom.xml
----------------------------------------------------------------------
diff --git a/tez-plugins/tez-yarn-timeline-history-with-acls/pom.xml b/tez-plugins/tez-yarn-timeline-history-with-acls/pom.xml
index aeae5cf..761bc10 100644
--- a/tez-plugins/tez-yarn-timeline-history-with-acls/pom.xml
+++ b/tez-plugins/tez-yarn-timeline-history-with-acls/pom.xml
@@ -19,7 +19,7 @@
   <parent>
     <groupId>org.apache.tez</groupId>
     <artifactId>tez-plugins</artifactId>
-    <version>0.8.0-SNAPSHOT</version>
+    <version>0.8.0-TEZ-2003-SNAPSHOT</version>
   </parent>
   <artifactId>tez-yarn-timeline-history-with-acls</artifactId>
 

http://git-wip-us.apache.org/repos/asf/tez/blob/baf2f248/tez-plugins/tez-yarn-timeline-history/pom.xml
----------------------------------------------------------------------
diff --git a/tez-plugins/tez-yarn-timeline-history/pom.xml b/tez-plugins/tez-yarn-timeline-history/pom.xml
index ca76988..1e40329 100644
--- a/tez-plugins/tez-yarn-timeline-history/pom.xml
+++ b/tez-plugins/tez-yarn-timeline-history/pom.xml
@@ -19,7 +19,7 @@
   <parent>
     <groupId>org.apache.tez</groupId>
     <artifactId>tez-plugins</artifactId>
-    <version>0.8.0-SNAPSHOT</version>
+    <version>0.8.0-TEZ-2003-SNAPSHOT</version>
   </parent>
   <artifactId>tez-yarn-timeline-history</artifactId>
 

http://git-wip-us.apache.org/repos/asf/tez/blob/baf2f248/tez-runtime-internals/pom.xml
----------------------------------------------------------------------
diff --git a/tez-runtime-internals/pom.xml b/tez-runtime-internals/pom.xml
index 3756fa9..b6dee8c 100644
--- a/tez-runtime-internals/pom.xml
+++ b/tez-runtime-internals/pom.xml
@@ -20,7 +20,7 @@
   <parent>
     <groupId>org.apache.tez</groupId>
     <artifactId>tez</artifactId>
-    <version>0.8.0-SNAPSHOT</version>
+    <version>0.8.0-TEZ-2003-SNAPSHOT</version>
   </parent>
   <artifactId>tez-runtime-internals</artifactId>
 

http://git-wip-us.apache.org/repos/asf/tez/blob/baf2f248/tez-runtime-library/pom.xml
----------------------------------------------------------------------
diff --git a/tez-runtime-library/pom.xml b/tez-runtime-library/pom.xml
index 03e0ec3..8b566df 100644
--- a/tez-runtime-library/pom.xml
+++ b/tez-runtime-library/pom.xml
@@ -20,7 +20,7 @@
   <parent>
     <groupId>org.apache.tez</groupId>
     <artifactId>tez</artifactId>
-    <version>0.8.0-SNAPSHOT</version>
+    <version>0.8.0-TEZ-2003-SNAPSHOT</version>
   </parent>
   <artifactId>tez-runtime-library</artifactId>
 

http://git-wip-us.apache.org/repos/asf/tez/blob/baf2f248/tez-tests/pom.xml
----------------------------------------------------------------------
diff --git a/tez-tests/pom.xml b/tez-tests/pom.xml
index 91bc753..530ab77 100644
--- a/tez-tests/pom.xml
+++ b/tez-tests/pom.xml
@@ -20,7 +20,7 @@
   <parent>
     <groupId>org.apache.tez</groupId>
     <artifactId>tez</artifactId>
-    <version>0.8.0-SNAPSHOT</version>
+    <version>0.8.0-TEZ-2003-SNAPSHOT</version>
   </parent>
   <artifactId>tez-tests</artifactId>
 

http://git-wip-us.apache.org/repos/asf/tez/blob/baf2f248/tez-ui/pom.xml
----------------------------------------------------------------------
diff --git a/tez-ui/pom.xml b/tez-ui/pom.xml
index bfc6f01..8ecf7d0 100644
--- a/tez-ui/pom.xml
+++ b/tez-ui/pom.xml
@@ -21,7 +21,7 @@
   <parent>
     <groupId>org.apache.tez</groupId>
     <artifactId>tez</artifactId>
-    <version>0.8.0-SNAPSHOT</version>
+    <version>0.8.0-TEZ-2003-SNAPSHOT</version>
   </parent>
   <artifactId>tez-ui</artifactId>
   <packaging>war</packaging>


[09/50] [abbrv] tez git commit: TEZ-2397. Translation of LocalResources via Tez plan serialization can be lossy. (Siddharth Seth via hitesh)

Posted by ss...@apache.org.
TEZ-2397. Translation of LocalResources via Tez plan serialization can be lossy. (Siddharth Seth via hitesh)


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

Branch: refs/heads/TEZ-2003
Commit: c924e8a25d61be50e8c99f721108772b1c97c326
Parents: c96eed3
Author: Hitesh Shah <hi...@apache.org>
Authored: Fri May 1 11:46:55 2015 -0700
Committer: Hitesh Shah <hi...@apache.org>
Committed: Fri May 1 11:46:55 2015 -0700

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../apache/tez/dag/api/DagTypeConverters.java   | 25 ++++++++------
 .../tez/dag/api/TestDagTypeConverters.java      | 35 ++++++++++++++++++++
 3 files changed, 51 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/c924e8a2/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index fe90418..7c718ed 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -332,6 +332,7 @@ TEZ-UI CHANGES (TEZ-8):
 Release 0.5.4: Unreleased
 
 ALL CHANGES:
+  TEZ-2397. Translation of LocalResources via Tez plan serialization can be lossy.
   TEZ-2221. VertexGroup name should be unqiue
   TEZ-1521. VertexDataMovementEventsGeneratedEvent may be logged twice in recovery log
   TEZ-2348. EOF exception during UnorderedKVReader.next().

http://git-wip-us.apache.org/repos/asf/tez/blob/c924e8a2/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
index b4185b1..4dc7b38 100644
--- 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
@@ -19,6 +19,7 @@ package org.apache.tez.dag.api;
 
 import java.io.DataOutputStream;
 import java.io.IOException;
+import java.net.URISyntaxException;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.HashMap;
@@ -212,16 +213,20 @@ public class DagTypeConverters {
     return VertexLocationHint.create(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
-    return resource.getScheme() + "://" + resource.getHost()
-        + ":" + resource.getPort() + resource.getFile();
+    Path p;
+    try {
+      p = ConverterUtils.getPathFromYarnURL(resource);
+    } catch (URISyntaxException e) {
+      throw new TezUncheckedException("Unable to translate resource: " + resource + " to Path");
+    }
+    String urlString = p.toString();
+    return urlString;
+  }
+
+  public static URL convertToYarnURL(String pathString) {
+    Path path = new Path(pathString);
+    return ConverterUtils.getYarnUrlFromPath(path);
   }
 
   public static Map<String, LocalResource> createLocalResourceMapFromDAGPlan(
@@ -235,7 +240,7 @@ public class DagTypeConverters {
       if(res.hasPattern()){
         r.setPattern(res.getPattern());
       }
-      r.setResource(ConverterUtils.getYarnUrlFromPath(new Path(res.getUri())));  // see above notes on HDFS URL handling
+      r.setResource(convertToYarnURL(res.getUri()));
       r.setSize(res.getSize());
       r.setTimestamp(res.getTimeStamp());
       r.setType(DagTypeConverters.convertFromDAGPlan(res.getType()));

http://git-wip-us.apache.org/repos/asf/tez/blob/c924e8a2/tez-api/src/test/java/org/apache/tez/dag/api/TestDagTypeConverters.java
----------------------------------------------------------------------
diff --git a/tez-api/src/test/java/org/apache/tez/dag/api/TestDagTypeConverters.java b/tez-api/src/test/java/org/apache/tez/dag/api/TestDagTypeConverters.java
index 5d07eea..51b179a 100644
--- a/tez-api/src/test/java/org/apache/tez/dag/api/TestDagTypeConverters.java
+++ b/tez-api/src/test/java/org/apache/tez/dag/api/TestDagTypeConverters.java
@@ -21,6 +21,10 @@ package org.apache.tez.dag.api;
 import java.io.IOException;
 
 import java.nio.ByteBuffer;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.yarn.api.records.URL;
+import org.apache.hadoop.yarn.util.ConverterUtils;
 import org.apache.tez.common.TezCommonUtils;
 import org.apache.tez.dag.api.records.DAGProtos.TezEntityDescriptorProto;
 import org.junit.Assert;
@@ -54,4 +58,35 @@ public class TestDagTypeConverters {
     Assert.assertEquals(historytext, actualHistoryText);
   }
 
+  @Test(timeout = 5000)
+  public void testYarnPathTranslation() {
+    // Without port
+    String p1String = "hdfs://mycluster/file";
+    Path p1Path = new Path(p1String);
+    // Users would translate this via this mechanic.
+    URL lr1Url = ConverterUtils.getYarnUrlFromPath(p1Path);
+    // Serialize to dag plan.
+    String p1StringSerialized = DagTypeConverters.convertToDAGPlan(lr1Url);
+    // Deserialize
+    URL lr1UrlDeserialized = DagTypeConverters.convertToYarnURL(p1StringSerialized);
+    Assert.assertEquals("mycluster", lr1UrlDeserialized.getHost());
+    Assert.assertEquals("/file", lr1UrlDeserialized.getFile());
+    Assert.assertEquals("hdfs", lr1UrlDeserialized.getScheme());
+
+
+    // With port
+    String p2String = "hdfs://mycluster:2311/file";
+    Path p2Path = new Path(p2String);
+    // Users would translate this via this mechanic.
+    URL lr2Url = ConverterUtils.getYarnUrlFromPath(p2Path);
+    // Serialize to dag plan.
+    String p2StringSerialized = DagTypeConverters.convertToDAGPlan(lr2Url);
+    // Deserialize
+    URL lr2UrlDeserialized = DagTypeConverters.convertToYarnURL(p2StringSerialized);
+    Assert.assertEquals("mycluster", lr2UrlDeserialized.getHost());
+    Assert.assertEquals("/file", lr2UrlDeserialized.getFile());
+    Assert.assertEquals("hdfs", lr2UrlDeserialized.getScheme());
+    Assert.assertEquals(2311, lr2UrlDeserialized.getPort());
+  }
+
 }


[08/50] [abbrv] tez git commit: TEZ-2237. Create project by-laws. (hitesh)

Posted by ss...@apache.org.
TEZ-2237. Create project by-laws. (hitesh)


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

Branch: refs/heads/TEZ-2003
Commit: c96eed3e2290bdbc0765e72cf2d3e3a06854a924
Parents: 1dd5f9f
Author: Hitesh Shah <hi...@apache.org>
Authored: Fri May 1 11:00:10 2015 -0700
Committer: Hitesh Shah <hi...@apache.org>
Committed: Fri May 1 11:00:10 2015 -0700

----------------------------------------------------------------------
 docs/src/site/markdown/by-laws.md | 113 +++++++++++++++++++++++++++++++++
 docs/src/site/site.xml            |   1 +
 2 files changed, 114 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/c96eed3e/docs/src/site/markdown/by-laws.md
----------------------------------------------------------------------
diff --git a/docs/src/site/markdown/by-laws.md b/docs/src/site/markdown/by-laws.md
new file mode 100644
index 0000000..9b23201
--- /dev/null
+++ b/docs/src/site/markdown/by-laws.md
@@ -0,0 +1,113 @@
+# Apache Tez Project Bylaws
+
+This document defines the bylaws under which the Apache Tez project operates. It defines the roles and responsibilities of the project, who may vote, how voting works, how conflicts are resolved, etc.
+
+Tez is a project of the [Apache Software Foundation]. The Foundation holds the copyright on Apache code including the code in the Tez codebase. The [Foundation FAQ] explains the operation and background of the foundation.
+
+Tez is typical of Apache projects in that it operates under a set of principles, known collectively as the "Apache Way". If you are new to Apache development, please refer to the [Incubator project] for more information on how Apache projects operate.
+
+##Roles and Responsibilities
+
+Apache projects define a [set of roles] with associated rights and responsibilities. These roles govern what tasks an individual may perform within the project. The roles are defined in the following sections:
+
+###Users
+
+The most important participants in the project are people who use our software. The majority of our developers start out as users and guide their development efforts from the user's perspective.
+
+Users contribute to the Apache projects by providing feedback to developers in the form of bug reports and feature suggestions. As well, users participate in the Apache community by helping other users on mailing lists and user support forums.
+
+###Developers
+
+All of the volunteers who are contributing time, code, documentation, or resources to the Tez Project. A developer that makes sustained, welcome contributions to the project may be invited to become a Committer, though the exact timing of such invitations depends on many factors.
+
+###Committers
+
+The project's Committers are responsible for the project's technical management. All committers have write access to the project's source repositories. Committers may cast binding votes on any technical discussion regarding the project.
+
+Committer access is by invitation only and must be approved by lazy consensus of the active PMC members. A Committer may request removal of their commit privileges by their own declaration. A committer will be considered "emeritus/inactive" by not contributing in any form to the project for over 1 year. An emeritus committer may request reinstatement of commit access from the PMC. Such reinstatement is subject to lazy consensus of active PMC members.
+
+Commit access can be revoked by a unanimous vote of all the active PMC members (except the committer in question if they are also a PMC member).
+
+All Apache committers are required to have a signed Contributor License Agreement ([CLA]) on file with the Apache Software Foundation. There is a [Committer FAQ] which provides more details on the requirements for Committers
+
+A committer who makes a sustained contribution to the project may be invited to become a member of the PMC. The form of contribution is not limited to code. It can also include code review, helping out users on the mailing lists, documentation, etc.
+
+###Project Management Committee
+
+The Project Management Committee (PMC) for Apache Tez was created by a resolution of the board of the Apache Software Foundation on 16th July, 2014. The PMC is responsible to the board and the ASF for the management and oversight of the Apache Tez codebase. The responsibilities of the PMC include:
+
+   - Deciding what is distributed as products of the Apache Tez project. In particular all releases must be approved by the PMC
+   - Maintaining the project's shared resources, including the codebase repository, mailing lists, websites.
+   - Speaking on behalf of the project.
+   - Resolving license disputes regarding products of the project
+   - Nominating new PMC members and committers
+   - Maintaining these bylaws and other guidelines of the project
+
+Membership of the PMC is by invitation only and must be approved by a lazy consensus of active PMC members. A PMC member is considered "emeritus/inactive" by not contributing in any form to the project for over one year. An emeritus PMC member may request reinstatement to the PMC. Such reinstatement is subject to lazy consensus of active PMC members. A PMC member may resign their membership from the PMC by their own declaration. Membership of the PMC can be revoked by an unanimous vote of all the active PMC members other than the member in question.
+
+The chair of the PMC is appointed by the ASF board. The chair is an office holder of the Apache Software Foundation (Vice President, Apache Tez) and has primary responsibility to the board for the management of the projects within the scope of the Tez PMC. The chair reports to the board quarterly on developments within the Tez project. The PMC may consider the position of PMC chair annually, and if supported by a successful vote to change the PMC chair, may recommend a new chair to the board. Ultimately, however, it is the board's responsibility who it chooses to appoint as the PMC chair.
+
+##Decision Making
+
+Within the Tez project, different types of decisions require different forms of approval. For example, the previous section describes several decisions which require "lazy consensus" approval. This section defines how voting is performed, the types of approvals, and which types of decision require which type of approval.
+
+###Voting
+
+Decisions regarding the project are made by votes on the primary project development mailing list (dev@tez.apache.org). Where necessary, PMC voting may take place on the private Tez PMC mailing list. Votes are clearly indicated by subject line starting with [VOTE]. Votes may contain multiple items for approval and these should be clearly separated. Voting is carried out by replying to the vote mail. Voting may take four flavours:
+
+|---|---|
+| +1 | "Yes," "Agree," or "the action should be performed." In general, this vote also indicates a willingness on the behalf of the voter in "making it happen" |
+| +0 | This vote indicates a willingness for the action under consideration to go ahead. The voter, however will not be able to help. |
+| -0 | This vote indicates that the voter does not, in general, agree with the proposed action but is not concerned enough to prevent the action going ahead. |
+| -1 | This is a negative vote. On issues where consensus is required, this vote counts as a veto. All vetoes must contain an explanation of why the veto is appropriate. Vetoes with no explanation are void. It may also be appropriate for a -1 vote to include an alternative course of action. |
+
+All participants in the Tez project are encouraged to show their agreement with or against a particular action by voting. For technical decisions, only the votes of active committers are binding. Non binding votes are still useful for those with binding votes to understand the perception of an action in the wider Tez community. For PMC decisions, only the votes of PMC members are binding.
+
+Voting can also be applied to changes made to the Tez codebase. These typically take the form of a veto (-1) in reply to the commit message sent when the commit is made.
+
+###Approvals
+
+These are the types of approvals that can be sought. Different actions require different types of approvals:
+
+|---|---|
+| Consensus | For this to pass, all voters with binding votes must vote and there can be no binding vetoes (-1). Consensus votes are rarely required due to the impracticality of getting all eligible voters to cast a vote. |
+| Lazy Consensus | Lazy consensus requires 3 binding +1 votes and no binding vetoes. |
+| Lazy Majority | A lazy majority vote requires 3 binding +1 votes and more binding +1 votes that -1 votes. |
+| Lazy Approval | An action with lazy approval requires at least 1 binding +1 vote unless a -1 vote is received, at which time, depending on the type of action, either lazy majority or lazy consensus approval must be obtained. |
+| 2/3 Majority | Some actions require a 2/3 majority of active committers or PMC members to pass. Such actions typically affect the foundation of the project (e.g. adopting a new codebase to replace an existing product). The higher threshold is designed to ensure such changes are strongly supported. To pass this vote requires at least 2/3 of binding vote holders to vote +1 |
+
+###Vetoes
+
+A valid, binding veto cannot be overruled. If a veto is cast, it must be accompanied by a valid reason explaining the reasons for the veto. The validity of a veto, if challenged, can be confirmed by anyone who has a binding vote. This does not necessarily signify agreement with the veto - merely that the veto is valid.
+
+If you disagree with a valid veto, you must lobby the person casting the veto to withdraw their veto. If a veto is not withdrawn, the action that has been vetoed must be reversed in a timely manner.
+
+###Actions
+
+This section describes the various actions which are undertaken within the project, the corresponding approval required for that action and those who have binding votes over the action.
+
+| Action | Description | Approval | Binding Votes |
+| ------ | ----------- | -------- | ------------- |
+| Code Change | A change made to a codebase of the project and committed by a committer. This includes source code, documentation, website content, etc. | Lazy approval and then Lazy consensus. | Active committers |
+| Major Feature/Change via a Branch Merge | A major change made to the codebase of the project done via a branch merge. | Lazy consensus | Active committers | 
+| Release Plan | Defines the timetable and actions for a release. The plan also nominates a Release Manager. | Lazy majority | Active committers |
+| Product Release | When a release of one of the project's products is ready, a vote is required to accept the release as an official release of the project. | Lazy Majority |   Active PMC members |
+| Adoption of New Codebase | When the codebase for an existing, released product is to be replaced with an alternative codebase. If such a vote fails to gain approval, the existing code base will continue. This also covers the creation of new sub-projects within the project. | 2/3 majority | Active committers |
+| New Committer | When a new committer is proposed for the project | Lazy consensus | Active PMC members |
+| New PMC Member | When a committer is proposed for the PMC | Lazy consensus | Active PMC members |
+| Committer Removal | When removal of commit privileges is sought. Note: Such actions will also be referred to the ASF board by the PMC chair | Consensus | Active PMC members (excluding the committer in question if a member of the PMC). |
+| PMC Member Removal | When removal of a PMC member is sought. Note: Such actions will also be referred to the ASF board by the PMC chair | Consensus | Active PMC members (excluding the member in question). |
+| Change to Project By-Laws | When a change is needed to the Project's By-Laws. | Lazy Consensus | Active PMC Members |
+| Change the PMC Chair | When the PMC Chair needs to be changed. | Lazy Consensus | Active PMC Members |
+
+###Voting Timeframes
+
+Votes are open for a period of a minimum of 3 days (excluding weekend days) to allow all active voters time to consider the vote. For any votes requiring full consensus or a 2/3 majority, the vote should remain open for a minimum of 1 week. Votes relating to code changes are not subject to a strict timetable but should be made as timely as possible.
+
+
+[Apache Software Foundation]: http://www.apache.org/foundation/
+[Incubator project]: http://incubator.apache.org/
+[Foundation FAQ]: http://www.apache.org/foundation/faq.html
+[Committer FAQ]: http://www.apache.org/dev/committers.html
+[CLA]: http://www.apache.org/licenses/icla.txt
+[set of roles]: http://www.apache.org/foundation/how-it-works.html#roles

http://git-wip-us.apache.org/repos/asf/tez/blob/c96eed3e/docs/src/site/site.xml
----------------------------------------------------------------------
diff --git a/docs/src/site/site.xml b/docs/src/site/site.xml
index dd59535..a29ddea 100644
--- a/docs/src/site/site.xml
+++ b/docs/src/site/site.xml
@@ -108,6 +108,7 @@
       <item name="Mailing Lists" href="mail-lists.html"/>
       <item name="Issue Tracking" href="issue-tracking.html"/>
       <item name="Project Team" href="team-list.html"/>
+      <item name="Project By-Laws" href="by-laws.html"/>
       <item name="User Meetup Group" href="http://www.meetup.com/Apache-Tez-User-Group"/>
       <item name="Tez Wiki" href="https://cwiki.apache.org/confluence/display/TEZ"/>
     </menu>


[35/50] [abbrv] tez git commit: TEZ-2122. Setup pluggable components at AM/Vertex level. (sseth)

Posted by ss...@apache.org.
TEZ-2122. Setup pluggable components at AM/Vertex level. (sseth)


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

Branch: refs/heads/TEZ-2003
Commit: adab48fa4bcff32da24272023000a529db5727c5
Parents: fb7c0d6
Author: Siddharth Seth <ss...@apache.org>
Authored: Thu Feb 19 14:59:18 2015 -0800
Committer: Siddharth Seth <ss...@apache.org>
Committed: Wed May 6 00:13:29 2015 -0700

----------------------------------------------------------------------
 TEZ-2003-CHANGES.txt                            |   1 +
 .../apache/tez/dag/api/TezConfiguration.java    |  29 +++-
 .../org/apache/tez/dag/api/TezConstants.java    |   3 +
 .../java/org/apache/tez/dag/app/AppContext.java |   4 +
 .../org/apache/tez/dag/app/DAGAppMaster.java    | 121 +++++++++++++-
 .../dag/app/TaskAttemptListenerImpTezDag.java   |  77 +++++----
 .../java/org/apache/tez/dag/app/dag/Vertex.java |   4 +
 .../tez/dag/app/dag/impl/TaskAttemptImpl.java   |   8 +-
 .../apache/tez/dag/app/dag/impl/VertexImpl.java |  47 ++++++
 .../app/launcher/ContainerLauncherRouter.java   |  93 +++++++----
 .../app/rm/AMSchedulerEventTALaunchRequest.java |  22 ++-
 .../dag/app/rm/TaskSchedulerEventHandler.java   | 163 +++++++++++--------
 .../apache/tez/dag/app/MockDAGAppMaster.java    |   5 +-
 .../app/TestTaskAttemptListenerImplTezDag.java  |  19 +--
 .../tez/dag/app/rm/TestContainerReuse.java      |   2 +-
 .../app/rm/TestTaskSchedulerEventHandler.java   |  12 +-
 .../dag/app/rm/TestTaskSchedulerHelpers.java    |  18 +-
 .../tez/tests/TestExternalTezServices.java      |  19 ++-
 18 files changed, 458 insertions(+), 189 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/adab48fa/TEZ-2003-CHANGES.txt
----------------------------------------------------------------------
diff --git a/TEZ-2003-CHANGES.txt b/TEZ-2003-CHANGES.txt
index 1cd74a4..4bfe08f 100644
--- a/TEZ-2003-CHANGES.txt
+++ b/TEZ-2003-CHANGES.txt
@@ -3,5 +3,6 @@ ALL CHANGES:
   TEZ-2006. Task communication plane needs to be pluggable.
   TEZ-2090. Add tests for jobs running in external services.
   TEZ-2117. Add a manager for ContainerLaunchers running in the AM.
+  TEZ-2122. Setup pluggable components at AM/Vertex level.
 
 INCOMPATIBLE CHANGES:

http://git-wip-us.apache.org/repos/asf/tez/blob/adab48fa/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java b/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java
index 533febb..d999dcf 100644
--- a/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java
+++ b/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java
@@ -1168,13 +1168,36 @@ public class TezConfiguration extends Configuration {
       + "tez-ui.webservice.enable";
   public static final boolean TEZ_AM_WEBSERVICE_ENABLE_DEFAULT = true;
 
+  /** defaults container-launcher for the specific vertex */
   @ConfigurationScope(Scope.VERTEX)
-  public static final String TEZ_AM_CONTAINER_LAUNCHER_CLASS = TEZ_AM_PREFIX + "container-launcher.class";
+  public static final String TEZ_AM_VERTEX_CONTAINER_LAUNCHER_NAME = TEZ_AM_PREFIX + "vertex.container-launcher.name";
+  /** defaults task-scheduler for the specific vertex */
   @ConfigurationScope(Scope.VERTEX)
-  public static final String TEZ_AM_TASK_SCHEDULER_CLASS = TEZ_AM_PREFIX + "task-scheduler.class";
+  public static final String TEZ_AM_VERTEX_TASK_SCHEDULER_NAME = TEZ_AM_PREFIX + "vertex.task-scheduler.name";
+  /** defaults task-communicator for the specific vertex */
   @ConfigurationScope(Scope.VERTEX)
-  public static final String TEZ_AM_TASK_COMMUNICATOR_CLASS = TEZ_AM_PREFIX + "task-communicator.class";
+  public static final String TEZ_AM_VERTEX_TASK_COMMUNICATOR_NAME = TEZ_AM_PREFIX + "vertex.task-communicator.name";
 
+  /** Comma separated list of named container-launcher classes running in the AM.
+   * The format for each entry is NAME:CLASSNAME, except for tez default which is specified as Tez
+   * e.g. Tez, ExtService:org.apache.ExtLauncherClasss
+   * */
+  @ConfigurationScope(Scope.AM)
+  public static final String TEZ_AM_CONTAINER_LAUNCHERS = TEZ_AM_PREFIX + "container-launchers";
+
+  /** Comma separated list of task-schedulers classes running in the AM.
+   * The format for each entry is NAME:CLASSNAME, except for tez default which is specified as Tez
+   * e.g. Tez, ExtService:org.apache.ExtSchedulerClasss
+   */
+  @ConfigurationScope(Scope.AM)
+  public static final String TEZ_AM_TASK_SCHEDULERS = TEZ_AM_PREFIX + "task-schedulers";
+
+  /** Comma separated list of task-communicators classes running in the AM.
+   * The format for each entry is NAME:CLASSNAME, except for tez default which is specified as Tez
+   * e.g. Tez, ExtService:org.apache.ExtTaskCommClass
+   * */
+   @ConfigurationScope(Scope.AM)
+  public static final String TEZ_AM_TASK_COMMUNICATORS = TEZ_AM_PREFIX + "task-communicators";
 
   // TODO only validate property here, value can also be validated if necessary
   public static void validateProperty(String property, Scope usedScope) {

http://git-wip-us.apache.org/repos/asf/tez/blob/adab48fa/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
index bc4208f..3b07c59 100644
--- 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
@@ -102,4 +102,7 @@ public class TezConstants {
   /// Version-related Environment variables
   public static final String TEZ_CLIENT_VERSION_ENV = "TEZ_CLIENT_VERSION";
 
+
+  public static final String TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT = "Tez";
+  public static final String TEZ_AM_SERVICE_PLUGINS_LOCAL_MODE_NAME_DEFAULT = "TezLocal";
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/adab48fa/tez-dag/src/main/java/org/apache/tez/dag/app/AppContext.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/AppContext.java b/tez-dag/src/main/java/org/apache/tez/dag/app/AppContext.java
index 37f7624..9463226 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/AppContext.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/AppContext.java
@@ -109,4 +109,8 @@ public interface AppContext {
   String getAMUser();
 
   Credentials getAppCredentials();
+
+  public Integer getTaskCommunicatorIdentifier(String name);
+  public Integer getTaskScheduerIdentifier(String name);
+  public Integer getContainerLauncherIdentifier(String name);
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/adab48fa/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 0f4d812..6814cda 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
@@ -54,6 +54,8 @@ import java.util.concurrent.locks.ReentrantReadWriteLock;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
+import com.google.common.collect.BiMap;
+import com.google.common.collect.HashBiMap;
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.cli.GnuParser;
 import org.apache.commons.cli.Options;
@@ -266,7 +268,12 @@ public class DAGAppMaster extends AbstractService {
   
   private ExecutorService rawExecutor;
   private ListeningExecutorService execService;
-  
+
+  // TODO May not need to be a bidi map
+  private final BiMap<String, Integer> taskSchedulers = HashBiMap.create();
+  private final BiMap<String, Integer> containerLaunchers = HashBiMap.create();
+  private final BiMap<String, Integer> taskCommunicators = HashBiMap.create();
+
   /**
    * set of already executed dag names.
    */
@@ -370,6 +377,29 @@ public class DAGAppMaster extends AbstractService {
     this.isLocal = conf.getBoolean(TezConfiguration.TEZ_LOCAL_MODE,
         TezConfiguration.TEZ_LOCAL_MODE_DEFAULT);
 
+    String tezDefaultClassIdentifier =
+        isLocal ? TezConstants.TEZ_AM_SERVICE_PLUGINS_LOCAL_MODE_NAME_DEFAULT :
+            TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT;
+
+    String[] taskSchedulerClassIdentifiers = parsePlugins(taskSchedulers,
+        conf.getTrimmedStrings(TezConfiguration.TEZ_AM_TASK_SCHEDULERS,
+            tezDefaultClassIdentifier),
+        TezConfiguration.TEZ_AM_TASK_SCHEDULERS);
+
+    String[] containerLauncherClassIdentifiers = parsePlugins(containerLaunchers,
+        conf.getTrimmedStrings(TezConfiguration.TEZ_AM_CONTAINER_LAUNCHERS,
+            tezDefaultClassIdentifier),
+        TezConfiguration.TEZ_AM_CONTAINER_LAUNCHERS);
+
+    String[] taskCommunicatorClassIdentifiers = parsePlugins(taskCommunicators,
+        conf.getTrimmedStrings(TezConfiguration.TEZ_AM_TASK_COMMUNICATORS,
+            tezDefaultClassIdentifier),
+        TezConfiguration.TEZ_AM_TASK_COMMUNICATORS);
+
+    LOG.info(buildPluginComponentLog(taskSchedulerClassIdentifiers, taskSchedulers, "TaskSchedulers"));
+    LOG.info(buildPluginComponentLog(containerLauncherClassIdentifiers, containerLaunchers, "ContainerLaunchers"));
+    LOG.info(buildPluginComponentLog(taskCommunicatorClassIdentifiers, taskCommunicators, "TaskCommunicators"));
+
     boolean disableVersionCheck = conf.getBoolean(
         TezConfiguration.TEZ_AM_DISABLE_CLIENT_VERSION_CHECK,
         TezConfiguration.TEZ_AM_DISABLE_CLIENT_VERSION_CHECK_DEFAULT);
@@ -435,7 +465,7 @@ public class DAGAppMaster extends AbstractService {
 
     //service to handle requests to TaskUmbilicalProtocol
     taskAttemptListener = createTaskAttemptListener(context,
-        taskHeartbeatHandler, containerHeartbeatHandler);
+        taskHeartbeatHandler, containerHeartbeatHandler, taskCommunicatorClassIdentifiers);
     addIfService(taskAttemptListener, true);
 
     containerSignatureMatcher = createContainerSignatureMatcher();
@@ -482,7 +512,8 @@ public class DAGAppMaster extends AbstractService {
     }
 
     this.taskSchedulerEventHandler = new TaskSchedulerEventHandler(context,
-        clientRpcServer, dispatcher.getEventHandler(), containerSignatureMatcher, webUIService);
+        clientRpcServer, dispatcher.getEventHandler(), containerSignatureMatcher, webUIService,
+        taskSchedulerClassIdentifiers);
     addIfService(taskSchedulerEventHandler, true);
 
     if (enableWebUIService()) {
@@ -500,7 +531,7 @@ public class DAGAppMaster extends AbstractService {
         taskSchedulerEventHandler);
     addIfServiceDependency(taskSchedulerEventHandler, clientRpcServer);
 
-    this.containerLauncherRouter = createContainerLauncherRouter(conf);
+    this.containerLauncherRouter = createContainerLauncherRouter(conf, containerLauncherClassIdentifiers);
     addIfService(containerLauncherRouter, true);
     dispatcher.register(NMCommunicatorEventType.class, containerLauncherRouter);
 
@@ -1007,9 +1038,9 @@ public class DAGAppMaster extends AbstractService {
   }
 
   protected TaskAttemptListener createTaskAttemptListener(AppContext context,
-      TaskHeartbeatHandler thh, ContainerHeartbeatHandler chh) {
+      TaskHeartbeatHandler thh, ContainerHeartbeatHandler chh, String[] taskCommunicatorClasses) {
     TaskAttemptListener lis =
-        new TaskAttemptListenerImpTezDag(context, thh, chh,jobTokenSecretManager);
+        new TaskAttemptListenerImpTezDag(context, thh, chh,jobTokenSecretManager, taskCommunicatorClasses);
     return lis;
   }
 
@@ -1030,9 +1061,9 @@ public class DAGAppMaster extends AbstractService {
     return chh;
   }
 
-  protected ContainerLauncherRouter createContainerLauncherRouter(Configuration conf) throws
+  protected ContainerLauncherRouter createContainerLauncherRouter(Configuration conf, String []containerLauncherClasses) throws
       UnknownHostException {
-    return  new ContainerLauncherRouter(conf, isLocal, context, taskAttemptListener, workingDirectory);
+    return  new ContainerLauncherRouter(conf, context, taskAttemptListener, workingDirectory, containerLauncherClasses);
 
   }
 
@@ -1459,6 +1490,21 @@ public class DAGAppMaster extends AbstractService {
     }
 
     @Override
+    public Integer getTaskCommunicatorIdentifier(String name) {
+      return taskCommunicators.get(name);
+    }
+
+    @Override
+    public Integer getTaskScheduerIdentifier(String name) {
+      return taskSchedulers.get(name);
+    }
+
+    @Override
+    public Integer getContainerLauncherIdentifier(String name) {
+      return taskCommunicators.get(name);
+    }
+
+    @Override
     public Map<ApplicationAccessType, String> getApplicationACLs() {
       if (getServiceState() != STATE.STARTED) {
         throw new TezUncheckedException(
@@ -2233,4 +2279,63 @@ public class DAGAppMaster extends AbstractService {
     return amConf.getBoolean(TezConfiguration.TEZ_AM_WEBSERVICE_ENABLE,
         TezConfiguration.TEZ_AM_WEBSERVICE_ENABLE_DEFAULT);
   }
+
+  // Tez default classnames are populated as TezConfiguration.TEZ_AM_SERVICE_PLUGINS_DEFAULT
+  private String[] parsePlugins(BiMap<String, Integer> pluginMap, String[] pluginStrings,
+                                   String context) {
+    Preconditions.checkState(pluginStrings != null && pluginStrings.length > 0,
+        "Plugin strings should not be null or empty: " + context);
+
+    String[] classNames = new String[pluginStrings.length];
+
+    int index = 0;
+    for (String pluginString : pluginStrings) {
+
+      String className;
+      String identifierString;
+
+      Preconditions.checkState(pluginString != null && !pluginString.isEmpty(),
+          "Plugin string: " + pluginString + " should not be null or empty");
+      if (pluginString.equals(TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT) ||
+          pluginString.equals(TezConstants.TEZ_AM_SERVICE_PLUGINS_LOCAL_MODE_NAME_DEFAULT)) {
+        // Kind of ugly, but Tez internal routing is encoded via a String instead of classnames.
+        // Individual components - TaskComm, Scheduler, Launcher deal with actual classname translation,
+        // and avoid reflection.
+        identifierString = pluginString;
+        className = pluginString;
+      } else {
+        String[] parts = pluginString.split(":");
+        Preconditions.checkState(
+            parts.length == 2 && parts[0] != null && !parts[0].isEmpty() && parts[1] != null &&
+                !parts[1].isEmpty(),
+            "Invalid configuration string for " + context + ": " + pluginString);
+        Preconditions.checkState(
+            !parts[0].equals(TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT) &&
+                !parts[0].equals(TezConstants.TEZ_AM_SERVICE_PLUGINS_LOCAL_MODE_NAME_DEFAULT),
+            "Identifier cannot be " + TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT + " or " +
+                TezConstants.TEZ_AM_SERVICE_PLUGINS_LOCAL_MODE_NAME_DEFAULT + " for " +
+                pluginString);
+        identifierString = parts[0];
+        className = parts[1];
+      }
+      pluginMap.put(identifierString, index);
+      classNames[index] = className;
+    }
+    return classNames;
+  }
+
+  String buildPluginComponentLog(String[] classIdentifiers, BiMap<String, Integer> map,
+                                 String component) {
+    StringBuilder sb = new StringBuilder();
+    sb.append("AM Level configured ").append(component).append(": ");
+    for (int i = 0; i < classIdentifiers.length; i++) {
+      sb.append("[").append(i).append(":").append(map.inverse().get(i)).append(":")
+          .append(taskSchedulers.inverse().get(i)).append(
+          "]");
+      if (i != classIdentifiers.length - 1) {
+        sb.append(",");
+      }
+    }
+    return sb.toString();
+  }
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/adab48fa/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 c34723a..fc4d787 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
@@ -38,7 +38,6 @@ import org.apache.tez.runtime.api.impl.EventType;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.service.AbstractService;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
@@ -47,7 +46,7 @@ import org.apache.tez.common.ReflectionUtils;
 import org.apache.tez.dag.api.TaskCommunicator;
 import org.apache.tez.dag.api.TaskCommunicatorContext;
 import org.apache.tez.dag.api.TaskHeartbeatResponse;
-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.hadoop.yarn.api.records.ContainerId;
@@ -57,7 +56,6 @@ import org.apache.tez.dag.app.dag.DAG;
 import org.apache.tez.dag.app.dag.Task;
 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.TaskSchedulerService;
 import org.apache.tez.dag.app.rm.container.AMContainerTask;
 import org.apache.tez.dag.records.TezTaskAttemptID;
 import org.apache.tez.dag.records.TezVertexID;
@@ -74,7 +72,7 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
       .getLogger(TaskAttemptListenerImpTezDag.class);
 
   private final AppContext context;
-  private TaskCommunicator taskCommunicator;
+  private final TaskCommunicator[] taskCommunicators;
 
   protected final TaskHeartbeatHandler taskHeartbeatHandler;
   protected final ContainerHeartbeatHandler containerHeartbeatHandler;
@@ -100,28 +98,52 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
   public TaskAttemptListenerImpTezDag(AppContext context,
                                       TaskHeartbeatHandler thh, ContainerHeartbeatHandler chh,
                                       // TODO TEZ-2003 pre-merge. Remove reference to JobTokenSecretManager.
-                                      JobTokenSecretManager jobTokenSecretManager) {
+                                      JobTokenSecretManager jobTokenSecretManager,
+                                      String [] taskCommunicatorClassIdentifiers) {
     super(TaskAttemptListenerImpTezDag.class.getName());
     this.context = context;
     this.taskHeartbeatHandler = thh;
     this.containerHeartbeatHandler = chh;
-    this.taskCommunicator = new TezTaskCommunicatorImpl(this);
+    if (taskCommunicatorClassIdentifiers == null || taskCommunicatorClassIdentifiers.length == 0) {
+      taskCommunicatorClassIdentifiers = new String[] {TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT};
+    }
+    this.taskCommunicators = new TaskCommunicator[taskCommunicatorClassIdentifiers.length];
+    for (int i = 0 ; i < taskCommunicatorClassIdentifiers.length ; i++) {
+      taskCommunicators[i] = createTaskCommunicator(taskCommunicatorClassIdentifiers[i]);
+    }
+    // TODO TEZ-2118 Start using taskCommunicator indices properly
+  }
+
+  @Override
+  public void serviceStart() {
+    // TODO Why is init tied to serviceStart
+    for (int i = 0 ; i < taskCommunicators.length ; i++) {
+      taskCommunicators[i].init(getConfig());
+      taskCommunicators[i].start();
+    }
   }
 
   @Override
-  public void serviceInit(Configuration conf) {
-    String taskCommClassName = conf.get(TezConfiguration.TEZ_AM_TASK_COMMUNICATOR_CLASS);
-    if (taskCommClassName == null) {
+  public void serviceStop() {
+    for (int i = 0 ; i < taskCommunicators.length ; i++) {
+      taskCommunicators[i].stop();
+    }
+  }
+
+  private TaskCommunicator createTaskCommunicator(String taskCommClassIdentifier) {
+    if (taskCommClassIdentifier.equals(TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT) ||
+        taskCommClassIdentifier
+            .equals(TezConstants.TEZ_AM_SERVICE_PLUGINS_LOCAL_MODE_NAME_DEFAULT)) {
       LOG.info("Using Default Task Communicator");
-      this.taskCommunicator = new TezTaskCommunicatorImpl(this);
+      return new TezTaskCommunicatorImpl(this);
     } else {
-      LOG.info("Using TaskCommunicator: " + taskCommClassName);
+      LOG.info("Using TaskCommunicator: " + taskCommClassIdentifier);
       Class<? extends TaskCommunicator> taskCommClazz = (Class<? extends TaskCommunicator>) ReflectionUtils
-          .getClazz(taskCommClassName);
+          .getClazz(taskCommClassIdentifier);
       try {
         Constructor<? extends TaskCommunicator> ctor = taskCommClazz.getConstructor(TaskCommunicatorContext.class);
         ctor.setAccessible(true);
-        this.taskCommunicator = ctor.newInstance(this);
+        return ctor.newInstance(this);
       } catch (NoSuchMethodException e) {
         throw new TezUncheckedException(e);
       } catch (InvocationTargetException e) {
@@ -135,20 +157,6 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
   }
 
   @Override
-  public void serviceStart() {
-    taskCommunicator.init(getConfig());
-    taskCommunicator.start();
-  }
-
-  @Override
-  public void serviceStop() {
-    if (taskCommunicator != null) {
-      taskCommunicator.stop();
-      taskCommunicator = null;
-    }
-  }
-
-  @Override
   public ApplicationAttemptId getApplicationAttemptId() {
     return context.getApplicationAttemptId();
   }
@@ -236,7 +244,8 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
 
   @Override
   public void taskStartedRemotely(TezTaskAttemptID taskAttemptID, ContainerId containerId) {
-    context.getEventHandler().handle(new TaskAttemptEventStartedRemotely(taskAttemptID, containerId, null));
+    context.getEventHandler()
+        .handle(new TaskAttemptEventStartedRemotely(taskAttemptID, containerId, null));
     pingContainerHeartbeatHandler(containerId);
   }
 
@@ -266,7 +275,7 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
 
   @Override
   public InetSocketAddress getAddress() {
-    return taskCommunicator.getAddress();
+    return taskCommunicators[0].getAddress();
   }
 
   // The TaskAttemptListener register / unregister methods in this class are not thread safe.
@@ -298,7 +307,7 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
           "Multiple registrations for containerId: " + containerId);
     }
     NodeId nodeId = context.getAllContainers().get(containerId).getContainer().getNodeId();
-    taskCommunicator.registerRunningContainer(containerId, nodeId.getHost(), nodeId.getPort());
+    taskCommunicators[0].registerRunningContainer(containerId, nodeId.getHost(), nodeId.getPort());
   }
 
   @Override
@@ -310,7 +319,7 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
     if (containerInfo.taskAttemptId != null) {
       registeredAttempts.remove(containerInfo.taskAttemptId);
     }
-    taskCommunicator.registerContainerEnd(containerId);
+    taskCommunicators[0].registerContainerEnd(containerId);
   }
 
   @Override
@@ -345,7 +354,7 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
           + amContainerTask.getTask().getTaskAttemptID() + " to container: " + containerId
           + " when already assigned to: " + containerIdFromMap);
     }
-    taskCommunicator.registerRunningTaskAttempt(containerId, amContainerTask.getTask(),
+    taskCommunicators[0].registerRunningTaskAttempt(containerId, amContainerTask.getTask(),
         amContainerTask.getAdditionalResources(), amContainerTask.getCredentials(),
         amContainerTask.haveCredentialsChanged());
   }
@@ -365,7 +374,7 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
     }
     // Explicitly putting in a new entry so that synchronization is not required on the existing element in the map.
     registeredContainers.put(containerId, NULL_CONTAINER_INFO);
-    taskCommunicator.unregisterRunningTaskAttempt(attemptId);
+    taskCommunicators[0].unregisterRunningTaskAttempt(attemptId);
   }
 
   private void pingContainerHeartbeatHandler(ContainerId containerId) {
@@ -405,6 +414,6 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
   }
 
   public TaskCommunicator getTaskCommunicator() {
-    return taskCommunicator;
+    return taskCommunicators[0];
   }
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/adab48fa/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 77ef6e0..3fa3669 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
@@ -168,4 +168,8 @@ public interface Vertex extends Comparable<Vertex> {
   public int getKilledTaskAttemptCount();
 
   public Configuration getConf();
+
+  public int getTaskSchedulerIdentifier();
+  public int getContainerLauncherIdentifier();
+  public int getTaskCommunicatorIdentifier();
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/adab48fa/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 b1c0acc..c18dc00 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
@@ -1066,9 +1066,15 @@ public class TaskAttemptImpl implements TaskAttempt,
         priority = (scheduleEvent.getPriorityHighLimit() + scheduleEvent.getPriorityLowLimit()) / 2;
       }
 
+      // TODO Jira post TEZ-2003 getVertex implementation is very inefficient. This should be via references, instead of locked table lookups.
+      Vertex vertex = ta.getVertex();
       AMSchedulerEventTALaunchRequest launchRequestEvent = new AMSchedulerEventTALaunchRequest(
           ta.attemptId, ta.taskResource, remoteTaskSpec, ta, locationHint,
-          priority, ta.containerContext);
+          priority, ta.containerContext,
+          vertex.getTaskSchedulerIdentifier(),
+          vertex.getContainerLauncherIdentifier(),
+          vertex.getTaskCommunicatorIdentifier());
+
       ta.sendEvent(launchRequestEvent);
       return TaskAttemptStateInternal.START_WAIT;
     }

http://git-wip-us.apache.org/repos/asf/tez/blob/adab48fa/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 9ed7441..ddf670f 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
@@ -73,6 +73,7 @@ import org.apache.tez.dag.api.ProcessorDescriptor;
 import org.apache.tez.dag.api.RootInputLeafOutput;
 import org.apache.tez.dag.api.Scope;
 import org.apache.tez.dag.api.TezConfiguration;
+import org.apache.tez.dag.api.TezConstants;
 import org.apache.tez.dag.api.TezUncheckedException;
 import org.apache.tez.dag.api.VertexLocationHint;
 import org.apache.tez.dag.api.TaskLocationHint;
@@ -225,6 +226,10 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex, EventHandl
   
   private final boolean isSpeculationEnabled;
 
+  private final int taskSchedulerIdentifier;
+  private final int containerLauncherIdentifier;
+  private final int taskCommunicatorIdentifier;
+
   //fields initialized in init
 
   @VisibleForTesting
@@ -934,6 +939,33 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex, EventHandl
     // This "this leak" is okay because the retained pointer is in an
     //  instance variable.
 
+    boolean isLocal = vertexConf.getBoolean(TezConfiguration.TEZ_LOCAL_MODE,
+        TezConfiguration.TEZ_LOCAL_MODE_DEFAULT);
+
+    String tezDefaultComponentName =
+        isLocal ? TezConstants.TEZ_AM_SERVICE_PLUGINS_LOCAL_MODE_NAME_DEFAULT :
+            TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT;
+    String taskSchedulerName =
+        vertexConf.get(TezConfiguration.TEZ_AM_VERTEX_TASK_SCHEDULER_NAME, tezDefaultComponentName);
+    String taskCommName = vertexConf
+        .get(TezConfiguration.TEZ_AM_VERTEX_TASK_COMMUNICATOR_NAME, tezDefaultComponentName);
+    String containerLauncherName = vertexConf
+        .get(TezConfiguration.TEZ_AM_VERTEX_CONTAINER_LAUNCHER_NAME, tezDefaultComponentName);
+    taskSchedulerIdentifier = appContext.getTaskScheduerIdentifier(taskSchedulerName);
+    taskCommunicatorIdentifier = appContext.getTaskCommunicatorIdentifier(taskCommName);
+    containerLauncherIdentifier = appContext.getContainerLauncherIdentifier(containerLauncherName);
+
+    Preconditions.checkNotNull(taskSchedulerIdentifier, "Unknown taskScheduler: " + taskSchedulerName);
+    Preconditions.checkNotNull(taskCommunicatorIdentifier, "Unknown taskCommunicator: " + containerLauncherName);
+    Preconditions.checkNotNull(containerLauncherIdentifier, "Unknown containerLauncher: " + taskCommName);
+
+    StringBuilder sb = new StringBuilder();
+    sb.append("Running vertex: ").append(logIdentifier).append(" : ")
+        .append("TaskScheduler=").append(taskSchedulerIdentifier).append(":").append(taskSchedulerName)
+        .append(", ContainerLauncher=").append(containerLauncherIdentifier).append(":").append(containerLauncherName)
+        .append(", TaskCommunicator=").append(taskCommunicatorIdentifier).append(":").append(taskCommName);
+    LOG.info(sb.toString());
+
     stateMachine = new StateMachineTez<VertexState, VertexEventType, VertexEvent, VertexImpl>(
         stateMachineFactory.make(this), this);
     augmentStateMachine();
@@ -944,6 +976,21 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex, EventHandl
     return vertexConf;
   }
 
+  @Override
+  public int getTaskSchedulerIdentifier() {
+    return this.taskSchedulerIdentifier;
+  }
+
+  @Override
+  public int getContainerLauncherIdentifier() {
+    return this.containerLauncherIdentifier;
+  }
+
+  @Override
+  public int getTaskCommunicatorIdentifier() {
+    return this.taskCommunicatorIdentifier;
+  }
+
   private boolean isSpeculationEnabled() {
     return isSpeculationEnabled;
   }

http://git-wip-us.apache.org/repos/asf/tez/blob/adab48fa/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherRouter.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherRouter.java b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherRouter.java
index 34001ed..621e4a8 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherRouter.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherRouter.java
@@ -26,6 +26,7 @@ import org.apache.hadoop.service.AbstractService;
 import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.tez.common.ReflectionUtils;
 import org.apache.tez.dag.api.TezConfiguration;
+import org.apache.tez.dag.api.TezConstants;
 import org.apache.tez.dag.api.TezUncheckedException;
 import org.apache.tez.dag.app.AppContext;
 import org.apache.tez.dag.app.TaskAttemptListener;
@@ -36,73 +37,93 @@ public class ContainerLauncherRouter extends AbstractService
 
   static final Log LOG = LogFactory.getLog(ContainerLauncherImpl.class);
 
-  private final ContainerLauncher containerLauncher;
+  private final ContainerLauncher containerLaunchers[];
 
   @VisibleForTesting
   public ContainerLauncherRouter(ContainerLauncher containerLauncher) {
     super(ContainerLauncherRouter.class.getName());
-    this.containerLauncher = containerLauncher;
+    containerLaunchers = new ContainerLauncher[] {containerLauncher};
   }
 
   // Accepting conf to setup final parameters, if required.
-  public ContainerLauncherRouter(Configuration conf, boolean isLocal, AppContext context,
+  public ContainerLauncherRouter(Configuration conf, AppContext context,
                                  TaskAttemptListener taskAttemptListener,
-                                 String workingDirectory) throws UnknownHostException {
+                                 String workingDirectory,
+                                 String[] containerLauncherClassIdentifiers) throws UnknownHostException {
     super(ContainerLauncherRouter.class.getName());
 
-    if (isLocal) {
+    if (containerLauncherClassIdentifiers == null || containerLauncherClassIdentifiers.length == 0) {
+      containerLauncherClassIdentifiers = new String[] {TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT};
+    }
+    containerLaunchers = new ContainerLauncher[containerLauncherClassIdentifiers.length];
+
+    for (int i = 0; i < containerLauncherClassIdentifiers.length; i++) {
+      containerLaunchers[i] = createContainerLauncher(containerLauncherClassIdentifiers[i], context,
+          taskAttemptListener, workingDirectory, conf);
+    }
+  }
+
+  private ContainerLauncher createContainerLauncher(String containerLauncherClassIdentifier,
+                                                    AppContext context,
+                                                    TaskAttemptListener taskAttemptListener,
+                                                    String workingDirectory,
+                                                    Configuration conf) throws
+      UnknownHostException {
+    if (containerLauncherClassIdentifier.equals(TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT)) {
+      LOG.info("Creating DefaultContainerLauncher");
+      return new ContainerLauncherImpl(context);
+    } else if (containerLauncherClassIdentifier
+        .equals(TezConstants.TEZ_AM_SERVICE_PLUGINS_LOCAL_MODE_NAME_DEFAULT)) {
       LOG.info("Creating LocalContainerLauncher");
-      containerLauncher =
+      return
           new LocalContainerLauncher(context, taskAttemptListener, workingDirectory);
     } else {
-      // TODO: Temporary reflection with specific parameters until a clean interface is defined.
-      String containerLauncherClassName =
-          conf.get(TezConfiguration.TEZ_AM_CONTAINER_LAUNCHER_CLASS);
-      if (containerLauncherClassName == null) {
-        LOG.info("Creating Default Container Launcher");
-        containerLauncher = new ContainerLauncherImpl(context);
-      } else {
-        LOG.info("Creating container launcher : " + containerLauncherClassName);
-        Class<? extends ContainerLauncher> containerLauncherClazz =
-            (Class<? extends ContainerLauncher>) ReflectionUtils.getClazz(
-                containerLauncherClassName);
-        try {
-          Constructor<? extends ContainerLauncher> ctor = containerLauncherClazz
-              .getConstructor(AppContext.class, Configuration.class, TaskAttemptListener.class);
-          ctor.setAccessible(true);
-          containerLauncher = ctor.newInstance(context, conf, taskAttemptListener);
-        } catch (NoSuchMethodException e) {
-          throw new TezUncheckedException(e);
-        } catch (InvocationTargetException e) {
-          throw new TezUncheckedException(e);
-        } catch (InstantiationException e) {
-          throw new TezUncheckedException(e);
-        } catch (IllegalAccessException e) {
-          throw new TezUncheckedException(e);
-        }
+      LOG.info("Creating container launcher : " + containerLauncherClassIdentifier);
+      Class<? extends ContainerLauncher> containerLauncherClazz =
+          (Class<? extends ContainerLauncher>) ReflectionUtils.getClazz(
+              containerLauncherClassIdentifier);
+      try {
+        Constructor<? extends ContainerLauncher> ctor = containerLauncherClazz
+            .getConstructor(AppContext.class, Configuration.class, TaskAttemptListener.class);
+        ctor.setAccessible(true);
+        return ctor.newInstance(context, conf, taskAttemptListener);
+      } catch (NoSuchMethodException e) {
+        throw new TezUncheckedException(e);
+      } catch (InvocationTargetException e) {
+        throw new TezUncheckedException(e);
+      } catch (InstantiationException e) {
+        throw new TezUncheckedException(e);
+      } catch (IllegalAccessException e) {
+        throw new TezUncheckedException(e);
       }
-
     }
+    // TODO TEZ-2118 Handle routing to multiple launchers
   }
 
   @Override
   public void serviceInit(Configuration conf) {
-    ((AbstractService)containerLauncher).init(conf);
+    for (int i = 0 ; i < containerLaunchers.length ; i++) {
+      ((AbstractService) containerLaunchers[i]).init(conf);
+    }
   }
 
   @Override
   public void serviceStart() {
-    ((AbstractService)containerLauncher).start();
+    for (int i = 0 ; i < containerLaunchers.length ; i++) {
+      ((AbstractService) containerLaunchers[i]).start();
+    }
   }
 
   @Override
   public void serviceStop() {
-    ((AbstractService)containerLauncher).stop();
+    for (int i = 0 ; i < containerLaunchers.length ; i++) {
+      ((AbstractService) containerLaunchers[i]).stop();
+    }
   }
 
 
   @Override
   public void handle(NMCommunicatorEvent event) {
-    containerLauncher.handle(event);
+    containerLaunchers[0].handle(event);
   }
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/adab48fa/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 5c4d43c..c59193c 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
@@ -38,11 +38,16 @@ public class AMSchedulerEventTALaunchRequest extends AMSchedulerEvent {
   private final TaskSpec remoteTaskSpec;
   private final TaskAttempt taskAttempt;
 
+  private final int schedulerId;
+  private final int launcherId;
+  private final int taskCommId;
+
   public AMSchedulerEventTALaunchRequest(TezTaskAttemptID attemptId,
       Resource capability,
       TaskSpec remoteTaskSpec, TaskAttempt ta,
       TaskLocationHint locationHint, int priority,
-      ContainerContext containerContext) {
+      ContainerContext containerContext,
+      int schedulerId, int launcherId, int taskCommId) {
     super(AMSchedulerEventType.S_TA_LAUNCH_REQUEST);
     this.attemptId = attemptId;
     this.capability = capability;
@@ -51,6 +56,9 @@ public class AMSchedulerEventTALaunchRequest extends AMSchedulerEvent {
     this.locationHint = locationHint;
     this.priority = priority;
     this.containerContext = containerContext;
+    this.schedulerId = schedulerId;
+    this.launcherId = launcherId;
+    this.taskCommId = taskCommId;
   }
 
   public TezTaskAttemptID getAttemptID() {
@@ -81,6 +89,18 @@ public class AMSchedulerEventTALaunchRequest extends AMSchedulerEvent {
     return this.containerContext;
   }
 
+  public int getSchedulerId() {
+    return schedulerId;
+  }
+
+  public int getLauncherId() {
+    return launcherId;
+  }
+
+  public int getTaskCommId() {
+    return taskCommId;
+  }
+
   // Parameter replacement: @taskid@ will not be usable
   // ProfileTaskRange not available along with ContainerReUse
 

http://git-wip-us.apache.org/repos/asf/tez/blob/adab48fa/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerEventHandler.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerEventHandler.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerEventHandler.java
index 8c3ed87..72389e7 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerEventHandler.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerEventHandler.java
@@ -46,6 +46,7 @@ import org.apache.hadoop.yarn.event.Event;
 import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.tez.common.ReflectionUtils;
 import org.apache.tez.dag.api.TezConfiguration;
+import org.apache.tez.dag.api.TezConstants;
 import org.apache.tez.dag.api.TezUncheckedException;
 import org.apache.tez.dag.api.TaskLocationHint;
 import org.apache.tez.dag.api.TaskLocationHint.TaskBasedLocationAffinity;
@@ -92,7 +93,6 @@ public class TaskSchedulerEventHandler extends AbstractService
   @SuppressWarnings("rawtypes")
   private final EventHandler eventHandler;
   private final String historyUrl;
-  protected TaskSchedulerService taskScheduler;
   private DAGAppMaster dagAppMaster;
   private Map<ApplicationAccessType, String> appAcls = null;
   private Thread eventHandlingThread;
@@ -105,14 +105,27 @@ public class TaskSchedulerEventHandler extends AbstractService
   private AtomicBoolean shouldUnregisterFlag =
       new AtomicBoolean(false);
   private final WebUIService webUI;
+  private final String[] taskSchedulerClasses;
+  protected final TaskSchedulerService []taskSchedulers;
 
   BlockingQueue<AMSchedulerEvent> eventQueue
                               = new LinkedBlockingQueue<AMSchedulerEvent>();
 
+  /**
+   *
+   * @param appContext
+   * @param clientService
+   * @param eventHandler
+   * @param containerSignatureMatcher
+   * @param webUI
+   * @param schedulerClasses the list of scheduler classes / codes. Tez internal classes are represented as codes.
+   *                         An empty list defaults to using the YarnTaskScheduler as the only source.
+   */
   @SuppressWarnings("rawtypes")
   public TaskSchedulerEventHandler(AppContext appContext,
       DAGClientServer clientService, EventHandler eventHandler, 
-      ContainerSignatureMatcher containerSignatureMatcher, WebUIService webUI) {
+      ContainerSignatureMatcher containerSignatureMatcher, WebUIService webUI,
+      String [] schedulerClasses) {
     super(TaskSchedulerEventHandler.class.getName());
     this.appContext = appContext;
     this.eventHandler = eventHandler;
@@ -123,6 +136,12 @@ public class TaskSchedulerEventHandler extends AbstractService
     if (this.webUI != null) {
       this.webUI.setHistoryUrl(this.historyUrl);
     }
+    if (schedulerClasses == null || schedulerClasses.length == 0) {
+      this.taskSchedulerClasses = new String[] {TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT};
+    } else {
+      this.taskSchedulerClasses = schedulerClasses;
+    }
+    taskSchedulers = new TaskSchedulerService[this.taskSchedulerClasses.length];
   }
 
   public Map<ApplicationAccessType, String> getApplicationAcls() {
@@ -139,11 +158,11 @@ public class TaskSchedulerEventHandler extends AbstractService
   }
   
   public Resource getAvailableResources() {
-    return taskScheduler.getAvailableResources();
+    return taskSchedulers[0].getAvailableResources();
   }
 
   public Resource getTotalResources() {
-    return taskScheduler.getTotalResources();
+    return taskSchedulers[0].getTotalResources();
   }
 
   public synchronized void handleEvent(AMSchedulerEvent sEvent) {
@@ -209,9 +228,9 @@ public class TaskSchedulerEventHandler extends AbstractService
 
   private void handleNodeBlacklistUpdate(AMSchedulerEventNodeBlacklistUpdate event) {
     if (event.getType() == AMSchedulerEventType.S_NODE_BLACKLISTED) {
-      taskScheduler.blacklistNode(event.getNodeId());
+      taskSchedulers[0].blacklistNode(event.getNodeId());
     } else if (event.getType() == AMSchedulerEventType.S_NODE_UNBLACKLISTED) {
-      taskScheduler.unblacklistNode(event.getNodeId());
+      taskSchedulers[0].unblacklistNode(event.getNodeId());
     } else {
       throw new TezUncheckedException("Invalid event type: " + event.getType());
     }
@@ -223,14 +242,14 @@ public class TaskSchedulerEventHandler extends AbstractService
     // TODO what happens to the task that was connected to this container?
     // current assumption is that it will eventually call handleTaStopRequest
     //TaskAttempt taskAttempt = (TaskAttempt)
-    taskScheduler.deallocateContainer(containerId);
+    taskSchedulers[0].deallocateContainer(containerId);
     // TODO does this container need to be stopped via C_STOP_REQUEST
     sendEvent(new AMContainerEventStopRequest(containerId));
   }
 
   private void handleTAUnsuccessfulEnd(AMSchedulerEventTAEnded event) {
     TaskAttempt attempt = event.getAttempt();
-    boolean wasContainerAllocated = taskScheduler.deallocateTask(attempt, false);
+    boolean wasContainerAllocated = taskSchedulers[0].deallocateTask(attempt, false);
     // use stored value of container id in case the scheduler has removed this
     // assignment because the task has been deallocated earlier.
     // retroactive case
@@ -272,7 +291,7 @@ public class TaskSchedulerEventHandler extends AbstractService
           event.getAttemptID()));
     }
 
-    boolean wasContainerAllocated = taskScheduler.deallocateTask(attempt, true);
+    boolean wasContainerAllocated = taskSchedulers[0].deallocateTask(attempt, true);
     if (!wasContainerAllocated) {
       LOG.error("De-allocated successful task: " + attempt.getID()
           + ", but TaskScheduler reported no container assigned to task");
@@ -297,7 +316,7 @@ public class TaskSchedulerEventHandler extends AbstractService
         TaskAttempt affinityAttempt = vertex.getTask(taskIndex).getSuccessfulAttempt();
         if (affinityAttempt != null) {
           Preconditions.checkNotNull(affinityAttempt.getAssignedContainerID(), affinityAttempt.getID());
-          taskScheduler.allocateTask(taskAttempt,
+          taskSchedulers[0].allocateTask(taskAttempt,
               event.getCapability(),
               affinityAttempt.getAssignedContainerID(),
               Priority.newInstance(event.getPriority()),
@@ -316,57 +335,59 @@ public class TaskSchedulerEventHandler extends AbstractService
             .toArray(new String[locationHint.getRacks().size()]) : null;
       }
     }
-    
-    taskScheduler.allocateTask(taskAttempt,
-                               event.getCapability(),
-                               hosts,
-                               racks,
-                               Priority.newInstance(event.getPriority()),
-                               event.getContainerContext(),
-                               event);
-  }
-
-
-  protected TaskSchedulerService createTaskScheduler(String host, int port,
-      String trackingUrl, AppContext appContext) {
-    boolean isLocal = getConfig().getBoolean(TezConfiguration.TEZ_LOCAL_MODE,
-        TezConfiguration.TEZ_LOCAL_MODE_DEFAULT);
-    if (isLocal) {
-      LOG.info("Using TaskScheduler: LocalTaskSchedulerService");
+
+    taskSchedulers[0].allocateTask(taskAttempt,
+        event.getCapability(),
+        hosts,
+        racks,
+        Priority.newInstance(event.getPriority()),
+        event.getContainerContext(),
+        event);
+  }
+
+  private TaskSchedulerService createTaskScheduler(String host, int port, String trackingUrl,
+                                                   AppContext appContext,
+                                                   String schedulerClassName) {
+    if (schedulerClassName.equals(TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT)) {
+      LOG.info("Creating TaskScheduler: YarnTaskSchedulerService");
+      return new YarnTaskSchedulerService(this, this.containerSignatureMatcher,
+          host, port, trackingUrl, appContext);
+    } else if (schedulerClassName.equals(TezConstants.TEZ_AM_SERVICE_PLUGINS_LOCAL_MODE_NAME_DEFAULT)) {
+      LOG.info("Creating TaskScheduler: Local TaskScheduler");
       return new LocalTaskSchedulerService(this, this.containerSignatureMatcher,
           host, port, trackingUrl, appContext);
-    }
-    else {
-      String schedulerClassName = getConfig().get(TezConfiguration.TEZ_AM_TASK_SCHEDULER_CLASS);
-      if (schedulerClassName == null) {
-        LOG.info("Using TaskScheduler: YarnTaskSchedulerService");
-        return new YarnTaskSchedulerService(this, this.containerSignatureMatcher,
-            host, port, trackingUrl, appContext);
-      } else {
-        LOG.info("Using custom TaskScheduler: " + schedulerClassName);
-        // TODO Temporary reflection with specific parameters. Remove once there is a clean interface.
-        Class<? extends TaskSchedulerService> taskSchedulerClazz =
-            (Class<? extends TaskSchedulerService>) ReflectionUtils.getClazz(schedulerClassName);
-        try {
-          Constructor<? extends TaskSchedulerService> ctor = taskSchedulerClazz
-              .getConstructor(TaskSchedulerAppCallback.class, AppContext.class, String.class,
-                  int.class, String.class, Configuration.class);
-          ctor.setAccessible(true);
-          TaskSchedulerService taskSchedulerService =
-              ctor.newInstance(this, appContext, host, port, trackingUrl, getConfig());
-          return taskSchedulerService;
-        } catch (NoSuchMethodException e) {
-          throw new TezUncheckedException(e);
-        } catch (InvocationTargetException e) {
-          throw new TezUncheckedException(e);
-        } catch (InstantiationException e) {
-          throw new TezUncheckedException(e);
-        } catch (IllegalAccessException e) {
-          throw new TezUncheckedException(e);
-        }
+    } else {
+      LOG.info("Creating custom TaskScheduler: " + schedulerClassName);
+      // TODO TEZ-2003 Temporary reflection with specific parameters. Remove once there is a clean interface.
+      Class<? extends TaskSchedulerService> taskSchedulerClazz =
+          (Class<? extends TaskSchedulerService>) ReflectionUtils.getClazz(schedulerClassName);
+      try {
+        Constructor<? extends TaskSchedulerService> ctor = taskSchedulerClazz
+            .getConstructor(TaskSchedulerAppCallback.class, AppContext.class, String.class,
+                int.class, String.class, Configuration.class);
+        ctor.setAccessible(true);
+        return ctor.newInstance(this, appContext, host, port, trackingUrl, getConfig());
+      } catch (NoSuchMethodException e) {
+        throw new TezUncheckedException(e);
+      } catch (InvocationTargetException e) {
+        throw new TezUncheckedException(e);
+      } catch (InstantiationException e) {
+        throw new TezUncheckedException(e);
+      } catch (IllegalAccessException e) {
+        throw new TezUncheckedException(e);
       }
     }
   }
+
+  @VisibleForTesting
+  protected void instantiateScheduelrs(String host, int port, String trackingUrl, AppContext appContext) {
+    // Iterate over the list and create all the taskSchedulers
+    for (int i = 0; i < taskSchedulerClasses.length; i++) {
+      taskSchedulers[i] = createTaskScheduler(host, port,
+          trackingUrl, appContext, taskSchedulerClasses[i]);
+    }
+  }
+
   
   @Override
   public synchronized void serviceStart() {
@@ -377,13 +398,17 @@ public class TaskSchedulerEventHandler extends AbstractService
     // always try to connect to AM and proxy the response. hence it wont work if the webUIService
     // is not enabled.
     String trackingUrl = (webUI != null) ? webUI.getTrackingURL() : "";
-    taskScheduler = createTaskScheduler(serviceAddr.getHostName(),
-        serviceAddr.getPort(), trackingUrl, appContext);
-    taskScheduler.init(getConfig());
-    taskScheduler.start();
+    instantiateScheduelrs(serviceAddr.getHostName(), serviceAddr.getPort(), trackingUrl, appContext);
+
+    for (int i = 0 ; i < taskSchedulers.length ; i++) {
+      taskSchedulers[i].init(getConfig());
+      taskSchedulers[i].start();
+    }
+
+    // TODO TEZ-2118 Start using multiple task schedulers
     if (shouldUnregisterFlag.get()) {
       // Flag may have been set earlier when task scheduler was not initialized
-      taskScheduler.setShouldUnregister();
+      taskSchedulers[0].setShouldUnregister();
     }
 
     this.eventHandlingThread = new Thread("TaskSchedulerEventHandlerThread") {
@@ -432,8 +457,8 @@ public class TaskSchedulerEventHandler extends AbstractService
       if (eventHandlingThread != null)
         eventHandlingThread.interrupt();
     }
-    if (taskScheduler != null) {
-      ((AbstractService)taskScheduler).stop();
+    if (taskSchedulers[0] != null) {
+      ((AbstractService)taskSchedulers[0]).stop();
     }
   }
 
@@ -578,7 +603,7 @@ public class TaskSchedulerEventHandler extends AbstractService
   public float getProgress() {
     // at this point allocate has been called and so node count must be available
     // may change after YARN-1722
-    int nodeCount = taskScheduler.getClusterNodeCount();
+    int nodeCount = taskSchedulers[0].getClusterNodeCount();
     if (nodeCount != cachedNodeCount) {
       cachedNodeCount = nodeCount;
       sendEvent(new AMNodeEventNodeCountUpdated(cachedNodeCount));
@@ -593,7 +618,7 @@ public class TaskSchedulerEventHandler extends AbstractService
   }
 
   public void dagCompleted() {
-    taskScheduler.dagComplete();
+    taskSchedulers[0].dagComplete();
   }
 
   public void dagSubmitted() {
@@ -603,7 +628,7 @@ public class TaskSchedulerEventHandler extends AbstractService
 
   @Override
   public void preemptContainer(ContainerId containerId) {
-    taskScheduler.deallocateContainer(containerId);
+    taskSchedulers[0].deallocateContainer(containerId);
     // Inform the Containers about completion.
     sendEvent(new AMContainerEventCompleted(containerId, ContainerExitStatus.INVALID,
         "Container preempted internally", TaskAttemptTerminationCause.INTERNAL_PREEMPTION));
@@ -612,13 +637,13 @@ public class TaskSchedulerEventHandler extends AbstractService
   public void setShouldUnregisterFlag() {
     LOG.info("TaskScheduler notified that it should unregister from RM");
     this.shouldUnregisterFlag.set(true);
-    if (this.taskScheduler != null) {
-      this.taskScheduler.setShouldUnregister();
+    if (this.taskSchedulers[0] != null) {
+      this.taskSchedulers[0].setShouldUnregister();
     }
   }
 
   public boolean hasUnregistered() {
-    return this.taskScheduler.hasUnregistered();
+    return this.taskSchedulers[0].hasUnregistered();
   }
 
   @VisibleForTesting

http://git-wip-us.apache.org/repos/asf/tez/blob/adab48fa/tez-dag/src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java b/tez-dag/src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java
index 2913cee..26fc1ab 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java
@@ -512,10 +512,11 @@ public class MockDAGAppMaster extends DAGAppMaster {
     this.handlerConcurrency = handlerConcurrency;
     this.numConcurrentContainers = numConcurrentContainers;
   }
-  
+
   // use mock container launcher for tests
   @Override
-  protected ContainerLauncherRouter createContainerLauncherRouter(final Configuration conf)
+  protected ContainerLauncherRouter createContainerLauncherRouter(final Configuration conf,
+                                                                  String[] containerLaunchers)
       throws UnknownHostException {
     return new ContainerLauncherRouter(containerLauncher);
   }

http://git-wip-us.apache.org/repos/asf/tez/blob/adab48fa/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag.java b/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag.java
index 286e897..16bd1d3 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag.java
@@ -31,6 +31,7 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
+import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
@@ -107,18 +108,9 @@ public class TestTaskAttemptListenerImplTezDag {
     doReturn(dag).when(appContext).getCurrentDAG();
     doReturn(appAcls).when(appContext).getApplicationACLs();
     doReturn(amContainerMap).when(appContext).getAllContainers();
-    NodeId nodeId = NodeId.newInstance("localhost", 0);
-    AMContainer amContainer = mock(AMContainer.class);
-    Container container = mock(Container.class);
-    doReturn(nodeId).when(container).getNodeId();
-    doReturn(amContainer).when(amContainerMap).get(any(ContainerId.class));
-    doReturn(container).when(amContainer).getContainer();
-
-    taskAttemptListener =
-        new TaskAttemptListenerImpTezDag(appContext, mock(TaskHeartbeatHandler.class),
-            mock(ContainerHeartbeatHandler.class), null);
-    TezTaskCommunicatorImpl taskCommunicator = (TezTaskCommunicatorImpl)taskAttemptListener.getTaskCommunicator();
-    TezTaskUmbilicalProtocol tezUmbilical = taskCommunicator.getUmbilical();
+
+    taskAttemptListener = new TaskAttemptListenerImplForTest(appContext,
+        mock(TaskHeartbeatHandler.class), mock(ContainerHeartbeatHandler.class), null);
 
     taskSpec = mock(TaskSpec.class);
     doReturn(taskAttemptID).when(taskSpec).getTaskAttemptID();
@@ -129,6 +121,9 @@ public class TestTaskAttemptListenerImplTezDag {
   @Test(timeout = 5000)
   public void testGetTask() throws IOException {
 
+    TezTaskCommunicatorImpl taskCommunicator = (TezTaskCommunicatorImpl)taskAttemptListener.getTaskCommunicator();
+    TezTaskUmbilicalProtocol tezUmbilical = taskCommunicator.getUmbilical();
+
     ContainerId containerId1 = createContainerId(appId, 1);
     ContainerContext containerContext1 = new ContainerContext(containerId1.toString());
     containerTask = tezUmbilical.getTask(containerContext1);

http://git-wip-us.apache.org/repos/asf/tez/blob/adab48fa/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 89b77a7..54b9adb 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
@@ -1322,7 +1322,7 @@ public class TestContainerReuse {
           InputDescriptor.create("inputClassName"), 1)),
       Collections.singletonList(new OutputSpec("vertexName",
           OutputDescriptor.create("outputClassName"), 1)), null), ta, locationHint,
-      priority.getPriority(), containerContext);
+      priority.getPriority(), containerContext, 0, 0, 0);
     return lr;
   }
 

http://git-wip-us.apache.org/repos/asf/tez/blob/adab48fa/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerEventHandler.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerEventHandler.java b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerEventHandler.java
index af3e40d..291e786 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerEventHandler.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerEventHandler.java
@@ -89,13 +89,13 @@ public class TestTaskSchedulerEventHandler {
     public MockTaskSchedulerEventHandler(AppContext appContext,
         DAGClientServer clientService, EventHandler eventHandler,
         ContainerSignatureMatcher containerSignatureMatcher, WebUIService webUI) {
-      super(appContext, clientService, eventHandler, containerSignatureMatcher, webUI);
+      super(appContext, clientService, eventHandler, containerSignatureMatcher, webUI, new String[] {});
     }
-    
+
     @Override
-    protected TaskSchedulerService createTaskScheduler(String host, int port,
-        String trackingUrl, AppContext appContext) {
-      return mockTaskScheduler;
+    protected void instantiateScheduelrs(String host, int port, String trackingUrl,
+                                         AppContext appContext) {
+      taskSchedulers[0] = mockTaskScheduler;
     }
     
     @Override
@@ -194,7 +194,7 @@ public class TestTaskSchedulerEventHandler {
     when(mockAppContext.getCurrentDAG().getVertex(affVertexName)).thenReturn(affVertex);
     Resource resource = Resource.newInstance(100, 1);
     AMSchedulerEventTALaunchRequest event = new AMSchedulerEventTALaunchRequest
-        (taId, resource, null, mockTaskAttempt, locHint, 3, null);
+        (taId, resource, null, mockTaskAttempt, locHint, 3, null, 0, 0, 0);
     schedulerHandler.notify.set(false);
     schedulerHandler.handle(event);
     synchronized (schedulerHandler.notify) {

http://git-wip-us.apache.org/repos/asf/tez/blob/adab48fa/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerHelpers.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerHelpers.java b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerHelpers.java
index 77c98b7..d775300 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerHelpers.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerHelpers.java
@@ -127,31 +127,29 @@ class TestTaskSchedulerHelpers {
         EventHandler eventHandler,
         TezAMRMClientAsync<CookieContainerRequest> amrmClientAsync,
         ContainerSignatureMatcher containerSignatureMatcher) {
-      super(appContext, null, eventHandler, containerSignatureMatcher, null);
+      super(appContext, null, eventHandler, containerSignatureMatcher, null, new String[]{});
       this.amrmClientAsync = amrmClientAsync;
       this.containerSignatureMatcher = containerSignatureMatcher;
     }
 
     @Override
-    public TaskSchedulerService createTaskScheduler(String host, int port,
-        String trackingUrl, AppContext appContext) {
-      return new TaskSchedulerWithDrainableAppCallback(this,
+    public void instantiateScheduelrs(String host, int port, String trackingUrl, AppContext appContext) {
+      taskSchedulers[0] = new TaskSchedulerWithDrainableAppCallback(this,
           containerSignatureMatcher, host, port, trackingUrl, amrmClientAsync,
           appContext);
     }
 
     public TaskSchedulerService getSpyTaskScheduler() {
-      return this.taskScheduler;
+      return taskSchedulers[0];
     }
 
     @Override
     public void serviceStart() {
-      TaskSchedulerService taskSchedulerReal = createTaskScheduler("host", 0, "",
-        appContext);
+      instantiateScheduelrs("host", 0, "", appContext);
       // Init the service so that reuse configuration is picked up.
-      ((AbstractService)taskSchedulerReal).init(getConfig());
-      ((AbstractService)taskSchedulerReal).start();
-      taskScheduler = spy(taskSchedulerReal);
+      ((AbstractService)taskSchedulers[0]).init(getConfig());
+      ((AbstractService)taskSchedulers[0]).start();
+      taskSchedulers[0] = spy(taskSchedulers[0]);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/tez/blob/adab48fa/tez-ext-service-tests/src/test/java/org/apache/tez/tests/TestExternalTezServices.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/tests/TestExternalTezServices.java b/tez-ext-service-tests/src/test/java/org/apache/tez/tests/TestExternalTezServices.java
index a93c1a4..ae7e7f8 100644
--- a/tez-ext-service-tests/src/test/java/org/apache/tez/tests/TestExternalTezServices.java
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/tests/TestExternalTezServices.java
@@ -45,6 +45,8 @@ public class TestExternalTezServices {
 
   private static final Log LOG = LogFactory.getLog(TestExternalTezServices.class);
 
+  private static final String EXT_PUSH_ENTITY_NAME = "ExtServiceTestPush";
+
   private static MiniTezCluster tezCluster;
   private static MiniDFSCluster dfsCluster;
   private static MiniTezTestServiceCluster tezTestServiceCluster;
@@ -106,12 +108,17 @@ public class TestExternalTezServices {
     remoteFs.mkdirs(stagingDirPath);
     // This is currently configured to push tasks into the Service, and then use the standard RPC
     confForJobs.set(TezConfiguration.TEZ_AM_STAGING_DIR, stagingDirPath.toString());
-    confForJobs.set(TezConfiguration.TEZ_AM_TASK_SCHEDULER_CLASS,
-        TezTestServiceTaskSchedulerService.class.getName());
-    confForJobs.set(TezConfiguration.TEZ_AM_CONTAINER_LAUNCHER_CLASS,
-        TezTestServiceNoOpContainerLauncher.class.getName());
-    confForJobs.set(TezConfiguration.TEZ_AM_TASK_COMMUNICATOR_CLASS,
-        TezTestServiceTaskCommunicatorImpl.class.getName());
+    confForJobs.set(TezConfiguration.TEZ_AM_TASK_SCHEDULERS,
+        EXT_PUSH_ENTITY_NAME + ":" + TezTestServiceTaskSchedulerService.class.getName());
+    confForJobs.set(TezConfiguration.TEZ_AM_CONTAINER_LAUNCHERS,
+        EXT_PUSH_ENTITY_NAME + ":" + TezTestServiceNoOpContainerLauncher.class.getName());
+    confForJobs.set(TezConfiguration.TEZ_AM_TASK_COMMUNICATORS,
+        EXT_PUSH_ENTITY_NAME + ":" + TezTestServiceTaskCommunicatorImpl.class.getName());
+
+    confForJobs.set(TezConfiguration.TEZ_AM_VERTEX_TASK_SCHEDULER_NAME, EXT_PUSH_ENTITY_NAME);
+    confForJobs.set(TezConfiguration.TEZ_AM_VERTEX_CONTAINER_LAUNCHER_NAME, EXT_PUSH_ENTITY_NAME);
+    confForJobs.set(TezConfiguration.TEZ_AM_VERTEX_TASK_COMMUNICATOR_NAME, EXT_PUSH_ENTITY_NAME);
+
 
     TezConfiguration tezConf = new TezConfiguration(confForJobs);
 


[16/50] [abbrv] tez git commit: TEZ-2237. Valid events should be sent out when an Output is not started. (sseth)

Posted by ss...@apache.org.
TEZ-2237. Valid events should be sent out when an Output is not started. (sseth)


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

Branch: refs/heads/TEZ-2003
Commit: e762a35fd81228f85c455b612f7cc8ff6a305e41
Parents: ba6d7e0
Author: Siddharth Seth <ss...@apache.org>
Authored: Mon May 4 16:30:06 2015 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Mon May 4 16:30:06 2015 -0700

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../library/common/shuffle/ShuffleUtils.java    | 55 ++++++++++++++++
 .../output/OrderedPartitionedKVOutput.java      | 16 +++--
 .../library/output/UnorderedKVOutput.java       | 13 +++-
 .../output/UnorderedPartitionedKVOutput.java    | 11 +++-
 .../library/output/OutputTestHelpers.java       | 47 ++++++++++++++
 .../output/TestOrderedPartitionedKVOutput2.java | 67 ++++++++++++++++++++
 .../library/output/TestUnorderedKVOutput2.java  | 60 ++++++++++++++++++
 .../TestUnorderedPartitionedKVOutput2.java      | 62 ++++++++++++++++++
 9 files changed, 325 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/e762a35f/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 0027e98..5b18258 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -161,6 +161,7 @@ Release 0.6.1: Unreleased
 INCOMPATIBLE CHANGES
 
 ALL CHANGES:
+  TEZ-2237. Valid events should be sent out when an Output is not started.
   TEZ-1988. Tez UI: does not work when using file:// in a browser
   TEZ-2390. tez-tools swimlane tool fails to parse large jobs >8K containers
   TEZ-2256. Avoid use of BufferTooSmallException to signal end of buffer in UnorderedPartitionedKVWriter

http://git-wip-us.apache.org/repos/asf/tez/blob/e762a35f/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/ShuffleUtils.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/ShuffleUtils.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/ShuffleUtils.java
index 9a8b6b5..46489ed 100644
--- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/ShuffleUtils.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/ShuffleUtils.java
@@ -33,6 +33,7 @@ import javax.crypto.SecretKey;
 
 import com.google.common.base.Preconditions;
 import com.google.protobuf.ByteString;
+import org.apache.tez.runtime.api.events.DataMovementEvent;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -333,6 +334,60 @@ public class ShuffleUtils {
   }
 
   /**
+   * Generate events for outputs which have not been started.
+   * @param eventList
+   * @param numPhysicalOutputs
+   * @param context
+   * @param generateVmEvent whether to generate a vm event or not
+   * @param isCompositeEvent whether to generate a CompositeDataMovementEvent or a DataMovementEvent
+   * @throws IOException
+   */
+  public static void generateEventsForNonStartedOutput(List<Event> eventList,
+                                                       int numPhysicalOutputs,
+                                                       OutputContext context,
+                                                       boolean generateVmEvent,
+                                                       boolean isCompositeEvent) throws
+      IOException {
+    DataMovementEventPayloadProto.Builder payloadBuilder = DataMovementEventPayloadProto
+        .newBuilder();
+
+
+    // Construct the VertexManager event if required.
+    if (generateVmEvent) {
+      ShuffleUserPayloads.VertexManagerEventPayloadProto.Builder vmBuilder =
+          ShuffleUserPayloads.VertexManagerEventPayloadProto.newBuilder();
+      vmBuilder.setOutputSize(0);
+      VertexManagerEvent vmEvent = VertexManagerEvent.create(
+          context.getDestinationVertexName(),
+          vmBuilder.build().toByteString().asReadOnlyByteBuffer());
+      eventList.add(vmEvent);
+    }
+
+    // Construct the DataMovementEvent
+    // Always set empty partition information since no files were generated.
+    LOG.info("Setting all {} partitions as empty for non-started output", numPhysicalOutputs);
+    BitSet emptyPartitionDetails = new BitSet(numPhysicalOutputs);
+    emptyPartitionDetails.set(0, numPhysicalOutputs, true);
+    ByteString emptyPartitionsBytesString =
+        TezCommonUtils.compressByteArrayToByteString(
+            TezUtilsInternal.toByteArray(emptyPartitionDetails));
+    payloadBuilder.setEmptyPartitions(emptyPartitionsBytesString);
+    payloadBuilder.setRunDuration(0);
+    DataMovementEventPayloadProto payloadProto = payloadBuilder.build();
+    ByteBuffer dmePayload = payloadProto.toByteString().asReadOnlyByteBuffer();
+
+
+    if (isCompositeEvent) {
+      CompositeDataMovementEvent cdme =
+          CompositeDataMovementEvent.create(0, numPhysicalOutputs, dmePayload);
+      eventList.add(cdme);
+    } else {
+      DataMovementEvent dme = DataMovementEvent.create(0, dmePayload);
+      eventList.add(dme);
+    }
+  }
+
+  /**
    * Generate events when spill happens
    *
    * @param eventList events would be added to this list

http://git-wip-us.apache.org/repos/asf/tez/blob/e762a35f/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/OrderedPartitionedKVOutput.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/OrderedPartitionedKVOutput.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/OrderedPartitionedKVOutput.java
index 40edc76..6227fb9 100644
--- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/OrderedPartitionedKVOutput.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/OrderedPartitionedKVOutput.java
@@ -185,11 +185,13 @@ public class OrderedPartitionedKVOutput extends AbstractLogicalOutput {
       this.endTime = System.nanoTime();
       returnEvents = generateEvents();
     } else {
-      LOG.warn("Attempting to close output " + getContext().getDestinationVertexName()
-          + " before it was started");
-      returnEvents = Collections.emptyList();
+      LOG.warn(
+          "Attempting to close output {} of type {} before it was started. Generating empty events",
+          getContext().getDestinationVertexName(), this.getClass().getSimpleName());
+      returnEvents = generateEmptyEvents();
     }
-    
+
+    // This works for non-started outputs since new counters will be created with an initial value of 0
     long outputSize = getContext().getCounters().findCounter(TaskCounter.OUTPUT_BYTES).getValue();
     getContext().getStatisticsReporter().reportDataSize(outputSize);
     long outputRecords = getContext().getCounters()
@@ -210,6 +212,12 @@ public class OrderedPartitionedKVOutput extends AbstractLogicalOutput {
     return eventList;
   }
 
+  private List<Event> generateEmptyEvents() throws IOException {
+    List<Event> eventList = Lists.newLinkedList();
+    ShuffleUtils.generateEventsForNonStartedOutput(eventList, getNumPhysicalOutputs(), getContext(), true, true);
+    return eventList;
+  }
+
 
   private static final Set<String> confKeys = new HashSet<String>();
 

http://git-wip-us.apache.org/repos/asf/tez/blob/e762a35f/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/UnorderedKVOutput.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/UnorderedKVOutput.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/UnorderedKVOutput.java
index 2c26374..08e6ec0 100644
--- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/UnorderedKVOutput.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/UnorderedKVOutput.java
@@ -20,10 +20,12 @@ package org.apache.tez.runtime.library.output;
 
 import java.util.Collections;
 import java.util.HashSet;
+import java.util.LinkedList;
 import java.util.List;
 import java.util.Set;
 import java.util.concurrent.atomic.AtomicBoolean;
 
+import org.apache.tez.runtime.library.common.shuffle.ShuffleUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
@@ -125,9 +127,16 @@ public class UnorderedKVOutput extends AbstractLogicalOutput {
       //TODO: Do we need to support sending payloads via events?
       returnEvents = kvWriter.close();
     } else {
-      returnEvents = Collections.emptyList();
+      LOG.warn(
+          "Attempting to close output {} of type {} before it was started. Generating empty events",
+          getContext().getDestinationVertexName(), this.getClass().getSimpleName());
+      returnEvents = new LinkedList<Event>();
+      ShuffleUtils
+          .generateEventsForNonStartedOutput(returnEvents, getNumPhysicalOutputs(), getContext(),
+              false, false);
     }
-    
+
+    // This works for non-started outputs since new counters will be created with an initial value of 0
     long outputSize = getContext().getCounters().findCounter(TaskCounter.OUTPUT_BYTES).getValue();
     getContext().getStatisticsReporter().reportDataSize(outputSize);
     long outputRecords = getContext().getCounters()

http://git-wip-us.apache.org/repos/asf/tez/blob/e762a35f/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/UnorderedPartitionedKVOutput.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/UnorderedPartitionedKVOutput.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/UnorderedPartitionedKVOutput.java
index 34f2e3e..38450ee 100644
--- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/UnorderedPartitionedKVOutput.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/UnorderedPartitionedKVOutput.java
@@ -20,12 +20,14 @@ package org.apache.tez.runtime.library.output;
 
 import java.util.Collections;
 import java.util.HashSet;
+import java.util.LinkedList;
 import java.util.List;
 import java.util.Set;
 import java.util.concurrent.atomic.AtomicBoolean;
 
 import com.google.common.base.Preconditions;
 
+import org.apache.tez.runtime.library.common.shuffle.ShuffleUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
@@ -102,9 +104,16 @@ public class UnorderedPartitionedKVOutput extends AbstractLogicalOutput {
     if (isStarted.get()) {
       returnEvents = kvWriter.close();
     } else {
-      returnEvents = Collections.emptyList();
+      LOG.warn(
+          "Attempting to close output {} of type {} before it was started. Generating empty events",
+          getContext().getDestinationVertexName(), this.getClass().getSimpleName());
+      returnEvents = new LinkedList<Event>();
+      ShuffleUtils
+          .generateEventsForNonStartedOutput(returnEvents, getNumPhysicalOutputs(), getContext(),
+              false, true);
     }
 
+    // This works for non-started outputs since new counters will be created with an initial value of 0
     long outputSize = getContext().getCounters().findCounter(TaskCounter.OUTPUT_BYTES).getValue();
     getContext().getStatisticsReporter().reportDataSize(outputSize);
     long outputRecords = getContext().getCounters()

http://git-wip-us.apache.org/repos/asf/tez/blob/e762a35f/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/output/OutputTestHelpers.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/output/OutputTestHelpers.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/output/OutputTestHelpers.java
new file mode 100644
index 0000000..db9a0ed
--- /dev/null
+++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/output/OutputTestHelpers.java
@@ -0,0 +1,47 @@
+/*
+ * 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.
+ */
+
+package org.apache.tez.runtime.library.output;
+
+import static org.mockito.Mockito.doReturn;
+import static org.mockito.Mockito.mock;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.tez.common.TezUtils;
+import org.apache.tez.common.counters.TezCounters;
+import org.apache.tez.dag.api.TezConfiguration;
+import org.apache.tez.dag.api.UserPayload;
+import org.apache.tez.runtime.api.OutputContext;
+import org.apache.tez.runtime.api.OutputStatisticsReporter;
+
+public class OutputTestHelpers {
+  static OutputContext createOutputContext() throws IOException {
+    OutputContext outputContext = mock(OutputContext.class);
+    Configuration conf = new TezConfiguration();
+    UserPayload payLoad = TezUtils.createUserPayloadFromConf(conf);
+    String[] workingDirs = new String[]{"workDir1"};
+    OutputStatisticsReporter statsReporter = mock(OutputStatisticsReporter.class);
+    TezCounters counters = new TezCounters();
+
+    doReturn("destinationVertex").when(outputContext).getDestinationVertexName();
+    doReturn(payLoad).when(outputContext).getUserPayload();
+    doReturn(workingDirs).when(outputContext).getWorkDirs();
+    doReturn(200 * 1024 * 1024l).when(outputContext).getTotalMemoryAvailableToTask();
+    doReturn(counters).when(outputContext).getCounters();
+    doReturn(statsReporter).when(outputContext).getStatisticsReporter();
+    return outputContext;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tez/blob/e762a35f/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/output/TestOrderedPartitionedKVOutput2.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/output/TestOrderedPartitionedKVOutput2.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/output/TestOrderedPartitionedKVOutput2.java
new file mode 100644
index 0000000..8e76a8b
--- /dev/null
+++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/output/TestOrderedPartitionedKVOutput2.java
@@ -0,0 +1,67 @@
+/*
+ * 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.
+ */
+
+package org.apache.tez.runtime.library.output;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.BitSet;
+import java.util.List;
+
+import com.google.protobuf.ByteString;
+import org.apache.tez.common.TezCommonUtils;
+import org.apache.tez.common.TezUtilsInternal;
+import org.apache.tez.runtime.api.Event;
+import org.apache.tez.runtime.api.OutputContext;
+import org.apache.tez.runtime.api.events.CompositeDataMovementEvent;
+import org.apache.tez.runtime.api.events.VertexManagerEvent;
+import org.apache.tez.runtime.library.shuffle.impl.ShuffleUserPayloads;
+import org.junit.Test;
+
+// Tests which don't require parameterization
+public class TestOrderedPartitionedKVOutput2 {
+
+
+  @Test(timeout = 5000)
+  public void testNonStartedOutput() throws IOException {
+    OutputContext outputContext = OutputTestHelpers.createOutputContext();
+    int numPartitions = 10;
+    OrderedPartitionedKVOutput output = new OrderedPartitionedKVOutput(outputContext, numPartitions);
+    output.initialize();
+    List<Event> events = output.close();
+    assertEquals(2, events.size());
+    Event event1 = events.get(0);
+    assertTrue(event1 instanceof VertexManagerEvent);
+    Event event2 = events.get(1);
+    assertTrue(event2 instanceof CompositeDataMovementEvent);
+    CompositeDataMovementEvent cdme = (CompositeDataMovementEvent) event2;
+    ByteBuffer bb = cdme.getUserPayload();
+    ShuffleUserPayloads.DataMovementEventPayloadProto shufflePayload =
+        ShuffleUserPayloads.DataMovementEventPayloadProto.parseFrom(ByteString.copyFrom(bb));
+    assertTrue(shufflePayload.hasEmptyPartitions());
+
+    byte[] emptyPartitions = TezCommonUtils.decompressByteStringToByteArray(shufflePayload
+        .getEmptyPartitions());
+    BitSet emptyPartionsBitSet = TezUtilsInternal.fromByteArray(emptyPartitions);
+    assertEquals(numPartitions, emptyPartionsBitSet.cardinality());
+    for (int i = 0 ; i < numPartitions ; i++) {
+      assertTrue(emptyPartionsBitSet.get(i));
+    }
+  }
+
+
+}

http://git-wip-us.apache.org/repos/asf/tez/blob/e762a35f/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/output/TestUnorderedKVOutput2.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/output/TestUnorderedKVOutput2.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/output/TestUnorderedKVOutput2.java
new file mode 100644
index 0000000..ecc1241
--- /dev/null
+++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/output/TestUnorderedKVOutput2.java
@@ -0,0 +1,60 @@
+/*
+ * 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.
+ */
+
+package org.apache.tez.runtime.library.output;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.nio.ByteBuffer;
+import java.util.BitSet;
+import java.util.List;
+
+import com.google.protobuf.ByteString;
+import org.apache.tez.common.TezCommonUtils;
+import org.apache.tez.common.TezUtilsInternal;
+import org.apache.tez.runtime.api.Event;
+import org.apache.tez.runtime.api.OutputContext;
+import org.apache.tez.runtime.api.events.DataMovementEvent;
+import org.apache.tez.runtime.library.shuffle.impl.ShuffleUserPayloads;
+import org.junit.Test;
+
+// Tests which don't require parameterization
+public class TestUnorderedKVOutput2 {
+
+  @Test(timeout = 5000)
+  public void testNonStartedOutput() throws Exception {
+    OutputContext outputContext = OutputTestHelpers.createOutputContext();
+    int numPartitions = 1;
+    UnorderedKVOutput output = new UnorderedKVOutput(outputContext, numPartitions);
+    output.initialize();
+    List<Event> events = output.close();
+    assertEquals(1, events.size());
+    Event event1 = events.get(0);
+    assertTrue(event1 instanceof DataMovementEvent);
+    DataMovementEvent dme = (DataMovementEvent) event1;
+    ByteBuffer bb = dme.getUserPayload();
+    ShuffleUserPayloads.DataMovementEventPayloadProto shufflePayload =
+        ShuffleUserPayloads.DataMovementEventPayloadProto.parseFrom(ByteString.copyFrom(bb));
+    assertTrue(shufflePayload.hasEmptyPartitions());
+
+    byte[] emptyPartitions = TezCommonUtils.decompressByteStringToByteArray(shufflePayload
+        .getEmptyPartitions());
+    BitSet emptyPartionsBitSet = TezUtilsInternal.fromByteArray(emptyPartitions);
+    assertEquals(numPartitions, emptyPartionsBitSet.cardinality());
+    for (int i = 0 ; i < numPartitions ; i++) {
+      assertTrue(emptyPartionsBitSet.get(i));
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/tez/blob/e762a35f/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/output/TestUnorderedPartitionedKVOutput2.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/output/TestUnorderedPartitionedKVOutput2.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/output/TestUnorderedPartitionedKVOutput2.java
new file mode 100644
index 0000000..eec4bf5
--- /dev/null
+++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/output/TestUnorderedPartitionedKVOutput2.java
@@ -0,0 +1,62 @@
+/*
+ * 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.
+ */
+
+package org.apache.tez.runtime.library.output;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.nio.ByteBuffer;
+import java.util.BitSet;
+import java.util.List;
+
+import com.google.protobuf.ByteString;
+import org.apache.tez.common.TezCommonUtils;
+import org.apache.tez.common.TezUtilsInternal;
+import org.apache.tez.runtime.api.Event;
+import org.apache.tez.runtime.api.OutputContext;
+import org.apache.tez.runtime.api.events.CompositeDataMovementEvent;
+import org.apache.tez.runtime.library.shuffle.impl.ShuffleUserPayloads;
+import org.junit.Test;
+
+// Tests which don't require parameterization
+public class TestUnorderedPartitionedKVOutput2 {
+
+
+  @Test(timeout = 5000)
+  public void testNonStartedOutput() throws Exception {
+    OutputContext outputContext = OutputTestHelpers.createOutputContext();
+    int numPartitions = 1;
+    UnorderedPartitionedKVOutput output =
+        new UnorderedPartitionedKVOutput(outputContext, numPartitions);
+    output.initialize();
+    List<Event> events = output.close();
+    assertEquals(1, events.size());
+    Event event1 = events.get(0);
+    assertTrue(event1 instanceof CompositeDataMovementEvent);
+    CompositeDataMovementEvent dme = (CompositeDataMovementEvent) event1;
+    ByteBuffer bb = dme.getUserPayload();
+    ShuffleUserPayloads.DataMovementEventPayloadProto shufflePayload =
+        ShuffleUserPayloads.DataMovementEventPayloadProto.parseFrom(ByteString.copyFrom(bb));
+    assertTrue(shufflePayload.hasEmptyPartitions());
+
+    byte[] emptyPartitions = TezCommonUtils.decompressByteStringToByteArray(shufflePayload
+        .getEmptyPartitions());
+    BitSet emptyPartionsBitSet = TezUtilsInternal.fromByteArray(emptyPartitions);
+    assertEquals(numPartitions, emptyPartionsBitSet.cardinality());
+    for (int i = 0; i < numPartitions; i++) {
+      assertTrue(emptyPartionsBitSet.get(i));
+    }
+  }
+}


[23/50] [abbrv] tez git commit: TEZ-2384. Add warning message in the case of prewarn under non-session mode. (Jeff Zhang via hitesh)

Posted by ss...@apache.org.
TEZ-2384. Add warning message in the case of prewarn under non-session mode. (Jeff Zhang via hitesh)


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

Branch: refs/heads/TEZ-2003
Commit: 077b352fbd7ec44a7e5c3224837d5048829f09c4
Parents: bce7ff2
Author: Hitesh Shah <hi...@apache.org>
Authored: Tue May 5 15:33:25 2015 -0700
Committer: Hitesh Shah <hi...@apache.org>
Committed: Tue May 5 15:33:25 2015 -0700

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


http://git-wip-us.apache.org/repos/asf/tez/blob/077b352f/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index c4ae033..8633419 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -9,6 +9,7 @@ INCOMPATIBLE CHANGES
   TEZ-1993. Implement a pluggable InputSizeEstimator for grouping fairly
 
 ALL CHANGES:
+  TEZ-2384. Add warning message in the case of prewarn under non-session mode.
   TEZ-2415. PMC RDF needs to use asfext:pmc, not asfext:PMC.
   TEZ-1752. Inputs / Outputs in the Runtime library should be interruptable.
   TEZ-2401. Tez UI: All-dag page has duration keep counting for KILLED dag.

http://git-wip-us.apache.org/repos/asf/tez/blob/077b352f/tez-api/src/main/java/org/apache/tez/client/TezClient.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/client/TezClient.java b/tez-api/src/main/java/org/apache/tez/client/TezClient.java
index b1e4239..ab76ca8 100644
--- a/tez-api/src/main/java/org/apache/tez/client/TezClient.java
+++ b/tez-api/src/main/java/org/apache/tez/client/TezClient.java
@@ -637,6 +637,7 @@ public class TezClient {
     if (!isSession) {
       // do nothing for non session mode. This is there to let the code 
       // work correctly in both modes
+      LOG.warn("preWarm is not supported in non-session mode, please use session-mode of TezClient");
       return;
     }
     


[24/50] [abbrv] tez git commit: Revert part of change of TEZ-2221 (zjffdu)

Posted by ss...@apache.org.
Revert part of change of TEZ-2221 (zjffdu)


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

Branch: refs/heads/TEZ-2003
Commit: 406bba9ac86b8c0e265c2b1850928de477ce0250
Parents: 077b352
Author: Jeff Zhang <zj...@apache.org>
Authored: Wed May 6 09:00:33 2015 +0800
Committer: Jeff Zhang <zj...@apache.org>
Committed: Wed May 6 09:00:33 2015 +0800

----------------------------------------------------------------------
 .../src/main/java/org/apache/tez/dag/api/DAG.java | 15 ---------------
 .../test/java/org/apache/tez/dag/api/TestDAG.java | 18 ++++--------------
 2 files changed, 4 insertions(+), 29 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/406bba9a/tez-api/src/main/java/org/apache/tez/dag/api/DAG.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/DAG.java b/tez-api/src/main/java/org/apache/tez/dag/api/DAG.java
index 92f865d..8ee1682 100644
--- a/tez-api/src/main/java/org/apache/tez/dag/api/DAG.java
+++ b/tez-api/src/main/java/org/apache/tez/dag/api/DAG.java
@@ -85,8 +85,6 @@ public class DAG {
   final Collection<URI> urisForCredentials = new HashSet<URI>();
   Credentials credentials = new Credentials();
   Set<VertexGroup> vertexGroups = Sets.newHashSet();
-  // to verify the vertex Group memberSet should be unique
-  private Set<Set<String>> vertexGroupMemberSets = Sets.newHashSet();
 
   Set<GroupInputEdge> groupInputEdges = Sets.newHashSet();
 
@@ -181,19 +179,6 @@ public class DAG {
    * @return {@link DAG}
    */
   public synchronized VertexGroup createVertexGroup(String name, Vertex... members) {
-    // vertex group member set should be unique
-    Collection<String> memberNames =
-        Collections2.transform(Lists.newArrayList(members), new Function<Vertex, String>() {
-      @Override
-      public String apply(Vertex v) {
-        return v.getName();
-      }
-    });
-    if (!vertexGroupMemberSets.add(Sets.newHashSet(memberNames))){
-      throw new IllegalStateException(
-          "VertexGroup " + memberNames + " already defined as another group!");
-    }
-
     // vertex group name should be unique.
     VertexGroup uv = new VertexGroup(name, members);
     if (!vertexGroups.add(uv)) {

http://git-wip-us.apache.org/repos/asf/tez/blob/406bba9a/tez-api/src/test/java/org/apache/tez/dag/api/TestDAG.java
----------------------------------------------------------------------
diff --git a/tez-api/src/test/java/org/apache/tez/dag/api/TestDAG.java b/tez-api/src/test/java/org/apache/tez/dag/api/TestDAG.java
index 245d9c0..0a66405 100644
--- a/tez-api/src/test/java/org/apache/tez/dag/api/TestDAG.java
+++ b/tez-api/src/test/java/org/apache/tez/dag/api/TestDAG.java
@@ -87,13 +87,7 @@ public class TestDAG {
 
     DAG dag = DAG.create("testDAG");
     dag.createVertexGroup("group_1", v1,v2);
-    try {
-      dag.createVertexGroup("group_1", v1,v2);
-      Assert.fail("should fail it due to duplicated VertexGroups");
-    } catch (Exception e) {
-      e.printStackTrace();
-      Assert.assertEquals("VertexGroup [v1, v2] already defined as another group!", e.getMessage());
-    }
+
     try {
       dag.createVertexGroup("group_1", v2, v3);
       Assert.fail("should fail it due to duplicated VertexGroups");
@@ -101,13 +95,9 @@ public class TestDAG {
       e.printStackTrace();
       Assert.assertEquals("VertexGroup group_1 already defined!", e.getMessage());
     }
-    try {
-      dag.createVertexGroup("group_2", v1, v2);
-      Assert.fail("should fail it due to duplicated VertexGroups");
-    } catch (Exception e) {
-      e.printStackTrace();
-      Assert.assertEquals("VertexGroup [v1, v2] already defined as another group!", e.getMessage());
-    }
+    // it is possible to create vertex group with same member but different group name 
+    dag.createVertexGroup("group_2", v1, v2);
+
   }
 
   @Test(timeout = 5000)


[28/50] [abbrv] tez git commit: TEZ-2090. Add tests for jobs running in external services. (sseth)

Posted by ss...@apache.org.
http://git-wip-us.apache.org/repos/asf/tez/blob/8b40191d/tez-ext-service-tests/src/test/java/org/apache/tez/tests/TestExternalTezServices.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/tests/TestExternalTezServices.java b/tez-ext-service-tests/src/test/java/org/apache/tez/tests/TestExternalTezServices.java
new file mode 100644
index 0000000..a93c1a4
--- /dev/null
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/tests/TestExternalTezServices.java
@@ -0,0 +1,183 @@
+/*
+ * 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.
+ */
+
+package org.apache.tez.tests;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+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.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.tez.client.TezClient;
+import org.apache.tez.dag.api.TezConfiguration;
+import org.apache.tez.dag.api.TezException;
+import org.apache.tez.dag.app.launcher.TezTestServiceNoOpContainerLauncher;
+import org.apache.tez.dag.app.rm.TezTestServiceTaskSchedulerService;
+import org.apache.tez.dag.app.taskcomm.TezTestServiceTaskCommunicatorImpl;
+import org.apache.tez.examples.HashJoinExample;
+import org.apache.tez.examples.JoinDataGen;
+import org.apache.tez.examples.JoinValidate;
+import org.apache.tez.service.MiniTezTestServiceCluster;
+import org.apache.tez.test.MiniTezCluster;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class TestExternalTezServices {
+
+  private static final Log LOG = LogFactory.getLog(TestExternalTezServices.class);
+
+  private static MiniTezCluster tezCluster;
+  private static MiniDFSCluster dfsCluster;
+  private static MiniTezTestServiceCluster tezTestServiceCluster;
+
+  private static Configuration clusterConf = new Configuration();
+  private static Configuration confForJobs;
+
+  private static FileSystem remoteFs;
+  private static FileSystem localFs;
+
+  private static TezClient sharedTezClient;
+
+  private static String TEST_ROOT_DIR = "target" + Path.SEPARATOR + TestExternalTezServices.class.getName()
+      + "-tmpDir";
+
+  @BeforeClass
+  public static void setup() throws IOException, TezException, InterruptedException {
+
+    localFs = FileSystem.getLocal(clusterConf);
+
+    try {
+      clusterConf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR, TEST_ROOT_DIR);
+      dfsCluster =
+          new MiniDFSCluster.Builder(clusterConf).numDataNodes(1).format(true).racks(null).build();
+      remoteFs = dfsCluster.getFileSystem();
+      LOG.info("MiniDFSCluster started");
+    } catch (IOException io) {
+      throw new RuntimeException("problem starting mini dfs cluster", io);
+    }
+
+    tezCluster = new MiniTezCluster(TestExternalTezServices.class.getName(), 1, 1, 1);
+    Configuration conf = new Configuration();
+    conf.set("fs.defaultFS", remoteFs.getUri().toString()); // use HDFS
+    tezCluster.init(conf);
+    tezCluster.start();
+    LOG.info("MiniTezCluster started");
+
+    clusterConf.set("fs.defaultFS", remoteFs.getUri().toString()); // use HDFS
+    for (Map.Entry<String, String> entry : tezCluster.getConfig()) {
+      clusterConf.set(entry.getKey(), entry.getValue());
+    }
+    long jvmMax = Runtime.getRuntime().maxMemory();
+
+    tezTestServiceCluster = MiniTezTestServiceCluster
+        .create(TestExternalTezServices.class.getSimpleName(), 3, ((long) (jvmMax * 0.5d)), 1);
+    tezTestServiceCluster.init(clusterConf);
+    tezTestServiceCluster.start();
+    LOG.info("MiniTezTestServer started");
+
+    confForJobs = new Configuration(clusterConf);
+    for (Map.Entry<String, String> entry : tezTestServiceCluster
+        .getClusterSpecificConfiguration()) {
+      confForJobs.set(entry.getKey(), entry.getValue());
+    }
+
+    // TODO TEZ-2003 Once per vertex configuration is possible, run separate tests for push vs pull (regular threaded execution)
+
+    Path stagingDirPath = new Path("/tmp/tez-staging-dir");
+    remoteFs.mkdirs(stagingDirPath);
+    // This is currently configured to push tasks into the Service, and then use the standard RPC
+    confForJobs.set(TezConfiguration.TEZ_AM_STAGING_DIR, stagingDirPath.toString());
+    confForJobs.set(TezConfiguration.TEZ_AM_TASK_SCHEDULER_CLASS,
+        TezTestServiceTaskSchedulerService.class.getName());
+    confForJobs.set(TezConfiguration.TEZ_AM_CONTAINER_LAUNCHER_CLASS,
+        TezTestServiceNoOpContainerLauncher.class.getName());
+    confForJobs.set(TezConfiguration.TEZ_AM_TASK_COMMUNICATOR_CLASS,
+        TezTestServiceTaskCommunicatorImpl.class.getName());
+
+    TezConfiguration tezConf = new TezConfiguration(confForJobs);
+
+    sharedTezClient = TezClient.create(TestExternalTezServices.class.getSimpleName() + "_session",
+        tezConf, true);
+    sharedTezClient.start();
+    LOG.info("Shared TezSession started");
+    sharedTezClient.waitTillReady();
+    LOG.info("Shared TezSession ready for submission");
+
+  }
+
+  @AfterClass
+  public static void tearDown() throws IOException, TezException {
+    if (sharedTezClient != null) {
+      sharedTezClient.stop();
+      sharedTezClient = null;
+    }
+
+    if (tezTestServiceCluster != null) {
+      tezTestServiceCluster.stop();
+      tezTestServiceCluster = null;
+    }
+
+    if (tezCluster != null) {
+      tezCluster.stop();
+      tezCluster = null;
+    }
+    if (dfsCluster != null) {
+      dfsCluster.shutdown();
+      dfsCluster = null;
+    }
+    // TODO Add cleanup code.
+  }
+
+
+  @Test(timeout = 60000)
+  public void test1() throws Exception {
+    Path testDir = new Path("/tmp/testHashJoinExample");
+
+    remoteFs.mkdirs(testDir);
+
+    Path dataPath1 = new Path(testDir, "inPath1");
+    Path dataPath2 = new Path(testDir, "inPath2");
+    Path expectedOutputPath = new Path(testDir, "expectedOutputPath");
+    Path outPath = new Path(testDir, "outPath");
+
+    TezConfiguration tezConf = new TezConfiguration(confForJobs);
+
+    JoinDataGen dataGen = new JoinDataGen();
+    String[] dataGenArgs = new String[]{
+        dataPath1.toString(), "1048576", dataPath2.toString(), "524288",
+        expectedOutputPath.toString(), "2"};
+    assertEquals(0, dataGen.run(tezConf, dataGenArgs, sharedTezClient));
+
+    HashJoinExample joinExample = new HashJoinExample();
+    String[] args = new String[]{
+        dataPath1.toString(), dataPath2.toString(), "2", outPath.toString()};
+    assertEquals(0, joinExample.run(tezConf, args, sharedTezClient));
+
+    JoinValidate joinValidate = new JoinValidate();
+    String[] validateArgs = new String[]{
+        expectedOutputPath.toString(), outPath.toString(), "3"};
+    assertEquals(0, joinValidate.run(tezConf, validateArgs, sharedTezClient));
+
+    // Ensure this was actually submitted to the external cluster
+    assertTrue(tezTestServiceCluster.getNumSubmissions() > 0);
+  }
+}

http://git-wip-us.apache.org/repos/asf/tez/blob/8b40191d/tez-ext-service-tests/src/test/java/org/apache/tez/util/ProtoConverters.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/util/ProtoConverters.java b/tez-ext-service-tests/src/test/java/org/apache/tez/util/ProtoConverters.java
new file mode 100644
index 0000000..60ebc53
--- /dev/null
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/util/ProtoConverters.java
@@ -0,0 +1,172 @@
+/*
+ * 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.
+ */
+
+package org.apache.tez.util;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.tez.dag.api.DagTypeConverters;
+import org.apache.tez.dag.api.InputDescriptor;
+import org.apache.tez.dag.api.OutputDescriptor;
+import org.apache.tez.dag.api.ProcessorDescriptor;
+import org.apache.tez.dag.records.TezTaskAttemptID;
+import org.apache.tez.runtime.api.impl.GroupInputSpec;
+import org.apache.tez.runtime.api.impl.InputSpec;
+import org.apache.tez.runtime.api.impl.OutputSpec;
+import org.apache.tez.runtime.api.impl.TaskSpec;
+import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos.GroupInputSpecProto;
+import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos.IOSpecProto;
+import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos.TaskSpecProto;
+
+public class ProtoConverters {
+
+  public static TaskSpec getTaskSpecfromProto(TaskSpecProto taskSpecProto) {
+    TezTaskAttemptID taskAttemptID =
+        TezTaskAttemptID.fromString(taskSpecProto.getTaskAttemptIdString());
+
+    ProcessorDescriptor processorDescriptor = null;
+    if (taskSpecProto.hasProcessorDescriptor()) {
+      processorDescriptor = DagTypeConverters
+          .convertProcessorDescriptorFromDAGPlan(taskSpecProto.getProcessorDescriptor());
+    }
+
+    List<InputSpec> inputSpecList = new ArrayList<InputSpec>(taskSpecProto.getInputSpecsCount());
+    if (taskSpecProto.getInputSpecsCount() > 0) {
+      for (IOSpecProto inputSpecProto : taskSpecProto.getInputSpecsList()) {
+        inputSpecList.add(getInputSpecFromProto(inputSpecProto));
+      }
+    }
+
+    List<OutputSpec> outputSpecList =
+        new ArrayList<OutputSpec>(taskSpecProto.getOutputSpecsCount());
+    if (taskSpecProto.getOutputSpecsCount() > 0) {
+      for (IOSpecProto outputSpecProto : taskSpecProto.getOutputSpecsList()) {
+        outputSpecList.add(getOutputSpecFromProto(outputSpecProto));
+      }
+    }
+
+    List<GroupInputSpec> groupInputSpecs =
+        new ArrayList<GroupInputSpec>(taskSpecProto.getGroupedInputSpecsCount());
+    if (taskSpecProto.getGroupedInputSpecsCount() > 0) {
+      for (GroupInputSpecProto groupInputSpecProto : taskSpecProto.getGroupedInputSpecsList()) {
+        groupInputSpecs.add(getGroupInputSpecFromProto(groupInputSpecProto));
+      }
+    }
+
+    TaskSpec taskSpec =
+        new TaskSpec(taskAttemptID, taskSpecProto.getDagName(), taskSpecProto.getVertexName(),
+            taskSpecProto.getVertexParallelism(), processorDescriptor, inputSpecList,
+            outputSpecList, groupInputSpecs);
+    return taskSpec;
+  }
+
+  public static TaskSpecProto convertTaskSpecToProto(TaskSpec taskSpec) {
+    TaskSpecProto.Builder builder = TaskSpecProto.newBuilder();
+    builder.setTaskAttemptIdString(taskSpec.getTaskAttemptID().toString());
+    builder.setDagName(taskSpec.getDAGName());
+    builder.setVertexName(taskSpec.getVertexName());
+    builder.setVertexParallelism(taskSpec.getVertexParallelism());
+
+    if (taskSpec.getProcessorDescriptor() != null) {
+      builder.setProcessorDescriptor(
+          DagTypeConverters.convertToDAGPlan(taskSpec.getProcessorDescriptor()));
+    }
+
+    if (taskSpec.getInputs() != null && !taskSpec.getInputs().isEmpty()) {
+      for (InputSpec inputSpec : taskSpec.getInputs()) {
+        builder.addInputSpecs(convertInputSpecToProto(inputSpec));
+      }
+    }
+
+    if (taskSpec.getOutputs() != null && !taskSpec.getOutputs().isEmpty()) {
+      for (OutputSpec outputSpec : taskSpec.getOutputs()) {
+        builder.addOutputSpecs(convertOutputSpecToProto(outputSpec));
+      }
+    }
+
+    if (taskSpec.getGroupInputs() != null && !taskSpec.getGroupInputs().isEmpty()) {
+      for (GroupInputSpec groupInputSpec : taskSpec.getGroupInputs()) {
+        builder.addGroupedInputSpecs(convertGroupInputSpecToProto(groupInputSpec));
+
+      }
+    }
+    return builder.build();
+  }
+
+
+  public static InputSpec getInputSpecFromProto(IOSpecProto inputSpecProto) {
+    InputDescriptor inputDescriptor = null;
+    if (inputSpecProto.hasIoDescriptor()) {
+      inputDescriptor =
+          DagTypeConverters.convertInputDescriptorFromDAGPlan(inputSpecProto.getIoDescriptor());
+    }
+    InputSpec inputSpec = new InputSpec(inputSpecProto.getConnectedVertexName(), inputDescriptor,
+        inputSpecProto.getPhysicalEdgeCount());
+    return inputSpec;
+  }
+
+  public static IOSpecProto convertInputSpecToProto(InputSpec inputSpec) {
+    IOSpecProto.Builder builder = IOSpecProto.newBuilder();
+    if (inputSpec.getSourceVertexName() != null) {
+      builder.setConnectedVertexName(inputSpec.getSourceVertexName());
+    }
+    if (inputSpec.getInputDescriptor() != null) {
+      builder.setIoDescriptor(DagTypeConverters.convertToDAGPlan(inputSpec.getInputDescriptor()));
+    }
+    builder.setPhysicalEdgeCount(inputSpec.getPhysicalEdgeCount());
+    return builder.build();
+  }
+
+  public static OutputSpec getOutputSpecFromProto(IOSpecProto outputSpecProto) {
+    OutputDescriptor outputDescriptor = null;
+    if (outputSpecProto.hasIoDescriptor()) {
+      outputDescriptor =
+          DagTypeConverters.convertOutputDescriptorFromDAGPlan(outputSpecProto.getIoDescriptor());
+    }
+    OutputSpec outputSpec =
+        new OutputSpec(outputSpecProto.getConnectedVertexName(), outputDescriptor,
+            outputSpecProto.getPhysicalEdgeCount());
+    return outputSpec;
+  }
+
+  public static IOSpecProto convertOutputSpecToProto(OutputSpec outputSpec) {
+    IOSpecProto.Builder builder = IOSpecProto.newBuilder();
+    if (outputSpec.getDestinationVertexName() != null) {
+      builder.setConnectedVertexName(outputSpec.getDestinationVertexName());
+    }
+    if (outputSpec.getOutputDescriptor() != null) {
+      builder.setIoDescriptor(DagTypeConverters.convertToDAGPlan(outputSpec.getOutputDescriptor()));
+    }
+    builder.setPhysicalEdgeCount(outputSpec.getPhysicalEdgeCount());
+    return builder.build();
+  }
+
+  public static GroupInputSpec getGroupInputSpecFromProto(GroupInputSpecProto groupInputSpecProto) {
+    GroupInputSpec groupSpec = new GroupInputSpec(groupInputSpecProto.getGroupName(),
+        groupInputSpecProto.getGroupVerticesList(), DagTypeConverters
+        .convertInputDescriptorFromDAGPlan(groupInputSpecProto.getMergedInputDescriptor()));
+    return groupSpec;
+  }
+
+  public static GroupInputSpecProto convertGroupInputSpecToProto(GroupInputSpec groupInputSpec) {
+    GroupInputSpecProto.Builder builder = GroupInputSpecProto.newBuilder();
+    builder.setGroupName(groupInputSpec.getGroupName());
+    builder.addAllGroupVertices(groupInputSpec.getGroupVertices());
+    builder.setMergedInputDescriptor(
+        DagTypeConverters.convertToDAGPlan(groupInputSpec.getMergedInputDescriptor()));
+    return builder.build();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/tez/blob/8b40191d/tez-ext-service-tests/src/test/proto/TezDaemonProtocol.proto
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/proto/TezDaemonProtocol.proto b/tez-ext-service-tests/src/test/proto/TezDaemonProtocol.proto
new file mode 100644
index 0000000..2f8b2e6
--- /dev/null
+++ b/tez-ext-service-tests/src/test/proto/TezDaemonProtocol.proto
@@ -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.
+ */
+
+option java_package = "org.apache.tez.test.service.rpc";
+option java_outer_classname = "TezTestServiceProtocolProtos";
+option java_generic_services = true;
+option java_generate_equals_and_hash = true;
+
+import "DAGApiRecords.proto";
+
+message IOSpecProto {
+  optional string connected_vertex_name = 1;
+  optional TezEntityDescriptorProto io_descriptor = 2;
+  optional int32 physical_edge_count = 3;
+}
+
+message GroupInputSpecProto {
+  optional string group_name = 1;
+  repeated string group_vertices = 2;
+  optional TezEntityDescriptorProto merged_input_descriptor = 3;
+}
+
+message TaskSpecProto {
+  optional string task_attempt_id_string = 1;
+  optional string dag_name = 2;
+  optional string vertex_name = 3;
+  optional TezEntityDescriptorProto processor_descriptor = 4;
+  repeated IOSpecProto input_specs = 5;
+  repeated IOSpecProto output_specs = 6;
+  repeated GroupInputSpecProto grouped_input_specs = 7;
+  optional int32 vertex_parallelism = 8;
+}
+
+
+message SubmitWorkRequestProto {
+  optional string container_id_string = 1;
+  optional string am_host = 2;
+  optional int32 am_port = 3;
+  optional string token_identifier = 4;
+  optional bytes credentials_binary = 5;
+  optional string user = 6;
+  optional string application_id_string = 7;
+  optional int32 app_attempt_number = 8;
+  optional TaskSpecProto task_spec = 9;
+}
+
+message SubmitWorkResponseProto {
+}
+
+
+
+message RunContainerRequestProto {
+  optional string container_id_string = 1;
+  optional string am_host = 2;
+  optional int32 am_port = 3;
+  optional string token_identifier = 4;
+  optional bytes credentials_binary = 5;
+  optional string user = 6;
+  optional string application_id_string = 7;
+  optional int32 app_attempt_number = 8;
+}
+
+message RunContainerResponseProto {
+}
+
+service TezTestServiceProtocol {
+  rpc runContainer(RunContainerRequestProto) returns (RunContainerResponseProto);
+  rpc submitWork(SubmitWorkRequestProto) returns (SubmitWorkResponseProto);
+}

http://git-wip-us.apache.org/repos/asf/tez/blob/8b40191d/tez-ext-service-tests/src/test/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/resources/log4j.properties b/tez-ext-service-tests/src/test/resources/log4j.properties
new file mode 100644
index 0000000..531b68b
--- /dev/null
+++ b/tez-ext-service-tests/src/test/resources/log4j.properties
@@ -0,0 +1,19 @@
+#   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.
+
+# log4j configuration used during build and unit tests
+
+log4j.rootLogger=info,stdout
+log4j.threshhold=ALL
+log4j.appender.stdout=org.apache.log4j.ConsoleAppender
+log4j.appender.stdout.layout=org.apache.log4j.PatternLayout
+log4j.appender.stdout.layout.ConversionPattern=%d{ISO8601} %-5p [%t] %c{2} (%F:%M(%L)) - %m%n

http://git-wip-us.apache.org/repos/asf/tez/blob/8b40191d/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezChild.java
----------------------------------------------------------------------
diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezChild.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezChild.java
index fd55992..3cba3ce 100644
--- a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezChild.java
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezChild.java
@@ -389,7 +389,7 @@ public class TezChild {
     private final Throwable throwable;
     private final String errorMessage;
 
-    ContainerExecutionResult(ExitStatus exitStatus, @Nullable Throwable throwable,
+    public ContainerExecutionResult(ExitStatus exitStatus, @Nullable Throwable throwable,
                              @Nullable String errorMessage) {
       this.exitStatus = exitStatus;
       this.throwable = throwable;

http://git-wip-us.apache.org/repos/asf/tez/blob/8b40191d/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezTaskRunner.java
----------------------------------------------------------------------
diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezTaskRunner.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezTaskRunner.java
index de83889..f54814b 100644
--- a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezTaskRunner.java
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezTaskRunner.java
@@ -67,7 +67,7 @@ public class TezTaskRunner implements TezUmbilical, ErrorReporter {
   private final AtomicBoolean taskRunning;
   private final AtomicBoolean shutdownRequested = new AtomicBoolean(false);
 
-  TezTaskRunner(Configuration tezConf, UserGroupInformation ugi, String[] localDirs,
+  public TezTaskRunner(Configuration tezConf, UserGroupInformation ugi, String[] localDirs,
       TaskSpec taskSpec, int appAttemptNumber,
       Map<String, ByteBuffer> serviceConsumerMetadata, Map<String, String> serviceProviderEnvMap,
       Multimap<String, String> startedInputsMap, TaskReporter taskReporter,


[12/50] [abbrv] tez git commit: TEZ-1897. Create a concurrent version of AsyncDispatcher (bikas)

Posted by ss...@apache.org.
TEZ-1897. Create a concurrent version of AsyncDispatcher (bikas)


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

Branch: refs/heads/TEZ-2003
Commit: f6ea0fb3306faa709c445e4d76081de60545d760
Parents: 9f09027
Author: Bikas Saha <bi...@apache.org>
Authored: Sat May 2 15:21:17 2015 -0700
Committer: Bikas Saha <bi...@apache.org>
Committed: Sat May 2 15:21:17 2015 -0700

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 .../apache/tez/dag/api/TezConfiguration.java    |  14 +
 tez-common/findbugs-exclude.xml                 |   5 +
 .../org/apache/tez/common/AsyncDispatcher.java  |  85 ++++-
 .../tez/common/AsyncDispatcherConcurrent.java   | 368 +++++++++++++++++++
 .../org/apache/tez/common/TezAbstractEvent.java |  45 +++
 .../org/apache/tez/dag/records/TezTaskID.java   |  20 +-
 .../apache/tez/common/TestAsyncDispatcher.java  |   2 +-
 .../common/TestAsyncDispatcherConcurrent.java   | 194 ++++++++++
 .../org/apache/tez/dag/app/DAGAppMaster.java    |  19 +-
 .../org/apache/tez/dag/app/dag/TaskAttempt.java |   2 -
 .../tez/dag/app/dag/event/CallableEvent.java    |   4 +-
 .../dag/app/dag/event/DAGAppMasterEvent.java    |   5 +-
 .../apache/tez/dag/app/dag/event/DAGEvent.java  |   4 +-
 .../tez/dag/app/dag/event/SpeculatorEvent.java  |   4 +-
 .../tez/dag/app/dag/event/TaskAttemptEvent.java |   9 +-
 .../apache/tez/dag/app/dag/event/TaskEvent.java |   9 +-
 .../tez/dag/app/dag/event/VertexEvent.java      |   4 +-
 .../apache/tez/dag/app/dag/impl/DAGImpl.java    |   1 -
 .../tez/dag/app/dag/impl/TaskAttemptImpl.java   |  29 +-
 .../apache/tez/dag/app/dag/impl/TaskImpl.java   |  10 +-
 .../apache/tez/dag/app/dag/impl/VertexImpl.java |  13 +-
 .../tez/dag/app/TestMockDAGAppMaster.java       |  50 ++-
 .../tez/dag/app/dag/impl/TestTaskAttempt.java   |  42 ++-
 .../app/dag/impl/TestTaskAttemptRecovery.java   |   2 +-
 .../tez/dag/app/dag/impl/TestTaskImpl.java      |   4 +-
 .../tez/dag/app/dag/impl/TestTaskRecovery.java  |   2 +-
 27 files changed, 857 insertions(+), 90 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/f6ea0fb3/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 609db3c..8108ac8 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -9,6 +9,7 @@ INCOMPATIBLE CHANGES
   TEZ-1993. Implement a pluggable InputSizeEstimator for grouping fairly
 
 ALL CHANGES:
+  TEZ-1897. Create a concurrent version of AsyncDispatcher
   TEZ-2394. Issues when there is an error in VertexManager callbacks
   TEZ-2386. Tez UI: Inconsistent usage of icon colors
   TEZ-2395. Tez UI: Minimum/Maximum Duration show a empty bracket next to 0 secs when you purposefully failed a job.

http://git-wip-us.apache.org/repos/asf/tez/blob/f6ea0fb3/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java b/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java
index 14e773d..a301957 100644
--- a/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java
+++ b/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java
@@ -141,6 +141,20 @@ public class TezConfiguration extends Configuration {
   @ConfigurationScope(Scope.AM)
   public static final String TEZ_CREDENTIALS_PATH = TEZ_PREFIX + "credentials.path";
 
+  @Private
+  @ConfigurationScope(Scope.AM)
+  public static final String TEZ_AM_USE_CONCURRENT_DISPATCHER = TEZ_AM_PREFIX
+      + "use.concurrent-dispatcher";
+  @Private
+  public static boolean TEZ_AM_USE_CONCURRENT_DISPATCHER_DEFAULT = true;
+  
+  @Private
+  @ConfigurationScope(Scope.AM)
+  public static final String TEZ_AM_CONCURRENT_DISPATCHER_CONCURRENCY = TEZ_AM_PREFIX
+      + "concurrent-dispatcher.concurrency";
+  @Private
+  public static final int TEZ_AM_CONCURRENT_DISPATCHER_CONCURRENCY_DEFAULT = 10;
+
   /**
    * Boolean value. Execution mode for the Tez application. True implies session mode. If the client
    * code is written according to best practices then the same code can execute in either mode based

http://git-wip-us.apache.org/repos/asf/tez/blob/f6ea0fb3/tez-common/findbugs-exclude.xml
----------------------------------------------------------------------
diff --git a/tez-common/findbugs-exclude.xml b/tez-common/findbugs-exclude.xml
index 7814585..6f6253d 100644
--- a/tez-common/findbugs-exclude.xml
+++ b/tez-common/findbugs-exclude.xml
@@ -20,4 +20,9 @@
     <Bug pattern="DM_EXIT"/>
   </Match>
 
+  <Match>
+    <Class name="org.apache.tez.common.AsyncDispatcherConcurrent$1"/>
+    <Method name="run" />
+    <Bug pattern="DM_EXIT"/>
+  </Match>
 </FindBugsFilter>

http://git-wip-us.apache.org/repos/asf/tez/blob/f6ea0fb3/tez-common/src/main/java/org/apache/tez/common/AsyncDispatcher.java
----------------------------------------------------------------------
diff --git a/tez-common/src/main/java/org/apache/tez/common/AsyncDispatcher.java b/tez-common/src/main/java/org/apache/tez/common/AsyncDispatcher.java
index 5aaa4cf..4319f4f 100644
--- a/tez-common/src/main/java/org/apache/tez/common/AsyncDispatcher.java
+++ b/tez-common/src/main/java/org/apache/tez/common/AsyncDispatcher.java
@@ -68,8 +68,11 @@ public class AsyncDispatcher extends CompositeService implements Dispatcher {
   private EventHandler handlerInstance = new GenericEventHandler();
 
   private Thread eventHandlingThread;
-  protected final Map<Class<? extends Enum>, EventHandler> eventHandlers;
-  protected final Map<Class<? extends Enum>, AsyncDispatcher> eventDispatchers;
+  protected final Map<Class<? extends Enum>, EventHandler> eventHandlers = Maps.newHashMap();
+  protected final Map<Class<? extends Enum>, AsyncDispatcher> eventDispatchers = Maps.newHashMap();
+  protected final Map<Class<? extends Enum>, AsyncDispatcherConcurrent> concurrentEventDispatchers = 
+      Maps.newHashMap();
+  
   private boolean exitOnDispatchException;
 
   public AsyncDispatcher(String name) {
@@ -77,11 +80,9 @@ public class AsyncDispatcher extends CompositeService implements Dispatcher {
   }
 
   public AsyncDispatcher(String name, BlockingQueue<Event> eventQueue) {
-    super("Dispatcher");
+    super(name);
     this.name = name;
     this.eventQueue = eventQueue;
-    this.eventHandlers = Maps.newHashMap();
-    this.eventDispatchers = Maps.newHashMap();
   }
 
   public Runnable createThread() {
@@ -195,6 +196,32 @@ public class AsyncDispatcher extends CompositeService implements Dispatcher {
       }
     }
   }
+  
+  private void checkForExistingHandler(Class<? extends Enum> eventType) {
+    EventHandler<Event> registeredHandler = (EventHandler<Event>) eventHandlers.get(eventType);
+    Preconditions.checkState(registeredHandler == null, 
+        "Cannot register same event on multiple dispatchers");
+  }
+
+  private void checkForExistingDispatcher(Class<? extends Enum> eventType) {
+    AsyncDispatcher registeredDispatcher = eventDispatchers.get(eventType);
+    Preconditions.checkState(registeredDispatcher == null, 
+        "Multiple dispatchers cannot be registered for: " + eventType.getName());
+  }
+
+  private void checkForExistingConcurrentDispatcher(Class<? extends Enum> eventType) {
+    AsyncDispatcherConcurrent concurrentDispatcher = concurrentEventDispatchers.get(eventType);
+    Preconditions.checkState(concurrentDispatcher == null, 
+        "Multiple concurrent dispatchers cannot be registered for: " + eventType.getName());
+  }
+  
+  private void checkForExistingDispatchers(boolean checkHandler, Class<? extends Enum> eventType) {
+    if (checkHandler) {
+      checkForExistingHandler(eventType);
+    }
+    checkForExistingDispatcher(eventType);
+    checkForExistingConcurrentDispatcher(eventType);
+  }
 
   /**
    * Add an EventHandler for events handled inline on this dispatcher
@@ -205,9 +232,7 @@ public class AsyncDispatcher extends CompositeService implements Dispatcher {
     Preconditions.checkState(getServiceState() == STATE.NOTINITED);
     /* check to see if we have a listener registered */
     EventHandler<Event> registeredHandler = (EventHandler<Event>) eventHandlers.get(eventType);
-    AsyncDispatcher registeredDispatcher = eventDispatchers.get(eventType);
-    Preconditions.checkState(registeredDispatcher == null,
-        "Cannot register same event on multiple dispatchers");
+    checkForExistingDispatchers(false, eventType);
     LOG.info("Registering " + eventType + " for " + handler.getClass());
     if (registeredHandler == null) {
       eventHandlers.put(eventType, handler);
@@ -231,20 +256,41 @@ public class AsyncDispatcher extends CompositeService implements Dispatcher {
   public void registerAndCreateDispatcher(Class<? extends Enum> eventType,
       EventHandler handler, String dispatcherName) {
     Preconditions.checkState(getServiceState() == STATE.NOTINITED);
-    AsyncDispatcher dispatcher = new AsyncDispatcher(dispatcherName);
-    dispatcher.register(eventType, handler);
     
     /* check to see if we have a listener registered */
-    AsyncDispatcher registeredDispatcher = eventDispatchers.get(eventType);
-    EventHandler<Event> registeredHandler = (EventHandler<Event>) eventHandlers.get(eventType);
-    Preconditions.checkState(registeredHandler == null, 
-        "Cannot register same event on multiple dispatchers");
+    checkForExistingDispatchers(true, eventType);
     LOG.info("Registering " + eventType + " for independent dispatch using: " + handler.getClass());
-    Preconditions.checkState(registeredDispatcher == null, 
-        "Multiple dispatchers cannot be registered for: " + eventType.getName());
+    AsyncDispatcher dispatcher = new AsyncDispatcher(dispatcherName);
+    dispatcher.register(eventType, handler);
     eventDispatchers.put(eventType, dispatcher);
     addIfService(dispatcher);
   }
+  
+  public AsyncDispatcherConcurrent registerAndCreateDispatcher(Class<? extends Enum> eventType,
+      EventHandler handler, String dispatcherName, int numThreads) {
+    Preconditions.checkState(getServiceState() == STATE.NOTINITED);
+    
+    /* check to see if we have a listener registered */
+    checkForExistingDispatchers(true, eventType);
+    LOG.info("Registering " + eventType + " for concurrent dispatch using: " + handler.getClass());
+    AsyncDispatcherConcurrent dispatcher = new AsyncDispatcherConcurrent(dispatcherName, numThreads);
+    dispatcher.register(eventType, handler);
+    concurrentEventDispatchers.put(eventType, dispatcher);
+    addIfService(dispatcher);
+    return dispatcher;
+  }
+  
+  public void registerWithExistingDispatcher(Class<? extends Enum> eventType,
+      EventHandler handler, AsyncDispatcherConcurrent dispatcher) {
+    Preconditions.checkState(getServiceState() == STATE.NOTINITED);
+    
+    /* check to see if we have a listener registered */
+    checkForExistingDispatchers(true, eventType);
+    LOG.info("Registering " + eventType + " wit existing concurrent dispatch using: "
+        + handler.getClass());
+    dispatcher.register(eventType, handler);
+    concurrentEventDispatchers.put(eventType, dispatcher);
+  }
 
   @Override
   public EventHandler getEventHandler() {
@@ -261,13 +307,18 @@ public class AsyncDispatcher extends CompositeService implements Dispatcher {
       }
       drained = false;
 
-      // offload to specific dispatcher is one exists
+      // offload to specific dispatcher if one exists
       Class<? extends Enum> type = event.getType().getDeclaringClass();
       AsyncDispatcher registeredDispatcher = eventDispatchers.get(type);
       if (registeredDispatcher != null) {
         registeredDispatcher.getEventHandler().handle(event);
         return;
       }
+      AsyncDispatcherConcurrent concurrentDispatcher = concurrentEventDispatchers.get(type);
+      if (concurrentDispatcher != null) {
+        concurrentDispatcher.getEventHandler().handle(event);
+        return;
+      }
       
       // no registered dispatcher. use internal dispatcher.
       

http://git-wip-us.apache.org/repos/asf/tez/blob/f6ea0fb3/tez-common/src/main/java/org/apache/tez/common/AsyncDispatcherConcurrent.java
----------------------------------------------------------------------
diff --git a/tez-common/src/main/java/org/apache/tez/common/AsyncDispatcherConcurrent.java b/tez-common/src/main/java/org/apache/tez/common/AsyncDispatcherConcurrent.java
new file mode 100644
index 0000000..d19bf9e
--- /dev/null
+++ b/tez-common/src/main/java/org/apache/tez/common/AsyncDispatcherConcurrent.java
@@ -0,0 +1,368 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.LinkedBlockingQueue;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.service.CompositeService;
+import org.apache.hadoop.util.ShutdownHookManager;
+import org.apache.hadoop.yarn.event.Dispatcher;
+import org.apache.hadoop.yarn.event.Event;
+import org.apache.hadoop.yarn.event.EventHandler;
+import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+
+/**
+ * A dispatcher that can schedule events concurrently. Uses a fixed size threadpool 
+ * to schedule events. Events that have the same serializing hash will get scheduled
+ * on the same thread in the threadpool. This can be used to prevent concurrency issues
+ * for events that may not be independently processed.
+ */
+@SuppressWarnings({"rawtypes", "unchecked"})
+@Private
+public class AsyncDispatcherConcurrent extends CompositeService implements Dispatcher {
+
+  private static final Logger LOG = LoggerFactory.getLogger(AsyncDispatcher.class);
+
+  private final String name;
+  private final ArrayList<LinkedBlockingQueue<Event>> eventQueues;
+  private volatile boolean stopped = false;
+
+  // Configuration flag for enabling/disabling draining dispatcher's events on
+  // stop functionality.
+  private volatile boolean drainEventsOnStop = false;
+
+  // Indicates all the remaining dispatcher's events on stop have been drained
+  // and processed.
+  private volatile boolean drained = true;
+  private Object waitForDrained = new Object();
+
+  // For drainEventsOnStop enabled only, block newly coming events into the
+  // queue while stopping.
+  private volatile boolean blockNewEvents = false;
+  private EventHandler handlerInstance = new GenericEventHandler();
+
+  private ExecutorService execService;
+  private final int numThreads;
+  
+  protected final Map<Class<? extends Enum>, EventHandler> eventHandlers = Maps.newHashMap();
+  protected final Map<Class<? extends Enum>, AsyncDispatcherConcurrent> eventDispatchers = 
+      Maps.newHashMap();
+  private boolean exitOnDispatchException;
+
+  AsyncDispatcherConcurrent(String name, int numThreads) {
+    super(name);
+    Preconditions.checkArgument(numThreads > 0);
+    this.name = name;
+    this.eventQueues = Lists.newArrayListWithCapacity(numThreads);
+    this.numThreads = numThreads;
+  }
+  
+  class DispatchRunner implements Runnable {
+    final LinkedBlockingQueue<Event> queue;
+    
+    public DispatchRunner(LinkedBlockingQueue<Event> queue) {
+      this.queue = queue;
+    }
+    
+    @Override
+    public void run() {
+      while (!stopped && !Thread.currentThread().isInterrupted()) {
+        drained = queue.isEmpty();
+        // blockNewEvents is only set when dispatcher is draining to stop,
+        // adding this check is to avoid the overhead of acquiring the lock
+        // and calling notify every time in the normal run of the loop.
+        if (blockNewEvents) {
+          synchronized (waitForDrained) {
+            if (drained) {
+              waitForDrained.notify();
+            }
+          }
+        }
+        Event event;
+        try {
+          event = queue.take();
+        } catch(InterruptedException ie) {
+          if (!stopped) {
+            LOG.warn("AsyncDispatcher thread interrupted", ie);
+          }
+          return;
+        }
+        if (event != null) {
+          dispatch(event);
+        }
+      }
+    }
+  };
+  
+  @Override
+  protected void serviceInit(Configuration conf) throws Exception {
+    // TODO TEZ-2049 remove YARN reference
+    this.exitOnDispatchException =
+        conf.getBoolean(Dispatcher.DISPATCHER_EXIT_ON_ERROR_KEY,
+          Dispatcher.DEFAULT_DISPATCHER_EXIT_ON_ERROR);
+    super.serviceInit(conf);
+  }
+
+  @Override
+  protected void serviceStart() throws Exception {
+    execService = Executors.newFixedThreadPool(numThreads, new ThreadFactoryBuilder().setDaemon(true)
+        .setNameFormat("Dispatcher [" + this.name + "] #%d").build());
+    for (int i=0; i<numThreads; ++i) {
+      eventQueues.add(new LinkedBlockingQueue<Event>());
+    }
+    for (int i=0; i<numThreads; ++i) {
+      execService.execute(new DispatchRunner(eventQueues.get(i)));
+    }
+    //start all the components
+    super.serviceStart();
+  }
+
+  public void setDrainEventsOnStop() {
+    drainEventsOnStop = true;
+  }
+
+  @Override
+  protected void serviceStop() throws Exception {
+    if (execService != null) {
+      if (drainEventsOnStop) {
+        blockNewEvents = true;
+        LOG.info("AsyncDispatcher is draining to stop, ignoring any new events.");
+        synchronized (waitForDrained) {
+          while (!drained && !execService.isShutdown()) {
+            LOG.info("Waiting for AsyncDispatcher to drain.");
+            waitForDrained.wait(1000);
+          }
+        }
+      }
+
+      stopped = true;
+
+      for (int i=0; i<numThreads; ++i) {
+        LOG.info("AsyncDispatcher stopping with events: " + eventQueues.get(i).size()
+            + " in queue: " + i);
+      }
+      execService.shutdownNow();
+    }
+
+    // stop all the components
+    super.serviceStop();
+  }
+
+  protected void dispatch(Event event) {
+    //all events go thru this loop
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Dispatching the event " + event.getClass().getName() + "."
+          + event.toString());
+    }
+
+    Class<? extends Enum> type = event.getType().getDeclaringClass();
+
+    try{
+      EventHandler handler = eventHandlers.get(type);
+      if(handler != null) {
+        handler.handle(event);
+      } else {
+        throw new Exception("No handler for registered for " + type);
+      }
+    } catch (Throwable t) {
+      LOG.error("Error in dispatcher thread", t);
+      // If serviceStop is called, we should exit this thread gracefully.
+      if (exitOnDispatchException
+          && (ShutdownHookManager.get().isShutdownInProgress()) == false
+          && stopped == false) {
+        Thread shutDownThread = new Thread(createShutDownThread());
+        shutDownThread.setName("AsyncDispatcher ShutDown handler");
+        shutDownThread.start();
+      }
+    }
+  }
+
+  private void checkForExistingHandler(Class<? extends Enum> eventType) {
+    EventHandler<Event> registeredHandler = (EventHandler<Event>) eventHandlers.get(eventType);
+    Preconditions.checkState(registeredHandler == null, 
+        "Cannot register same event on multiple dispatchers");
+  }
+
+  private void checkForExistingDispatcher(Class<? extends Enum> eventType) {
+    AsyncDispatcherConcurrent registeredDispatcher = eventDispatchers.get(eventType);
+    Preconditions.checkState(registeredDispatcher == null, 
+        "Multiple dispatchers cannot be registered for: " + eventType.getName());
+  }
+
+  private void checkForExistingDispatchers(boolean checkHandler, Class<? extends Enum> eventType) {
+    if (checkHandler) {
+      checkForExistingHandler(eventType);
+    }
+    checkForExistingDispatcher(eventType);
+  }
+
+  /**
+   * Add an EventHandler for events handled inline on this dispatcher
+   */
+  @Override
+  public void register(Class<? extends Enum> eventType,
+      EventHandler handler) {
+    Preconditions.checkState(getServiceState() == STATE.NOTINITED);
+    /* check to see if we have a listener registered */
+    EventHandler<Event> registeredHandler = (EventHandler<Event>) eventHandlers.get(eventType);
+    checkForExistingDispatchers(false, eventType);
+    LOG.info("Registering " + eventType + " for " + handler.getClass());
+    if (registeredHandler == null) {
+      eventHandlers.put(eventType, handler);
+    } else if (!(registeredHandler instanceof MultiListenerHandler)){
+      /* for multiple listeners of an event add the multiple listener handler */
+      MultiListenerHandler multiHandler = new MultiListenerHandler();
+      multiHandler.addHandler(registeredHandler);
+      multiHandler.addHandler(handler);
+      eventHandlers.put(eventType, multiHandler);
+    } else {
+      /* already a multilistener, just add to it */
+      MultiListenerHandler multiHandler
+      = (MultiListenerHandler) registeredHandler;
+      multiHandler.addHandler(handler);
+    }
+  }
+  
+  /**
+   * Add an EventHandler for events handled in their own dispatchers with given name and threads
+   */
+  
+  public AsyncDispatcherConcurrent registerAndCreateDispatcher(Class<? extends Enum> eventType,
+      EventHandler handler, String dispatcherName, int numThreads) {
+    Preconditions.checkState(getServiceState() == STATE.NOTINITED);
+    
+    /* check to see if we have a listener registered */
+    checkForExistingDispatchers(true, eventType);
+    LOG.info("Registering " + eventType + " for independent dispatch using: " + handler.getClass());
+    AsyncDispatcherConcurrent dispatcher = new AsyncDispatcherConcurrent(dispatcherName, numThreads);
+    dispatcher.register(eventType, handler);
+    eventDispatchers.put(eventType, dispatcher);
+    addIfService(dispatcher);
+    return dispatcher;
+  }
+  
+  public void registerWithExistingDispatcher(Class<? extends Enum> eventType,
+      EventHandler handler, AsyncDispatcherConcurrent dispatcher) {
+    Preconditions.checkState(getServiceState() == STATE.NOTINITED);
+    
+    /* check to see if we have a listener registered */
+    checkForExistingDispatchers(true, eventType);
+    LOG.info("Registering " + eventType + " wit existing concurrent dispatch using: "
+        + handler.getClass());
+    dispatcher.register(eventType, handler);
+    eventDispatchers.put(eventType, dispatcher);
+  }
+
+  @Override
+  public EventHandler getEventHandler() {
+    return handlerInstance;
+  }
+
+  class GenericEventHandler implements EventHandler<TezAbstractEvent> {
+    public void handle(TezAbstractEvent event) {
+      if (stopped) {
+        return;
+      }
+      if (blockNewEvents) {
+        return;
+      }
+      drained = false;
+      
+      // offload to specific dispatcher if one exists
+      Class<? extends Enum> type = event.getType().getDeclaringClass();
+      AsyncDispatcherConcurrent registeredDispatcher = eventDispatchers.get(type);
+      if (registeredDispatcher != null) {
+        registeredDispatcher.getEventHandler().handle(event);
+        return;
+      }
+      
+      int index = numThreads > 1 ? event.getSerializingHash() % numThreads : 0;
+
+     // no registered dispatcher. use internal dispatcher.
+      LinkedBlockingQueue<Event> queue = eventQueues.get(index);
+      /* all this method does is enqueue all the events onto the queue */
+      int qSize = queue.size();
+      if (qSize !=0 && qSize %1000 == 0) {
+        LOG.info("Size of event-queue is " + qSize);
+      }
+      int remCapacity = queue.remainingCapacity();
+      if (remCapacity < 1000) {
+        LOG.warn("Very low remaining capacity in the event-queue: "
+            + remCapacity);
+      }
+      try {
+        queue.put(event);
+      } catch (InterruptedException e) {
+        if (!stopped) {
+          LOG.warn("AsyncDispatcher thread interrupted", e);
+        }
+        throw new YarnRuntimeException(e);
+      }
+    };
+  }
+
+  /**
+   * Multiplexing an event. Sending it to different handlers that
+   * are interested in the event.
+   * @param <T> the type of event these multiple handlers are interested in.
+   */
+  static class MultiListenerHandler implements EventHandler<Event> {
+    List<EventHandler<Event>> listofHandlers;
+
+    public MultiListenerHandler() {
+      listofHandlers = new ArrayList<EventHandler<Event>>();
+    }
+
+    @Override
+    public void handle(Event event) {
+      for (EventHandler<Event> handler: listofHandlers) {
+        handler.handle(event);
+      }
+    }
+
+    void addHandler(EventHandler<Event> handler) {
+      listofHandlers.add(handler);
+    }
+
+  }
+
+  Runnable createShutDownThread() {
+    return new Runnable() {
+      @Override
+      public void run() {
+        LOG.info("Exiting, bbye..");
+        System.exit(-1);
+      }
+    };
+  }
+}

http://git-wip-us.apache.org/repos/asf/tez/blob/f6ea0fb3/tez-common/src/main/java/org/apache/tez/common/TezAbstractEvent.java
----------------------------------------------------------------------
diff --git a/tez-common/src/main/java/org/apache/tez/common/TezAbstractEvent.java b/tez-common/src/main/java/org/apache/tez/common/TezAbstractEvent.java
new file mode 100644
index 0000000..b736112
--- /dev/null
+++ b/tez-common/src/main/java/org/apache/tez/common/TezAbstractEvent.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;
+
+/**
+ * Event that allows running in parallel for different instances
+ * 
+ * @param <TYPE>
+ *          Event type
+ */
+public abstract class TezAbstractEvent<TYPE extends Enum<TYPE>> extends
+    org.apache.hadoop.yarn.event.AbstractEvent<TYPE> {
+
+  public TezAbstractEvent(TYPE type) {
+    super(type);
+  }
+
+  /**
+   * Returning a number that is identical for event instances that need to be
+   * serialized while processing.
+   * 
+   * @return Serializing identifier. Not overriding this causes serialization
+   *         for all events instances
+   */
+  public int getSerializingHash() {
+    return 0;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tez/blob/f6ea0fb3/tez-common/src/main/java/org/apache/tez/dag/records/TezTaskID.java
----------------------------------------------------------------------
diff --git a/tez-common/src/main/java/org/apache/tez/dag/records/TezTaskID.java b/tez-common/src/main/java/org/apache/tez/dag/records/TezTaskID.java
index b4c7b32..3d28348 100644
--- a/tez-common/src/main/java/org/apache/tez/dag/records/TezTaskID.java
+++ b/tez-common/src/main/java/org/apache/tez/dag/records/TezTaskID.java
@@ -44,6 +44,7 @@ import com.google.common.cache.LoadingCache;
 @InterfaceStability.Stable
 public class TezTaskID extends TezID {
   public static final String TASK = "task";
+  private final int serializingHash;
   
   static final ThreadLocal<NumberFormat> tezTaskIdFormat = new ThreadLocal<NumberFormat>() {
     @Override
@@ -67,10 +68,6 @@ public class TezTaskID extends TezID {
   
   private TezVertexID vertexId;
 
-  // Public for Writable serialization. Verify if this is actually required.
-  public TezTaskID() {
-  }
-
   /**
    * Constructs a TezTaskID object from given {@link TezVertexID}.
    * @param vertexID the vertexID object for this TezTaskID
@@ -91,6 +88,11 @@ public class TezTaskID extends TezID {
     super(id);
     Preconditions.checkArgument(vertexID != null, "vertexID cannot be null");
     this.vertexId = vertexID;
+    this.serializingHash = getHashCode(true);
+  }
+  
+  public int getSerializingHash() {
+    return serializingHash;
   }
 
   /** Returns the {@link TezVertexID} object that this task belongs to */
@@ -135,7 +137,15 @@ public class TezTaskID extends TezID {
 
   @Override
   public int hashCode() {
-    return vertexId.hashCode() * 535013 + id;
+    return getHashCode(false);
+  }
+
+  public int getHashCode(boolean makePositive) {
+    int code = vertexId.hashCode() * 535013 + id;
+    if (makePositive) {
+      code = (code < 0 ? -code : code);
+    }
+    return code;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/tez/blob/f6ea0fb3/tez-common/src/test/java/org/apache/tez/common/TestAsyncDispatcher.java
----------------------------------------------------------------------
diff --git a/tez-common/src/test/java/org/apache/tez/common/TestAsyncDispatcher.java b/tez-common/src/test/java/org/apache/tez/common/TestAsyncDispatcher.java
index ad7f5df..bcd1c5f 100644
--- a/tez-common/src/test/java/org/apache/tez/common/TestAsyncDispatcher.java
+++ b/tez-common/src/test/java/org/apache/tez/common/TestAsyncDispatcher.java
@@ -116,7 +116,7 @@ public class TestAsyncDispatcher {
       central.register(TestEventType1.class, new TestEventHandler1());
       Assert.fail();
     } catch (IllegalStateException e) {
-      Assert.assertTrue(e.getMessage().contains("Cannot register same event on multiple dispatchers"));
+      Assert.assertTrue(e.getMessage().contains("Multiple dispatchers cannot be registered for"));
     } finally {
       central.close();
     }

http://git-wip-us.apache.org/repos/asf/tez/blob/f6ea0fb3/tez-common/src/test/java/org/apache/tez/common/TestAsyncDispatcherConcurrent.java
----------------------------------------------------------------------
diff --git a/tez-common/src/test/java/org/apache/tez/common/TestAsyncDispatcherConcurrent.java b/tez-common/src/test/java/org/apache/tez/common/TestAsyncDispatcherConcurrent.java
new file mode 100644
index 0000000..1fa8123
--- /dev/null
+++ b/tez-common/src/test/java/org/apache/tez/common/TestAsyncDispatcherConcurrent.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.common;
+
+import java.util.concurrent.CountDownLatch;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.event.EventHandler;
+import org.junit.Assert;
+import org.junit.Test;
+
+@SuppressWarnings("unchecked")
+public class TestAsyncDispatcherConcurrent {
+
+  static class CountDownEventHandler {
+    static CountDownLatch latch;
+    static void init(CountDownLatch latch) {
+      CountDownEventHandler.latch = latch;
+    }
+
+    static void checkParallelCountersDoneAndFinish() throws Exception {
+      latch.countDown();
+      latch.await();
+    }
+    
+    public void handle() {
+      latch.countDown();
+      try {
+        latch.await();
+      } catch (InterruptedException e) {
+        throw new RuntimeException(e);
+      }
+    }
+  }
+
+  public enum TestEventType1 { TYPE1 }
+  public class TestEvent1 extends TezAbstractEvent<TestEventType1> {
+    final int hash;
+    public TestEvent1(TestEventType1 type, int hash) {
+      super(type);
+      this.hash = hash;
+    }
+    
+    @Override
+    public int getSerializingHash() {
+      return hash;
+    }
+  }
+  class TestEventHandler1 extends CountDownEventHandler implements EventHandler<TestEvent1> {
+    @Override
+    public void handle(TestEvent1 event) {
+      handle();
+    }
+  }
+  public enum TestEventType2 { TYPE2 }
+  public class TestEvent2 extends TezAbstractEvent<TestEventType2> {
+    public TestEvent2(TestEventType2 type) {
+      super(type);
+    }
+  }
+  class TestEventHandler2 extends CountDownEventHandler implements EventHandler<TestEvent2> {
+    @Override
+    public void handle(TestEvent2 event) {
+      handle();
+    }
+  }
+  public enum TestEventType3 { TYPE3 }
+  public class TestEvent3 extends TezAbstractEvent<TestEventType3> {
+    public TestEvent3(TestEventType3 type) {
+      super(type);
+    }
+  }
+  class TestEventHandler3 extends CountDownEventHandler implements EventHandler<TestEvent3> {
+    @Override
+    public void handle(TestEvent3 event) {
+      handle();
+    }
+  }
+
+  @Test (timeout=5000)
+  public void testBasic() throws Exception {
+    CountDownLatch latch = new CountDownLatch(4);
+    CountDownEventHandler.init(latch);
+    
+    AsyncDispatcher central = new AsyncDispatcher("Type1");
+    central.register(TestEventType1.class, new TestEventHandler1());
+    central.registerAndCreateDispatcher(TestEventType2.class, new TestEventHandler2(), "Type2", 1);
+    central.registerAndCreateDispatcher(TestEventType3.class, new TestEventHandler3(), "Type3", 1);
+    
+    central.init(new Configuration());
+    central.start();
+    // 3 threads in different dispatchers will handle 3 events
+    central.getEventHandler().handle(new TestEvent1(TestEventType1.TYPE1, 0));
+    central.getEventHandler().handle(new TestEvent2(TestEventType2.TYPE2));
+    central.getEventHandler().handle(new TestEvent3(TestEventType3.TYPE3));
+    // wait for all events to be run in parallel
+    CountDownEventHandler.checkParallelCountersDoneAndFinish();
+    central.close();
+  }
+  
+  @Test (timeout=5000)
+  public void testMultiThreads() throws Exception {
+    CountDownLatch latch = new CountDownLatch(4);
+    CountDownEventHandler.init(latch);
+    
+    AsyncDispatcherConcurrent central = new AsyncDispatcherConcurrent("Type1", 1);
+    central.registerAndCreateDispatcher(TestEventType1.class, new TestEventHandler1(), "Type1", 3);
+    
+    central.init(new Configuration());
+    central.start();
+    // 3 threads in the same dispatcher will handle 3 events
+    central.getEventHandler().handle(new TestEvent1(TestEventType1.TYPE1, 0));
+    central.getEventHandler().handle(new TestEvent1(TestEventType1.TYPE1, 1));
+    central.getEventHandler().handle(new TestEvent1(TestEventType1.TYPE1, 2));
+    // wait for all events to be run in parallel
+    CountDownEventHandler.checkParallelCountersDoneAndFinish();
+    central.close();
+  }
+  
+  @Test (timeout=5000)
+  public void testMultipleRegisterFail() throws Exception {
+    AsyncDispatcher central = new AsyncDispatcher("Type1");
+    try {
+      central.register(TestEventType1.class, new TestEventHandler1());
+      central.registerAndCreateDispatcher(TestEventType1.class, new TestEventHandler2(), "Type2", 1);
+      Assert.fail();
+    } catch (IllegalStateException e) {
+      Assert.assertTrue(e.getMessage().contains("Cannot register same event on multiple dispatchers"));
+    } finally {
+      central.close();
+    }
+    
+    central = new AsyncDispatcher("Type1");
+    try {
+      central.registerAndCreateDispatcher(TestEventType1.class, new TestEventHandler2(), "Type2", 1);
+      central.register(TestEventType1.class, new TestEventHandler1());
+      Assert.fail();
+    } catch (IllegalStateException e) {
+      Assert.assertTrue(e.getMessage().contains("Multiple concurrent dispatchers cannot be registered"));
+    } finally {
+      central.close();
+    }
+    
+    central = new AsyncDispatcher("Type1");
+    try {
+      central.registerAndCreateDispatcher(TestEventType1.class, new TestEventHandler2(), "Type2", 1);
+      central.registerAndCreateDispatcher(TestEventType1.class, new TestEventHandler2(), "Type2", 1);
+      Assert.fail();
+    } catch (IllegalStateException e) {
+      Assert.assertTrue(e.getMessage().contains("Multiple concurrent dispatchers cannot be registered"));
+    } finally {
+      central.close();
+    }
+    
+    central = new AsyncDispatcher("Type1");
+    try {
+      central.registerAndCreateDispatcher(TestEventType1.class, new TestEventHandler2(), "Type2");
+      central.registerAndCreateDispatcher(TestEventType1.class, new TestEventHandler2(), "Type2");
+      Assert.fail();
+    } catch (IllegalStateException e) {
+      Assert.assertTrue(e.getMessage().contains("Multiple dispatchers cannot be registered for"));
+    } finally {
+      central.close();
+    }
+    
+    central = new AsyncDispatcher("Type1");
+    try {
+      AsyncDispatcherConcurrent concDispatcher = central.registerAndCreateDispatcher(
+          TestEventType1.class, new TestEventHandler2(), "Type2", 1);
+      central.registerWithExistingDispatcher(TestEventType1.class, new TestEventHandler1(),
+          concDispatcher);
+      Assert.fail();
+    } catch (IllegalStateException e) {
+      Assert.assertTrue(e.getMessage().contains("Multiple concurrent dispatchers cannot be registered"));
+    } finally {
+      central.close();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/tez/blob/f6ea0fb3/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 27b9c37..3e3d6f0 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
@@ -91,6 +91,7 @@ import org.apache.hadoop.yarn.util.ConverterUtils;
 import org.apache.hadoop.yarn.util.ResourceCalculatorProcessTree;
 import org.apache.hadoop.yarn.util.SystemClock;
 import org.apache.tez.common.AsyncDispatcher;
+import org.apache.tez.common.AsyncDispatcherConcurrent;
 import org.apache.tez.common.GcTimeUpdater;
 import org.apache.tez.common.TezCommonUtils;
 import org.apache.tez.common.TezConverterUtils;
@@ -455,12 +456,22 @@ public class DAGAppMaster extends AbstractService {
     dispatcher.register(DAGAppMasterEventType.class, new DAGAppMasterEventHandler());
     dispatcher.register(DAGEventType.class, dagEventDispatcher);
     dispatcher.register(VertexEventType.class, vertexEventDispatcher);
-    dispatcher.register(TaskEventType.class, new TaskEventDispatcher());
-    dispatcher.register(TaskAttemptEventType.class, new TaskAttemptEventDispatcher());
+    if (!conf.getBoolean(TezConfiguration.TEZ_AM_USE_CONCURRENT_DISPATCHER,
+        TezConfiguration.TEZ_AM_USE_CONCURRENT_DISPATCHER_DEFAULT)) {
+      dispatcher.register(TaskEventType.class, new TaskEventDispatcher());
+      dispatcher.register(TaskAttemptEventType.class, new TaskAttemptEventDispatcher());
+    } else {
+      int concurrency = conf.getInt(TezConfiguration.TEZ_AM_CONCURRENT_DISPATCHER_CONCURRENCY, 
+          TezConfiguration.TEZ_AM_CONCURRENT_DISPATCHER_CONCURRENCY_DEFAULT);
+      AsyncDispatcherConcurrent sharedDispatcher = dispatcher.registerAndCreateDispatcher(
+          TaskEventType.class, new TaskEventDispatcher(), "TaskAndAttemptEventThread", concurrency);
+      dispatcher.registerWithExistingDispatcher(TaskAttemptEventType.class,
+          new TaskAttemptEventDispatcher(), sharedDispatcher);
+    }
     
     // register other delegating dispatchers
-    dispatcher.registerAndCreateDispatcher(SpeculatorEventType.class, new SpeculatorEventHandler(), "Speculator");
-
+    dispatcher.registerAndCreateDispatcher(SpeculatorEventType.class, new SpeculatorEventHandler(),
+        "Speculator");
 
     if (enableWebUIService()) {
       this.webUIService = new WebUIService(context);

http://git-wip-us.apache.org/repos/asf/tez/blob/f6ea0fb3/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 3f60a4e..6c85cc2 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
@@ -128,8 +128,6 @@ public interface TaskAttempt {
    */
   long getFinishTime();
   
-  public Task getTask();
-  
   TaskAttemptState restoreFromEvent(HistoryEvent event);
 
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/f6ea0fb3/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/CallableEvent.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/CallableEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/CallableEvent.java
index e148fe8..7e68752 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/CallableEvent.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/CallableEvent.java
@@ -20,11 +20,11 @@ package org.apache.tez.dag.app.dag.event;
 
 import java.util.concurrent.Callable;
 
-import org.apache.hadoop.yarn.event.AbstractEvent;
+import org.apache.tez.common.TezAbstractEvent;
 
 import com.google.common.util.concurrent.FutureCallback;
 
-public abstract class CallableEvent extends AbstractEvent<CallableEventType> implements
+public abstract class CallableEvent extends TezAbstractEvent<CallableEventType> implements
     Callable<Void> {
   private final FutureCallback<Void> callback;
 

http://git-wip-us.apache.org/repos/asf/tez/blob/f6ea0fb3/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/DAGAppMasterEvent.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/DAGAppMasterEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/DAGAppMasterEvent.java
index 0571cab..b7cb3a4 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/DAGAppMasterEvent.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/DAGAppMasterEvent.java
@@ -18,9 +18,10 @@
 
 package org.apache.tez.dag.app.dag.event;
 
-import org.apache.hadoop.yarn.event.AbstractEvent;
+import org.apache.tez.common.TezAbstractEvent;
 
-public class DAGAppMasterEvent extends AbstractEvent<DAGAppMasterEventType> {
+
+public class DAGAppMasterEvent extends TezAbstractEvent<DAGAppMasterEventType> {
 
   public DAGAppMasterEvent(DAGAppMasterEventType type) {
     super(type);

http://git-wip-us.apache.org/repos/asf/tez/blob/f6ea0fb3/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/DAGEvent.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/DAGEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/DAGEvent.java
index 1ec0222..a0a8a1a 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/DAGEvent.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/DAGEvent.java
@@ -18,14 +18,14 @@
 
 package org.apache.tez.dag.app.dag.event;
 
-import org.apache.hadoop.yarn.event.AbstractEvent;
+import org.apache.tez.common.TezAbstractEvent;
 import org.apache.tez.dag.records.TezDAGID;
 
 /**
  * This class encapsulates job related events.
  *
  */
-public class DAGEvent extends AbstractEvent<DAGEventType> {
+public class DAGEvent extends TezAbstractEvent<DAGEventType> {
 
   private TezDAGID dagId;
 

http://git-wip-us.apache.org/repos/asf/tez/blob/f6ea0fb3/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/SpeculatorEvent.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/SpeculatorEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/SpeculatorEvent.java
index 16fab8e..3863a2a 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/SpeculatorEvent.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/SpeculatorEvent.java
@@ -18,10 +18,10 @@
 
 package org.apache.tez.dag.app.dag.event;
 
-import org.apache.hadoop.yarn.event.AbstractEvent;
+import org.apache.tez.common.TezAbstractEvent;
 import org.apache.tez.dag.records.TezVertexID;
 
-public class SpeculatorEvent extends AbstractEvent<SpeculatorEventType> {
+public class SpeculatorEvent extends TezAbstractEvent<SpeculatorEventType> {
   private final TezVertexID vertexId;
   
   public SpeculatorEvent(SpeculatorEventType type, TezVertexID vertexId) {

http://git-wip-us.apache.org/repos/asf/tez/blob/f6ea0fb3/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskAttemptEvent.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskAttemptEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskAttemptEvent.java
index 56c03e3..63ef70f 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskAttemptEvent.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskAttemptEvent.java
@@ -18,14 +18,14 @@
 
 package org.apache.tez.dag.app.dag.event;
 
-import org.apache.hadoop.yarn.event.AbstractEvent;
+import org.apache.tez.common.TezAbstractEvent;
 import org.apache.tez.dag.records.TezTaskAttemptID;
 
 /**
  * This class encapsulates task attempt related events.
  *
  */
-public class TaskAttemptEvent extends AbstractEvent<TaskAttemptEventType> {
+public class TaskAttemptEvent extends TezAbstractEvent<TaskAttemptEventType> {
 
   private TezTaskAttemptID attemptID;
   
@@ -42,4 +42,9 @@ public class TaskAttemptEvent extends AbstractEvent<TaskAttemptEventType> {
   public TezTaskAttemptID getTaskAttemptID() {
     return attemptID;
   }
+  
+  @Override
+  public int getSerializingHash() {
+    return attemptID.getTaskID().getSerializingHash();
+  }
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/f6ea0fb3/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskEvent.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskEvent.java
index c7e5faa..def9ddf 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskEvent.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskEvent.java
@@ -18,14 +18,14 @@
 
 package org.apache.tez.dag.app.dag.event;
 
-import org.apache.hadoop.yarn.event.AbstractEvent;
+import org.apache.tez.common.TezAbstractEvent;
 import org.apache.tez.dag.records.TezTaskID;
 
 /**
  * this class encapsulates task related events.
  *
  */
-public class TaskEvent extends AbstractEvent<TaskEventType> {
+public class TaskEvent extends TezAbstractEvent<TaskEventType> {
 
   private TezTaskID taskId;
 
@@ -37,4 +37,9 @@ public class TaskEvent extends AbstractEvent<TaskEventType> {
   public TezTaskID getTaskID() {
     return taskId;
   }
+  
+  @Override
+  public int getSerializingHash() {
+    return taskId.getSerializingHash();
+  }
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/f6ea0fb3/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexEvent.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexEvent.java
index 9e94eb5..33128e4 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexEvent.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexEvent.java
@@ -18,14 +18,14 @@
 
 package org.apache.tez.dag.app.dag.event;
 
-import org.apache.hadoop.yarn.event.AbstractEvent;
+import org.apache.tez.common.TezAbstractEvent;
 import org.apache.tez.dag.records.TezVertexID;
 
 /**
  * this class encapsulates vertex related events.
  *
  */
-public class VertexEvent extends AbstractEvent<VertexEventType> {
+public class VertexEvent extends TezAbstractEvent<VertexEventType> {
 
   private TezVertexID vertexId;
 

http://git-wip-us.apache.org/repos/asf/tez/blob/f6ea0fb3/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 f562451..f769565 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
@@ -1261,7 +1261,6 @@ public class DAGImpl implements org.apache.tez.dag.app.dag.DAG,
     if (finishTime == 0) {
       setFinishTime();
     }
-    
     entityUpdateTracker.stop();
 
     boolean recoveryError = false;

http://git-wip-us.apache.org/repos/asf/tez/blob/f6ea0fb3/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 1f3e1cf..b1c0acc 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
@@ -144,6 +144,9 @@ public class TaskAttemptImpl implements TaskAttempt,
   private NodeId containerNodeId;
   private String nodeHttpAddress;
   private String nodeRackName;
+  
+  private final Task task;
+  private final Vertex vertex;
 
   @VisibleForTesting
   TaskAttemptStatus reportedStatus;
@@ -406,7 +409,8 @@ public class TaskAttemptImpl implements TaskAttempt,
       TaskAttemptListener taskAttemptListener, Configuration conf, Clock clock,
       TaskHeartbeatHandler taskHeartbeatHandler, AppContext appContext,
       boolean isRescheduled,
-      Resource resource, ContainerContext containerContext, boolean leafVertex) {
+      Resource resource, ContainerContext containerContext, boolean leafVertex,
+      Task task) {
     ReentrantReadWriteLock rwLock = new ReentrantReadWriteLock();
     this.readLock = rwLock.readLock();
     this.writeLock = rwLock.writeLock();
@@ -417,6 +421,9 @@ public class TaskAttemptImpl implements TaskAttempt,
     this.clock = clock;
     this.taskHeartbeatHandler = taskHeartbeatHandler;
     this.appContext = appContext;
+    this.task = task;
+    this.vertex = this.task.getVertex();
+
     this.reportedStatus = new TaskAttemptStatus(this.attemptId);
     initTaskAttemptStatus(reportedStatus);
     RackResolver.init(conf);
@@ -649,17 +656,9 @@ public class TaskAttemptImpl implements TaskAttempt,
       readLock.unlock();
     }
   }
-
-  @Override
-  public Task getTask() {
-    return appContext.getCurrentDAG()
-        .getVertex(attemptId.getTaskID().getVertexID())
-        .getTask(attemptId.getTaskID());
-  }
-
+  
   Vertex getVertex() {
-    return appContext.getCurrentDAG()
-        .getVertex(attemptId.getTaskID().getVertexID());
+    return vertex;
   }
 
   @SuppressWarnings("unchecked")
@@ -955,7 +954,7 @@ public class TaskAttemptImpl implements TaskAttempt,
     if (conf.getBoolean(YarnConfiguration.LOG_AGGREGATION_ENABLED,
         YarnConfiguration.DEFAULT_LOG_AGGREGATION_ENABLED)
         && conf.get(YarnConfiguration.YARN_LOG_SERVER_URL) != null) {
-      String contextStr = "v_" + getTask().getVertex().getName()
+      String contextStr = "v_" + getVertex().getName()
           + "_" + this.attemptId.toString();
       completedLogsUrl = conf.get(YarnConfiguration.YARN_LOG_SERVER_URL)
           + "/" + containerNodeId.toString()
@@ -964,7 +963,7 @@ public class TaskAttemptImpl implements TaskAttempt,
           + "/" + this.appContext.getUser();
     }
     TaskAttemptStartedEvent startEvt = new TaskAttemptStartedEvent(
-        attemptId, getTask().getVertex().getName(),
+        attemptId, getVertex().getName(),
         launchTime, containerId, containerNodeId,
         inProgressLogsUrl, completedLogsUrl, nodeHttpAddress);
     this.appContext.getHistoryHandler().handle(
@@ -976,7 +975,7 @@ public class TaskAttemptImpl implements TaskAttempt,
     if (getLaunchTime() == 0) return;
 
     TaskAttemptFinishedEvent finishEvt = new TaskAttemptFinishedEvent(
-        attemptId, getTask().getVertex().getName(), getLaunchTime(),
+        attemptId, getVertex().getName(), getLaunchTime(),
         getFinishTime(), TaskAttemptState.SUCCEEDED, null,
         "", getCounters());
     // FIXME how do we store information regd completion events
@@ -987,7 +986,7 @@ public class TaskAttemptImpl implements TaskAttempt,
   protected void logJobHistoryAttemptUnsuccesfulCompletion(
       TaskAttemptState state) {
     TaskAttemptFinishedEvent finishEvt = new TaskAttemptFinishedEvent(
-        attemptId, getTask().getVertex().getName(), getLaunchTime(),
+        attemptId, getVertex().getName(), getLaunchTime(),
         clock.getTime(), state,
         terminationCause,
         StringUtils.join(

http://git-wip-us.apache.org/repos/asf/tez/blob/f6ea0fb3/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 461339b..8b63734 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
@@ -119,6 +119,7 @@ public class TaskImpl implements Task, EventHandler<TaskEvent> {
   private Map<TezTaskAttemptID, TaskAttempt> attempts;
   private final int maxFailedAttempts;
   protected final Clock clock;
+  private final Vertex vertex;
   private final Lock readLock;
   private final Lock writeLock;
   private final List<String> diagnostics = new ArrayList<String>();
@@ -326,7 +327,8 @@ public class TaskImpl implements Task, EventHandler<TaskEvent> {
       Clock clock, TaskHeartbeatHandler thh, AppContext appContext,
       boolean leafVertex, Resource resource,
       ContainerContext containerContext,
-      StateChangeNotifier stateChangeNotifier) {
+      StateChangeNotifier stateChangeNotifier,
+      Vertex vertex) {
     this.conf = conf;
     this.clock = clock;
     ReadWriteLock readWriteLock = new ReentrantReadWriteLock();
@@ -342,7 +344,7 @@ public class TaskImpl implements Task, EventHandler<TaskEvent> {
     this.eventHandler = eventHandler;
     this.appContext = appContext;
     this.stateChangeNotifier = stateChangeNotifier;
-
+    this.vertex = vertex;
     this.leafVertex = leafVertex;
     this.taskResource = resource;
     this.containerContext = containerContext;
@@ -382,7 +384,7 @@ public class TaskImpl implements Task, EventHandler<TaskEvent> {
 
   @Override
   public Vertex getVertex() {
-    return appContext.getCurrentDAG().getVertex(taskId.getVertexID());
+    return vertex;
   }
 
   @Override
@@ -778,7 +780,7 @@ public class TaskImpl implements Task, EventHandler<TaskEvent> {
   TaskAttemptImpl createAttempt(int attemptNumber) {
     return new TaskAttemptImpl(getTaskId(), attemptNumber, eventHandler,
         taskAttemptListener, conf, clock, taskHeartbeatHandler, appContext,
-        (failedAttempts > 0), taskResource, containerContext, leafVertex);
+        (failedAttempts > 0), taskResource, containerContext, leafVertex, this);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/tez/blob/f6ea0fb3/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 c5de19b..9ed7441 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
@@ -109,9 +109,7 @@ import org.apache.tez.dag.app.dag.event.DAGEventType;
 import org.apache.tez.dag.app.dag.event.DAGEventVertexCompleted;
 import org.apache.tez.dag.app.dag.event.DAGEventVertexReRunning;
 import org.apache.tez.dag.app.dag.event.SpeculatorEvent;
-import org.apache.tez.dag.app.dag.event.TaskAttemptEvent;
 import org.apache.tez.dag.app.dag.event.TaskAttemptEventAttemptFailed;
-import org.apache.tez.dag.app.dag.event.TaskAttemptEventStatusUpdate;
 import org.apache.tez.dag.app.dag.event.TaskAttemptEventType;
 import org.apache.tez.dag.app.dag.event.TaskEvent;
 import org.apache.tez.dag.app.dag.event.TaskEventRecoverTask;
@@ -164,7 +162,6 @@ import org.apache.tez.runtime.api.events.InputFailedEvent;
 import org.apache.tez.runtime.api.events.InputDataInformationEvent;
 import org.apache.tez.runtime.api.events.InputInitializerEvent;
 import org.apache.tez.runtime.api.events.TaskAttemptFailedEvent;
-import org.apache.tez.runtime.api.events.TaskStatusUpdateEvent;
 import org.apache.tez.runtime.api.events.VertexManagerEvent;
 import org.apache.tez.runtime.api.impl.EventMetaData;
 import org.apache.tez.runtime.api.impl.EventType;
@@ -194,8 +191,7 @@ import org.slf4j.LoggerFactory;
  * The read and write calls use ReadWriteLock for concurrency.
  */
 @SuppressWarnings({ "rawtypes", "unchecked" })
-public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
-  EventHandler<VertexEvent> {
+public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex, EventHandler<VertexEvent> {
 
   private static final String LINE_SEPARATOR = System
       .getProperty("line.separator");
@@ -216,6 +212,7 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
   // TODO Metrics
   //private final MRAppMetrics metrics;
   private final AppContext appContext;
+  private final DAG dag;
 
   private boolean lazyTasksCopyNeeded = false;
   // must be a linked map for ordering
@@ -867,6 +864,7 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
     setTaskLocationHints(vertexLocationHint);
 
     this.dagUgi = appContext.getCurrentDAG().getDagUGI();
+    this.dag = appContext.getCurrentDAG();
 
     this.taskResource = DagTypeConverters
         .createResourceRequestFromTaskConfig(vertexPlan.getTaskConfig());
@@ -2154,7 +2152,8 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
           this.targetVertices.isEmpty() : true),
         this.taskResource,
         conContext,
-        this.stateChangeNotifier);
+        this.stateChangeNotifier,
+        this);
   }
   
   private void createTasks() {
@@ -4409,7 +4408,7 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
 
   @Override
   public DAG getDAG() {
-    return appContext.getCurrentDAG();
+    return dag;
   }
 
   private TezDAGID getDAGId() {

http://git-wip-us.apache.org/repos/asf/tez/blob/f6ea0fb3/tez-dag/src/test/java/org/apache/tez/dag/app/TestMockDAGAppMaster.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/TestMockDAGAppMaster.java b/tez-dag/src/test/java/org/apache/tez/dag/app/TestMockDAGAppMaster.java
index 2a061bc..87ffead 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/TestMockDAGAppMaster.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/TestMockDAGAppMaster.java
@@ -136,7 +136,7 @@ public class TestMockDAGAppMaster {
     lrVertex.put(lrName2, LocalResource.newInstance(URL.newInstance("file", "localhost", 0, "/test1"),
         LocalResourceType.FILE, LocalResourceVisibility.PUBLIC, 1, 1));
 
-    DAG dag = DAG.create("test").addTaskLocalFiles(lrDAG);
+    DAG dag = DAG.create("testLocalResourceSetup").addTaskLocalFiles(lrDAG);
     Vertex vA = Vertex.create("A", ProcessorDescriptor.create("Proc.class"), 5).addTaskLocalFiles(lrVertex);
     dag.addVertex(vA);
 
@@ -166,7 +166,7 @@ public class TestMockDAGAppMaster {
     MockContainerLauncher mockLauncher = mockApp.getContainerLauncher();
     mockLauncher.startScheduling(false);
     // there is only 1 task whose first attempt will be preempted
-    DAG dag = DAG.create("test");
+    DAG dag = DAG.create("testInternalPreemption");
     Vertex vA = Vertex.create("A", ProcessorDescriptor.create("Proc.class"), 1);
     dag.addVertex(vA);
 
@@ -197,7 +197,7 @@ public class TestMockDAGAppMaster {
     MockContainerLauncher mockLauncher = mockApp.getContainerLauncher();
     mockLauncher.startScheduling(false);
     mockApp.sendDMEvents = true;
-    DAG dag = DAG.create("test");
+    DAG dag = DAG.create("testBasicEvents");
     Vertex vA = Vertex.create("A", ProcessorDescriptor.create("Proc.class"), 2);
     Vertex vB = Vertex.create("B", ProcessorDescriptor.create("Proc.class"), 2);
     Vertex vC = Vertex.create("C", ProcessorDescriptor.create("Proc.class"), 2);
@@ -230,21 +230,27 @@ public class TestMockDAGAppMaster {
     List<TezEvent> tEvents = tImpl.getTaskEvents();
     Assert.assertEquals(2, tEvents.size()); // 2 from vA
     Assert.assertEquals(vA.getName(), tEvents.get(0).getDestinationInfo().getEdgeVertexName());
-    Assert.assertEquals(0, ((DataMovementEvent)tEvents.get(0).getEvent()).getTargetIndex());
     Assert.assertEquals(0, ((DataMovementEvent)tEvents.get(0).getEvent()).getSourceIndex());
     Assert.assertEquals(vA.getName(), tEvents.get(1).getDestinationInfo().getEdgeVertexName());
-    Assert.assertEquals(1, ((DataMovementEvent)tEvents.get(1).getEvent()).getTargetIndex());
     Assert.assertEquals(0, ((DataMovementEvent)tEvents.get(1).getEvent()).getSourceIndex());
+    int targetIndex1 = ((DataMovementEvent)tEvents.get(0).getEvent()).getTargetIndex();
+    int targetIndex2 = ((DataMovementEvent)tEvents.get(1).getEvent()).getTargetIndex();
+    // order of vA task completion can change order of events
+    Assert.assertTrue("t1: " + targetIndex1 + " t2: " + targetIndex2,
+        (targetIndex1 == 0 && targetIndex2 == 1) || (targetIndex1 == 1 && targetIndex2 == 0));
     vImpl = (VertexImpl) dagImpl.getVertex(vC.getName());
     tImpl = (TaskImpl) vImpl.getTask(1);
     tEvents = tImpl.getTaskEvents();
     Assert.assertEquals(2, tEvents.size()); // 2 from vA
     Assert.assertEquals(vA.getName(), tEvents.get(0).getDestinationInfo().getEdgeVertexName());
-    Assert.assertEquals(0, ((DataMovementEvent)tEvents.get(0).getEvent()).getTargetIndex());
     Assert.assertEquals(1, ((DataMovementEvent)tEvents.get(0).getEvent()).getSourceIndex());
     Assert.assertEquals(vA.getName(), tEvents.get(1).getDestinationInfo().getEdgeVertexName());
-    Assert.assertEquals(1, ((DataMovementEvent)tEvents.get(1).getEvent()).getTargetIndex());
     Assert.assertEquals(1, ((DataMovementEvent)tEvents.get(1).getEvent()).getSourceIndex());
+    targetIndex1 = ((DataMovementEvent)tEvents.get(0).getEvent()).getTargetIndex();
+    targetIndex2 = ((DataMovementEvent)tEvents.get(1).getEvent()).getTargetIndex();
+    // order of vA task completion can change order of events
+    Assert.assertTrue("t1: " + targetIndex1 + " t2: " + targetIndex2,
+        (targetIndex1 == 0 && targetIndex2 == 1) || (targetIndex1 == 1 && targetIndex2 == 0));
     vImpl = (VertexImpl) dagImpl.getVertex(vD.getName());
     tImpl = (TaskImpl) vImpl.getTask(1);
     tEvents = tImpl.getTaskEvents();
@@ -478,7 +484,7 @@ public class TestMockDAGAppMaster {
 
     final String vAName = "A";
     
-    DAG dag = DAG.create("testBasicCounters");
+    DAG dag = DAG.create("testBasicCounterMemory");
     Vertex vA = Vertex.create(vAName, ProcessorDescriptor.create("Proc.class"), 10000);
     dag.addVertex(vA);
 
@@ -511,6 +517,30 @@ public class TestMockDAGAppMaster {
     checkMemory(dag.getName(), mockApp);
     tezClient.stop();
   }
+  
+  @Ignore
+  @Test (timeout = 60000)
+  public void testTaskEventsProcessingSpeed() throws Exception {
+    Logger.getRootLogger().setLevel(Level.WARN);
+    TezConfiguration tezconf = new TezConfiguration(defaultConf);
+    tezconf.setBoolean(TezConfiguration.TEZ_AM_USE_CONCURRENT_DISPATCHER, true);
+    MockTezClient tezClient = new MockTezClient("testMockAM", tezconf, true, null, null, null,
+        null, false, false, 30, 1000);
+    tezClient.start();
+
+    final String vAName = "A";
+    
+    DAG dag = DAG.create("testTaskEventsProcessingSpeed");
+    Vertex vA = Vertex.create(vAName, ProcessorDescriptor.create("Proc.class"), 50000);
+    dag.addVertex(vA);
+
+    MockDAGAppMaster mockApp = tezClient.getLocalClient().getMockApp();
+    mockApp.doSleep = false;
+    DAGClient dagClient = tezClient.submitDAG(dag);
+    DAGStatus status = dagClient.waitForCompletion();
+    Assert.assertEquals(DAGStatus.State.SUCCEEDED, status.getState());
+    tezClient.stop();
+  }
 
   @Ignore
   @Test (timeout = 60000)
@@ -530,7 +560,7 @@ public class TestMockDAGAppMaster {
     ioStats.setItemsProcessed(1);
     TaskStatistics vAStats = new TaskStatistics();
 
-    DAG dag = DAG.create("testBasisStatistics");
+    DAG dag = DAG.create("testBasicStatisticsMemory");
     Vertex vA = Vertex.create(vAName, ProcessorDescriptor.create("Proc.class"), numTasks);
     for (int i=0; i<numSources; ++i) {
       final String sourceName = i + vAName;
@@ -623,7 +653,7 @@ public class TestMockDAGAppMaster {
     MockContainerLauncher mockLauncher = mockApp.getContainerLauncher();
     mockLauncher.startScheduling(false);
 
-    DAG dag = DAG.create("test");
+    DAG dag = DAG.create("testSchedulerErrorHandling");
     Vertex vA = Vertex.create("A", ProcessorDescriptor.create("Proc.class"), 5);
     dag.addVertex(vA);
 

http://git-wip-us.apache.org/repos/asf/tez/blob/f6ea0fb3/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 2a2df7c..50bb68c 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
@@ -81,6 +81,7 @@ 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;
 import org.apache.tez.dag.app.dag.event.TaskAttemptEventType;
+import org.apache.tez.dag.app.dag.event.TaskEvent;
 import org.apache.tez.dag.app.dag.event.TaskEventTAUpdate;
 import org.apache.tez.dag.app.dag.event.TaskEventType;
 import org.apache.tez.dag.app.dag.event.SpeculatorEventTaskAttemptStatusUpdate;
@@ -672,6 +673,35 @@ public class TestTaskAttempt {
   }
   
   @Test(timeout = 5000)
+  public void testEventSerializingHash() throws Exception {
+    ApplicationId appId = ApplicationId.newInstance(1, 2);
+    TezDAGID dagID = TezDAGID.getInstance(appId, 1);
+    TezVertexID vertexID = TezVertexID.getInstance(dagID, 1);
+    TezTaskID taskID1 = TezTaskID.getInstance(vertexID, 1);
+    TezTaskID taskID2 = TezTaskID.getInstance(vertexID, 2);
+    TezTaskAttemptID taID11 = TezTaskAttemptID.getInstance(taskID1, 0);
+    TezTaskAttemptID taID12 = TezTaskAttemptID.getInstance(taskID1, 1);
+    TezTaskAttemptID taID21 = TezTaskAttemptID.getInstance(taskID2, 1);
+    
+    TaskAttemptEvent taEventFail11 = new TaskAttemptEvent(taID11, TaskAttemptEventType.TA_FAILED);
+    TaskAttemptEvent taEventKill11 = new TaskAttemptEvent(taID11, TaskAttemptEventType.TA_KILL_REQUEST);
+    TaskAttemptEvent taEventKill12 = new TaskAttemptEvent(taID12, TaskAttemptEventType.TA_KILL_REQUEST);
+    TaskAttemptEvent taEventKill21 = new TaskAttemptEvent(taID21, TaskAttemptEventType.TA_KILL_REQUEST);
+    TaskEvent tEventKill1 = new TaskEvent(taskID1, TaskEventType.T_ATTEMPT_KILLED);
+    TaskEvent tEventFail1 = new TaskEvent(taskID1, TaskEventType.T_ATTEMPT_FAILED);
+    TaskEvent tEventFail2 = new TaskEvent(taskID2, TaskEventType.T_ATTEMPT_FAILED);
+    
+    // all of them should have the same value
+    assertEquals(taEventFail11.getSerializingHash(), taEventKill11.getSerializingHash());
+    assertEquals(taEventKill11.getSerializingHash(), taEventKill12.getSerializingHash());
+    assertEquals(tEventFail1.getSerializingHash(), tEventKill1.getSerializingHash());
+    assertEquals(taEventFail11.getSerializingHash(), tEventKill1.getSerializingHash());
+    assertEquals(taEventKill21.getSerializingHash(), tEventFail2.getSerializingHash());
+    // events from different tasks may not have the same value
+    assertFalse(tEventFail1.getSerializingHash() == tEventFail2.getSerializingHash());
+  }
+  
+  @Test(timeout = 5000)
   public void testSuccess() throws Exception {
     ApplicationId appId = ApplicationId.newInstance(1, 2);
     ApplicationAttemptId appAttemptId = ApplicationAttemptId.newInstance(
@@ -695,7 +725,7 @@ public class TestTaskAttempt {
     Resource resource = Resource.newInstance(1024, 1);
 
     NodeId nid = NodeId.newInstance("127.0.0.1", 0);
-    ContainerId contId = ContainerId.newInstance(appAttemptId, 3);
+    ContainerId contId = ContainerId.newContainerId(appAttemptId, 3);
     Container container = mock(Container.class);
     when(container.getId()).thenReturn(contId);
     when(container.getNodeId()).thenReturn(nid);
@@ -786,7 +816,7 @@ public class TestTaskAttempt {
     Resource resource = Resource.newInstance(1024, 1);
 
     NodeId nid = NodeId.newInstance("127.0.0.1", 0);
-    ContainerId contId = ContainerId.newInstance(appAttemptId, 3);
+    ContainerId contId = ContainerId.newContainerId(appAttemptId, 3);
     Container container = mock(Container.class);
     when(container.getId()).thenReturn(contId);
     when(container.getNodeId()).thenReturn(nid);
@@ -881,7 +911,7 @@ public class TestTaskAttempt {
     Resource resource = Resource.newInstance(1024, 1);
 
     NodeId nid = NodeId.newInstance("127.0.0.1", 0);
-    ContainerId contId = ContainerId.newInstance(appAttemptId, 3);
+    ContainerId contId = ContainerId.newContainerId(appAttemptId, 3);
     Container container = mock(Container.class);
     when(container.getId()).thenReturn(contId);
     when(container.getNodeId()).thenReturn(nid);
@@ -984,7 +1014,7 @@ public class TestTaskAttempt {
     Resource resource = Resource.newInstance(1024, 1);
 
     NodeId nid = NodeId.newInstance("127.0.0.1", 0);
-    ContainerId contId = ContainerId.newInstance(appAttemptId, 3);
+    ContainerId contId = ContainerId.newContainerId(appAttemptId, 3);
     Container container = mock(Container.class);
     when(container.getId()).thenReturn(contId);
     when(container.getNodeId()).thenReturn(nid);
@@ -1084,7 +1114,7 @@ public class TestTaskAttempt {
     Resource resource = Resource.newInstance(1024, 1);
 
     NodeId nid = NodeId.newInstance("127.0.0.1", 0);
-    ContainerId contId = ContainerId.newInstance(appAttemptId, 3);
+    ContainerId contId = ContainerId.newContainerId(appAttemptId, 3);
     Container container = mock(Container.class);
     when(container.getId()).thenReturn(contId);
     when(container.getNodeId()).thenReturn(nid);
@@ -1211,7 +1241,7 @@ public class TestTaskAttempt {
         Resource resource, ContainerContext containerContext, boolean leafVertex) {
       super(taskId, attemptNumber, eventHandler, tal, conf,
           clock, taskHeartbeatHandler, appContext,
-          isRescheduled, resource, containerContext, leafVertex);
+          isRescheduled, resource, containerContext, leafVertex, mock(TaskImpl.class));
       this.locationHint = locationHint;
     }
     

http://git-wip-us.apache.org/repos/asf/tez/blob/f6ea0fb3/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskAttemptRecovery.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskAttemptRecovery.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskAttemptRecovery.java
index b8b09d0..d6d874d 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskAttemptRecovery.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskAttemptRecovery.java
@@ -148,7 +148,7 @@ public class TestTaskAttemptRecovery {
             mock(TaskAttemptListener.class), new Configuration(),
             new SystemClock(), mock(TaskHeartbeatHandler.class),
             mockAppContext, false, Resource.newInstance(1, 1),
-            mock(ContainerContext.class), false);
+            mock(ContainerContext.class), false, mockTask);
     taId = ta.getID();
   }
 

http://git-wip-us.apache.org/repos/asf/tez/blob/f6ea0fb3/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 9509df4..66e6724 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
@@ -630,7 +630,7 @@ public class TestTaskImpl {
         ContainerContext containerContext, Vertex vertex) {
       super(vertexId, partition, eventHandler, conf, taskAttemptListener,
           clock, thh, appContext, leafVertex, resource,
-          containerContext, mock(StateChangeNotifier.class));
+          containerContext, mock(StateChangeNotifier.class), vertex);
       this.vertex = vertex;
       this.locationHint = locationHint;
     }
@@ -687,7 +687,7 @@ public class TestTaskImpl {
         TaskLocationHint locationHint, boolean isRescheduled,
         Resource resource, ContainerContext containerContext) {
       super(taskId, attemptNumber, eventHandler, tal, conf, clock, thh,
-          appContext, isRescheduled, resource, containerContext, false);
+          appContext, isRescheduled, resource, containerContext, false, mock(TaskImpl.class));
       this.locationHint = locationHint;
     }
 

http://git-wip-us.apache.org/repos/asf/tez/blob/f6ea0fb3/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskRecovery.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskRecovery.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskRecovery.java
index e182f24..2a49826 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskRecovery.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskRecovery.java
@@ -190,7 +190,7 @@ public class TestTaskRecovery {
             new Configuration(), mock(TaskAttemptListener.class),
             new SystemClock(), mock(TaskHeartbeatHandler.class),
             mockAppContext, false, Resource.newInstance(1, 1),
-            mock(ContainerContext.class), mock(StateChangeNotifier.class));
+            mock(ContainerContext.class), mock(StateChangeNotifier.class), vertex);
 
     Map<String, OutputCommitter> committers =
         new HashMap<String, OutputCommitter>();


[43/50] [abbrv] tez git commit: TEZ-2285. Allow TaskCommunicators to indicate task/container liveness. (sseth)

Posted by ss...@apache.org.
TEZ-2285. Allow TaskCommunicators to indicate task/container liveness.  (sseth)


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

Branch: refs/heads/TEZ-2003
Commit: 2f8a2735c8ed3ac35c6137b0a3f85ba30cad3a87
Parents: 854a439
Author: Siddharth Seth <ss...@apache.org>
Authored: Tue Apr 7 13:22:09 2015 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Wed May 6 00:18:08 2015 -0700

----------------------------------------------------------------------
 TEZ-2003-CHANGES.txt                                |  1 +
 .../apache/tez/dag/api/TaskCommunicatorContext.java |  4 ++++
 .../tez/dag/app/TaskAttemptListenerImpTezDag.java   | 10 ++++++++++
 .../apache/tez/dag/app/TezTaskCommunicatorImpl.java | 16 +++++++++-------
 4 files changed, 24 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/2f8a2735/TEZ-2003-CHANGES.txt
----------------------------------------------------------------------
diff --git a/TEZ-2003-CHANGES.txt b/TEZ-2003-CHANGES.txt
index e2c428d..9d6b220 100644
--- a/TEZ-2003-CHANGES.txt
+++ b/TEZ-2003-CHANGES.txt
@@ -14,5 +14,6 @@ ALL CHANGES:
   TEZ-2241. Miscellaneous fixes after last reabse.
   TEZ-2283. Fixes after rebase 04/07.
   TEZ-2284. Separate TaskReporter into an interface.
+  TEZ-2285. Allow TaskCommunicators to indicate task/container liveness.
 
 INCOMPATIBLE CHANGES:

http://git-wip-us.apache.org/repos/asf/tez/blob/2f8a2735/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicatorContext.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicatorContext.java b/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicatorContext.java
index a85fb7f..0c3bac3 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicatorContext.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicatorContext.java
@@ -43,6 +43,10 @@ public interface TaskCommunicatorContext {
 
   boolean isKnownContainer(ContainerId containerId);
 
+  void taskAlive(TezTaskAttemptID taskAttemptId);
+
+  void containerAlive(ContainerId containerId);
+
   // TODO TEZ-2003 Move to vertex, taskIndex, version. Rename to taskAttempt*
   void taskStartedRemotely(TezTaskAttemptID taskAttemptID, ContainerId containerId);
 

http://git-wip-us.apache.org/repos/asf/tez/blob/2f8a2735/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 aaaa01c..a6ccbfa 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
@@ -259,6 +259,16 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
   }
 
   @Override
+  public void taskAlive(TezTaskAttemptID taskAttemptId) {
+    taskHeartbeatHandler.pinged(taskAttemptId);
+  }
+
+  @Override
+  public void containerAlive(ContainerId containerId) {
+    pingContainerHeartbeatHandler(containerId);
+  }
+
+  @Override
   public void taskStartedRemotely(TezTaskAttemptID taskAttemptID, ContainerId containerId) {
     context.getEventHandler()
         .handle(new TaskAttemptEventStartedRemotely(taskAttemptID, containerId, null));

http://git-wip-us.apache.org/repos/asf/tez/blob/2f8a2735/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java
index 9d0c031..ef4f764 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java
@@ -65,17 +65,19 @@ public class TezTaskCommunicatorImpl extends TaskCommunicator {
       null, true, null, null, false);
 
   private final TaskCommunicatorContext taskCommunicatorContext;
+  private final TezTaskUmbilicalProtocol taskUmbilical;
 
-  private final ConcurrentMap<ContainerId, ContainerInfo> registeredContainers =
+  protected final ConcurrentMap<ContainerId, ContainerInfo> registeredContainers =
       new ConcurrentHashMap<ContainerId, ContainerInfo>();
-  private final ConcurrentMap<TaskAttempt, ContainerId> attemptToContainerMap =
+  protected final ConcurrentMap<TaskAttempt, ContainerId> attemptToContainerMap =
       new ConcurrentHashMap<TaskAttempt, ContainerId>();
 
-  private final TezTaskUmbilicalProtocol taskUmbilical;
-  private final String tokenIdentifier;
-  private final Token<JobTokenIdentifier> sessionToken;
+
+  protected final String tokenIdentifier;
+  protected final Token<JobTokenIdentifier> sessionToken;
   protected InetSocketAddress address;
-  private Server server;
+
+  protected volatile Server server;
 
   public static final class ContainerInfo {
 
@@ -438,7 +440,7 @@ public class TezTaskCommunicatorImpl extends TaskCommunicator {
 
 
   // Holder for Task information, which eventually will likely be VertexImplm taskIndex, attemptIndex
-  private static class TaskAttempt {
+  protected static class TaskAttempt {
     // TODO TEZ-2003 Change this to work with VertexName, int id, int version
     // TODO TEZ-2003 Avoid constructing this unit all over the place
     private TezTaskAttemptID taskAttemptId;


[11/50] [abbrv] tez git commit: TEZ-2394. Issues when there is an error in VertexManager callbacks (bikas)

Posted by ss...@apache.org.
TEZ-2394. Issues when there is an error in VertexManager callbacks (bikas)


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

Branch: refs/heads/TEZ-2003
Commit: 9f090279d269fbcd63b357781318eb2163c82762
Parents: 1a53175
Author: Bikas Saha <bi...@apache.org>
Authored: Fri May 1 16:04:21 2015 -0700
Committer: Bikas Saha <bi...@apache.org>
Committed: Fri May 1 16:04:21 2015 -0700

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../tez/dag/app/dag/StateChangeNotifier.java    |  4 +-
 .../tez/dag/app/dag/impl/VertexManager.java     | 77 ++++++++++++++------
 3 files changed, 58 insertions(+), 24 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/9f090279/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 7c718ed..609db3c 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -9,6 +9,7 @@ INCOMPATIBLE CHANGES
   TEZ-1993. Implement a pluggable InputSizeEstimator for grouping fairly
 
 ALL CHANGES:
+  TEZ-2394. Issues when there is an error in VertexManager callbacks
   TEZ-2386. Tez UI: Inconsistent usage of icon colors
   TEZ-2395. Tez UI: Minimum/Maximum Duration show a empty bracket next to 0 secs when you purposefully failed a job.
   TEZ-2360. per-io counters flag should generate both overall and per-edge counters

http://git-wip-us.apache.org/repos/asf/tez/blob/9f090279/tez-dag/src/main/java/org/apache/tez/dag/app/dag/StateChangeNotifier.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/StateChangeNotifier.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/StateChangeNotifier.java
index 260cbf3..990bdea 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/StateChangeNotifier.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/StateChangeNotifier.java
@@ -71,7 +71,7 @@ public class StateChangeNotifier {
       this.listener = listener;
     }
     
-    void sentUpdate() {
+    void sendUpdate() {
       listener.onStateUpdated(update);
     }
     
@@ -105,7 +105,7 @@ public class StateChangeNotifier {
             continue;
           }
           try {
-            event.sentUpdate();
+            event.sendUpdate();
             processedEventFromQueue();
           } catch (Exception e) {
             // TODO send user code exception - TEZ-2332

http://git-wip-us.apache.org/repos/asf/tez/blob/9f090279/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexManager.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexManager.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexManager.java
index 1ed42fc..945d9ba 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexManager.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexManager.java
@@ -58,6 +58,8 @@ import org.apache.tez.dag.app.dag.Task;
 import org.apache.tez.dag.app.dag.TaskAttempt;
 import org.apache.tez.dag.app.dag.Vertex;
 import org.apache.tez.dag.app.dag.event.CallableEvent;
+import org.apache.tez.dag.app.dag.event.DAGEvent;
+import org.apache.tez.dag.app.dag.event.DAGEventType;
 import org.apache.tez.dag.app.dag.event.VertexEventInputDataInformation;
 import org.apache.tez.dag.app.dag.event.VertexEventManagerUserCodeError;
 import org.apache.tez.dag.app.dag.impl.AMUserCodeException.Source;
@@ -432,10 +434,22 @@ public class VertexManager {
     }
     if (eventInFlight.compareAndSet(false, true)) {
       // no event was in flight
+      // ensures only 1 event is in flight
       VertexManagerEvent e = eventQueue.poll();
-      Preconditions.checkState(e != null);
-      ListenableFuture<Void> future = execService.submit(e);
-      Futures.addCallback(future, e.getCallback());
+      if (e != null) {
+        ListenableFuture<Void> future = execService.submit(e);
+        Futures.addCallback(future, e.getCallback());
+      } else {
+        // This may happen. Lets say Callback succeeded on threadA. It set eventInFlight to false 
+        // and called tryScheduleNextEvent() and found queue not empty but got paused before it 
+        // could check eventInFlight.compareAndSet(). Another thread managed to dequeue the event 
+        // and schedule a callback. That callback succeeded and set eventInFlight to false, found 
+        // the queue empty and completed. Now threadA woke up and successfully did compareAndSet()
+        // tried to dequeue an event and got null.
+        // This could also happen if there is a bug and we manage to schedule for than 1 callback
+        // verify that is not the case
+        Preconditions.checkState(eventInFlight.compareAndSet(true, false));
+      }
     }
   }
 
@@ -484,36 +498,55 @@ public class VertexManager {
 
     @Override
     public void onFailure(Throwable t) {
-      // stop further event processing
-      pluginFailed.set(true);
-      eventQueue.clear();
-      // catch real root cause of failure, it would throw UndeclaredThrowableException
-      // if using UGI.doAs
-      if (t instanceof UndeclaredThrowableException) {
-        t = t.getCause();
+      try {
+        Preconditions.checkState(eventInFlight.get());
+        // stop further event processing
+        pluginFailed.set(true);
+        eventQueue.clear();
+        // catch real root cause of failure, it would throw UndeclaredThrowableException
+        // if using UGI.doAs
+        if (t instanceof UndeclaredThrowableException) {
+          t = t.getCause();
+        }
+        Preconditions.checkState(appContext != null);
+        Preconditions.checkState(managedVertex != null);
+        // state change must be triggered via an event transition
+        appContext.getEventHandler().handle(
+            new VertexEventManagerUserCodeError(managedVertex.getVertexId(),
+                new AMUserCodeException(Source.VertexManager, t)));
+        // enqueue no further events due to user code error
+      } catch (Exception e) {
+        sendInternalError(e);
       }
-      Preconditions.checkState(appContext != null);
-      Preconditions.checkState(managedVertex != null);
-      // state change must be triggered via an event transition
-      appContext.getEventHandler().handle(
-          new VertexEventManagerUserCodeError(managedVertex.getVertexId(),
-              new AMUserCodeException(Source.VertexManager, t)));
-      // enqueue no further events due to user code error
     }
     
     @Override
     public void onSuccess(Void result) {
-      Preconditions.checkState(eventInFlight.get());
-      eventInFlight.set(false);
-      tryScheduleNextEvent();
+      try {
+        onSuccessDerived(result);
+        Preconditions.checkState(eventInFlight.compareAndSet(true, false));
+        tryScheduleNextEvent();
+      } catch (Exception e) {
+        sendInternalError(e);
+      }
+    }
+    
+    protected void onSuccessDerived(Void result) {
+    }
+    
+    private void sendInternalError(Exception e) {
+      // fail the DAG so that we dont hang
+      // state change must be triggered via an event transition
+      LOG.error("Error after vertex manager callback " + managedVertex.getLogIdentifier(), e);
+      appContext.getEventHandler().handle(
+          (new DAGEvent(managedVertex.getVertexId().getDAGId(), DAGEventType.INTERNAL_ERROR)));
     }
   }
   
   private class VertexManagerRootInputInitializedCallback extends VertexManagerCallback {
 
     @Override
-    public void onSuccess(Void result) {
-      super.onSuccess(result);
+    protected void onSuccessDerived(Void result) {
       if (LOG.isDebugEnabled()) {
         LOG.debug("vertex:" + managedVertex.getLogIdentifier()
             + "; after call of VertexManagerPlugin.onRootVertexInitialized" + " on input:"


[37/50] [abbrv] tez git commit: TEZ-2123. Fix component managers to use pluggable components. Enable hybrid mode. (sseth)

Posted by ss...@apache.org.
TEZ-2123. Fix component managers to use pluggable components. Enable
hybrid mode. (sseth)


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

Branch: refs/heads/TEZ-2003
Commit: 25980c1a70a05fb915e89d246643dc8549e793cc
Parents: adab48f
Author: Siddharth Seth <ss...@apache.org>
Authored: Fri Feb 20 11:59:03 2015 -0800
Committer: Siddharth Seth <ss...@apache.org>
Committed: Wed May 6 00:13:29 2015 -0700

----------------------------------------------------------------------
 TEZ-2003-CHANGES.txt                            |   1 +
 .../org/apache/tez/dag/app/DAGAppMaster.java    |   4 +-
 .../apache/tez/dag/app/TaskAttemptListener.java |  12 +-
 .../dag/app/TaskAttemptListenerImpTezDag.java   |  30 ++--
 .../tez/dag/app/dag/impl/TaskAttemptImpl.java   |   4 +-
 .../TezRootInputInitializerContextImpl.java     |   2 +-
 .../apache/tez/dag/app/dag/impl/VertexImpl.java |   2 +-
 .../tez/dag/app/dag/impl/VertexManager.java     |   2 +-
 .../app/launcher/ContainerLauncherRouter.java   |   2 +-
 .../app/launcher/LocalContainerLauncher.java    |  10 +-
 .../rm/AMSchedulerEventDeallocateContainer.java |   7 +-
 .../rm/AMSchedulerEventNodeBlacklistUpdate.java |   8 +-
 .../tez/dag/app/rm/AMSchedulerEventTAEnded.java |  10 +-
 .../dag/app/rm/LocalTaskSchedulerService.java   |  19 ++-
 .../tez/dag/app/rm/NMCommunicatorEvent.java     |  12 +-
 .../rm/NMCommunicatorLaunchRequestEvent.java    |  11 +-
 .../app/rm/NMCommunicatorStopRequestEvent.java  |   4 +-
 .../dag/app/rm/TaskSchedulerEventHandler.java   | 151 ++++++++++++-----
 .../tez/dag/app/rm/container/AMContainer.java   |   3 +
 .../AMContainerEventLaunchRequest.java          |  15 +-
 .../dag/app/rm/container/AMContainerImpl.java   |  39 +++--
 .../dag/app/rm/container/AMContainerMap.java    |   4 +-
 .../apache/tez/dag/app/rm/node/AMNodeImpl.java  |   6 +-
 .../apache/tez/dag/app/MockDAGAppMaster.java    |   2 +-
 .../app/TestTaskAttemptListenerImplTezDag.java  |  31 ++--
 .../tez/dag/app/dag/impl/TestTaskAttempt.java   |  69 ++++----
 .../tez/dag/app/dag/impl/TestVertexImpl.java    |   8 +-
 .../tez/dag/app/rm/TestContainerReuse.java      |  34 ++--
 .../tez/dag/app/rm/TestLocalTaskScheduler.java  |   2 +-
 .../app/rm/TestLocalTaskSchedulerService.java   |  18 ++-
 .../app/rm/TestTaskSchedulerEventHandler.java   |  11 +-
 .../dag/app/rm/TestTaskSchedulerHelpers.java    |   2 +-
 .../dag/app/rm/container/TestAMContainer.java   | 108 +++++++------
 .../app/rm/container/TestAMContainerMap.java    |   6 +-
 .../org/apache/tez/examples/JoinValidate.java   |  30 +++-
 .../TezTestServiceContainerLauncher.java        |   5 +-
 .../rm/TezTestServiceTaskSchedulerService.java  | 100 ++----------
 .../tez/examples/JoinValidateConfigured.java    |  53 ++++++
 .../tez/tests/TestExternalTezServices.java      | 160 ++++++++++++++-----
 39 files changed, 638 insertions(+), 359 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/25980c1a/TEZ-2003-CHANGES.txt
----------------------------------------------------------------------
diff --git a/TEZ-2003-CHANGES.txt b/TEZ-2003-CHANGES.txt
index 4bfe08f..1a2264c 100644
--- a/TEZ-2003-CHANGES.txt
+++ b/TEZ-2003-CHANGES.txt
@@ -4,5 +4,6 @@ ALL CHANGES:
   TEZ-2090. Add tests for jobs running in external services.
   TEZ-2117. Add a manager for ContainerLaunchers running in the AM.
   TEZ-2122. Setup pluggable components at AM/Vertex level.
+  TEZ-2123. Fix component managers to use pluggable components. (Enable hybrid mode)
 
 INCOMPATIBLE CHANGES:

http://git-wip-us.apache.org/repos/asf/tez/blob/25980c1a/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 6814cda..89b6506 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
@@ -513,7 +513,7 @@ public class DAGAppMaster extends AbstractService {
 
     this.taskSchedulerEventHandler = new TaskSchedulerEventHandler(context,
         clientRpcServer, dispatcher.getEventHandler(), containerSignatureMatcher, webUIService,
-        taskSchedulerClassIdentifiers);
+        taskSchedulerClassIdentifiers, isLocal);
     addIfService(taskSchedulerEventHandler, true);
 
     if (enableWebUIService()) {
@@ -2283,6 +2283,7 @@ public class DAGAppMaster extends AbstractService {
   // Tez default classnames are populated as TezConfiguration.TEZ_AM_SERVICE_PLUGINS_DEFAULT
   private String[] parsePlugins(BiMap<String, Integer> pluginMap, String[] pluginStrings,
                                    String context) {
+    // TODO TEZ-2003 Duplicate error checking - ideally in the client itself. Depends on the final API.
     Preconditions.checkState(pluginStrings != null && pluginStrings.length > 0,
         "Plugin strings should not be null or empty: " + context);
 
@@ -2320,6 +2321,7 @@ public class DAGAppMaster extends AbstractService {
       }
       pluginMap.put(identifierString, index);
       classNames[index] = className;
+      index++;
     }
     return classNames;
   }

http://git-wip-us.apache.org/repos/asf/tez/blob/25980c1a/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListener.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListener.java b/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListener.java
index 9caa7cf..e4dad27 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListener.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListener.java
@@ -22,6 +22,7 @@ import java.net.InetSocketAddress;
 
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.tez.dag.app.dag.DAG;
+import org.apache.tez.dag.api.TaskCommunicator;
 import org.apache.tez.dag.app.rm.container.AMContainerTask;
 import org.apache.tez.dag.records.TezTaskAttemptID;
 /**
@@ -29,18 +30,17 @@ import org.apache.tez.dag.records.TezTaskAttemptID;
  */
 public interface TaskAttemptListener {
 
-  InetSocketAddress getAddress();
+  void registerRunningContainer(ContainerId containerId, int taskCommId);
 
-  void registerRunningContainer(ContainerId containerId);
-
-  void registerTaskAttempt(AMContainerTask amContainerTask, ContainerId containerId);
+  void registerTaskAttempt(AMContainerTask amContainerTask, ContainerId containerId, int taskCommId);
   
-  void unregisterRunningContainer(ContainerId containerId);
+  void unregisterRunningContainer(ContainerId containerId, int taskCommId);
   
-  void unregisterTaskAttempt(TezTaskAttemptID attemptID);
+  void unregisterTaskAttempt(TezTaskAttemptID attemptID, int taskCommId);
 
   void dagComplete(DAG dag);
 
   void dagSubmitted();
 
+  TaskCommunicator getTaskCommunicator(int taskCommIndex);
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/25980c1a/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 fc4d787..71b0d2a 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
@@ -273,11 +273,6 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
     return task.canCommit(taskAttemptId);
   }
 
-  @Override
-  public InetSocketAddress getAddress() {
-    return taskCommunicators[0].getAddress();
-  }
-
   // The TaskAttemptListener register / unregister methods in this class are not thread safe.
   // The Tez framework should not invoke these methods from multiple threads.
   @Override
@@ -297,7 +292,7 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
   }
 
   @Override
-  public void registerRunningContainer(ContainerId containerId) {
+  public void registerRunningContainer(ContainerId containerId, int taskCommId) {
     if (LOG.isDebugEnabled()) {
       LOG.debug("ContainerId: " + containerId + " registered with TaskAttemptListener");
     }
@@ -307,11 +302,12 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
           "Multiple registrations for containerId: " + containerId);
     }
     NodeId nodeId = context.getAllContainers().get(containerId).getContainer().getNodeId();
-    taskCommunicators[0].registerRunningContainer(containerId, nodeId.getHost(), nodeId.getPort());
+    taskCommunicators[taskCommId].registerRunningContainer(containerId, nodeId.getHost(),
+        nodeId.getPort());
   }
 
   @Override
-  public void unregisterRunningContainer(ContainerId containerId) {
+  public void unregisterRunningContainer(ContainerId containerId, int taskCommId) {
     if (LOG.isDebugEnabled()) {
       LOG.debug("Unregistering Container from TaskAttemptListener: " + containerId);
     }
@@ -319,12 +315,12 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
     if (containerInfo.taskAttemptId != null) {
       registeredAttempts.remove(containerInfo.taskAttemptId);
     }
-    taskCommunicators[0].registerContainerEnd(containerId);
+    taskCommunicators[taskCommId].registerContainerEnd(containerId);
   }
 
   @Override
   public void registerTaskAttempt(AMContainerTask amContainerTask,
-                                  ContainerId containerId) {
+                                  ContainerId containerId, int taskCommId) {
     ContainerInfo containerInfo = registeredContainers.get(containerId);
     if (containerInfo == null) {
       throw new TezUncheckedException("Registering task attempt: "
@@ -354,13 +350,13 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
           + amContainerTask.getTask().getTaskAttemptID() + " to container: " + containerId
           + " when already assigned to: " + containerIdFromMap);
     }
-    taskCommunicators[0].registerRunningTaskAttempt(containerId, amContainerTask.getTask(),
+    taskCommunicators[taskCommId].registerRunningTaskAttempt(containerId, amContainerTask.getTask(),
         amContainerTask.getAdditionalResources(), amContainerTask.getCredentials(),
         amContainerTask.haveCredentialsChanged());
   }
 
   @Override
-  public void unregisterTaskAttempt(TezTaskAttemptID attemptId) {
+  public void unregisterTaskAttempt(TezTaskAttemptID attemptId, int taskCommId) {
     ContainerId containerId = registeredAttempts.remove(attemptId);
     if (containerId == null) {
       LOG.warn("Unregister task attempt: " + attemptId + " from unknown container");
@@ -374,7 +370,12 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
     }
     // Explicitly putting in a new entry so that synchronization is not required on the existing element in the map.
     registeredContainers.put(containerId, NULL_CONTAINER_INFO);
-    taskCommunicators[0].unregisterRunningTaskAttempt(attemptId);
+    taskCommunicators[taskCommId].unregisterRunningTaskAttempt(attemptId);
+  }
+
+  @Override
+  public TaskCommunicator getTaskCommunicator(int taskCommIndex) {
+    return taskCommunicators[taskCommIndex];
   }
 
   private void pingContainerHeartbeatHandler(ContainerId containerId) {
@@ -413,7 +414,4 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
     return taskAttemptEvent;
   }
 
-  public TaskCommunicator getTaskCommunicator() {
-    return taskCommunicators[0];
-  }
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/25980c1a/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 c18dc00..c80571d 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
@@ -1218,7 +1218,7 @@ public class TaskAttemptImpl implements TaskAttempt,
       // Inform the scheduler
       if (sendSchedulerEvent()) {
         ta.sendEvent(new AMSchedulerEventTAEnded(ta, ta.containerId, helper
-            .getTaskAttemptState()));
+            .getTaskAttemptState(), ta.getVertex().getTaskSchedulerIdentifier()));
       }
     }
   }
@@ -1300,7 +1300,7 @@ public class TaskAttemptImpl implements TaskAttempt,
 
       // Inform the Scheduler.
       ta.sendEvent(new AMSchedulerEventTAEnded(ta, ta.containerId,
-          TaskAttemptState.SUCCEEDED));
+          TaskAttemptState.SUCCEEDED, ta.getVertex().getTaskSchedulerIdentifier()));
 
       // Inform the task.
       ta.sendEvent(new TaskEventTAUpdate(ta.attemptId,

http://git-wip-us.apache.org/repos/asf/tez/blob/25980c1a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TezRootInputInitializerContextImpl.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TezRootInputInitializerContextImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TezRootInputInitializerContextImpl.java
index d4ef4d5..4ca4024 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TezRootInputInitializerContextImpl.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TezRootInputInitializerContextImpl.java
@@ -96,7 +96,7 @@ public class TezRootInputInitializerContextImpl implements
 
   @Override
   public Resource getTotalAvailableResource() {
-    return appContext.getTaskScheduler().getTotalResources();
+    return appContext.getTaskScheduler().getTotalResources(vertex.getTaskSchedulerIdentifier());
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/tez/blob/25980c1a/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 ddf670f..81e1732 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
@@ -4207,7 +4207,7 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex, EventHandl
         eventHandler, getTotalTasks(),
         appContext.getTaskScheduler().getNumClusterNodes(),
         getTaskResource(),
-        appContext.getTaskScheduler().getTotalResources());
+        appContext.getTaskScheduler().getTotalResources(taskSchedulerIdentifier));
     List<RootInputLeafOutput<InputDescriptor, InputInitializerDescriptor>>
         inputList = Lists.newArrayListWithCapacity(inputsWithInitializers.size());
     for (String inputName : inputsWithInitializers) {

http://git-wip-us.apache.org/repos/asf/tez/blob/25980c1a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexManager.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexManager.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexManager.java
index 945d9ba..1300fc0 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexManager.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexManager.java
@@ -286,7 +286,7 @@ public class VertexManager {
     @Override
     public synchronized Resource getTotalAvailableResource() {
       checkAndThrowIfDone();
-      return appContext.getTaskScheduler().getTotalResources();
+      return appContext.getTaskScheduler().getTotalResources(managedVertex.getTaskSchedulerIdentifier());
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/tez/blob/25980c1a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherRouter.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherRouter.java b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherRouter.java
index 621e4a8..4f9b5bf 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherRouter.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherRouter.java
@@ -124,6 +124,6 @@ public class ContainerLauncherRouter extends AbstractService
 
   @Override
   public void handle(NMCommunicatorEvent event) {
-    containerLaunchers[0].handle(event);
+    containerLaunchers[event.getLauncherId()].handle(event);
   }
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/25980c1a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/LocalContainerLauncher.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/LocalContainerLauncher.java b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/LocalContainerLauncher.java
index e9ba9d7..9a38732 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/LocalContainerLauncher.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/LocalContainerLauncher.java
@@ -59,7 +59,6 @@ import org.apache.tez.dag.api.TezException;
 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.app.TaskAttemptListenerImpTezDag;
 import org.apache.tez.dag.app.TezTaskCommunicatorImpl;
 import org.apache.tez.dag.app.rm.NMCommunicatorEvent;
 import org.apache.tez.dag.app.rm.NMCommunicatorLaunchRequestEvent;
@@ -88,9 +87,9 @@ public class LocalContainerLauncher extends AbstractService implements
 
   private static final Logger LOG = LoggerFactory.getLogger(LocalContainerLauncher.class);
   private final AppContext context;
-  private final TezTaskUmbilicalProtocol taskUmbilicalProtocol;
   private final AtomicBoolean serviceStopped = new AtomicBoolean(false);
   private final String workingDirectory;
+  private final TaskAttemptListener tal;
   private final Map<String, String> localEnv = new HashMap<String, String>();
   private final ExecutionContext executionContext;
   private int numExecutors;
@@ -116,9 +115,8 @@ public class LocalContainerLauncher extends AbstractService implements
                                 String workingDirectory) throws UnknownHostException {
     super(LocalContainerLauncher.class.getName());
     this.context = context;
-    TaskAttemptListenerImpTezDag taListener = (TaskAttemptListenerImpTezDag)taskAttemptListener;
-    TezTaskCommunicatorImpl taskComm = (TezTaskCommunicatorImpl) taListener.getTaskCommunicator();
-    this.taskUmbilicalProtocol = taskComm.getUmbilical();
+    this.tal = taskAttemptListener;
+
     this.workingDirectory = workingDirectory;
     AuxiliaryServiceHelper.setServiceDataIntoEnv(
         ShuffleUtils.SHUFFLE_HANDLER_SERVICE_ID, ByteBuffer.allocate(4).putInt(0), localEnv);
@@ -219,7 +217,7 @@ public class LocalContainerLauncher extends AbstractService implements
         tezChild =
             createTezChild(context.getAMConf(), event.getContainerId(), tokenIdentifier,
                 context.getApplicationAttemptId().getAttemptId(), context.getLocalDirs(),
-                taskUmbilicalProtocol,
+                ((TezTaskCommunicatorImpl)tal.getTaskCommunicator(event.getTaskCommId())).getUmbilical(),
                 TezCommonUtils.parseCredentialsBytes(event.getContainerLaunchContext().getTokens().array()));
       } catch (InterruptedException e) {
         handleLaunchFailed(e, event.getContainerId());

http://git-wip-us.apache.org/repos/asf/tez/blob/25980c1a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/AMSchedulerEventDeallocateContainer.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/AMSchedulerEventDeallocateContainer.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/AMSchedulerEventDeallocateContainer.java
index 1b51920..5270aa2 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/AMSchedulerEventDeallocateContainer.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/AMSchedulerEventDeallocateContainer.java
@@ -23,15 +23,20 @@ import org.apache.hadoop.yarn.api.records.ContainerId;
 public class AMSchedulerEventDeallocateContainer extends AMSchedulerEvent {
 
   private final ContainerId containerId;
+  private final int schedulerId;
   
-  public AMSchedulerEventDeallocateContainer(ContainerId containerId) {
+  public AMSchedulerEventDeallocateContainer(ContainerId containerId, int schedulerId) {
     super(AMSchedulerEventType.S_CONTAINER_DEALLOCATE);
     this.containerId = containerId;
+    this.schedulerId = schedulerId;
   }
   
   public ContainerId getContainerId() {
     return this.containerId;
   }
 
+  public int getSchedulerId() {
+    return schedulerId;
+  }
 }
 

http://git-wip-us.apache.org/repos/asf/tez/blob/25980c1a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/AMSchedulerEventNodeBlacklistUpdate.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/AMSchedulerEventNodeBlacklistUpdate.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/AMSchedulerEventNodeBlacklistUpdate.java
index ed7ebc3..679705a 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/AMSchedulerEventNodeBlacklistUpdate.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/AMSchedulerEventNodeBlacklistUpdate.java
@@ -23,14 +23,20 @@ import org.apache.hadoop.yarn.api.records.NodeId;
 public class AMSchedulerEventNodeBlacklistUpdate extends AMSchedulerEvent {
 
   private final NodeId nodeId;
+  private final int schedulerId;
 
-  public AMSchedulerEventNodeBlacklistUpdate(NodeId nodeId, boolean add) {
+  public AMSchedulerEventNodeBlacklistUpdate(NodeId nodeId, boolean add, int schedulerId) {
     super((add ? AMSchedulerEventType.S_NODE_BLACKLISTED
         : AMSchedulerEventType.S_NODE_UNBLACKLISTED));
     this.nodeId = nodeId;
+    this.schedulerId = schedulerId;
   }
 
   public NodeId getNodeId() {
     return this.nodeId;
   }
+
+  public int getSchedulerId() {
+    return schedulerId;
+  }
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/25980c1a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/AMSchedulerEventTAEnded.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/AMSchedulerEventTAEnded.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/AMSchedulerEventTAEnded.java
index 90e76b7..2ace642 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/AMSchedulerEventTAEnded.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/AMSchedulerEventTAEnded.java
@@ -26,14 +26,16 @@ public class AMSchedulerEventTAEnded extends AMSchedulerEvent {
 
   private final TaskAttempt attempt;
   private final ContainerId containerId;
-  private TaskAttemptState state;
+  private final TaskAttemptState state;
+  private final int schedulerId;
 
   public AMSchedulerEventTAEnded(TaskAttempt attempt, ContainerId containerId,
-      TaskAttemptState state) {
+      TaskAttemptState state, int schedulerId) {
     super(AMSchedulerEventType.S_TA_ENDED);
     this.attempt = attempt;
     this.containerId = containerId;
     this.state = state;
+    this.schedulerId = schedulerId;
   }
 
   public TezTaskAttemptID getAttemptID() {
@@ -51,4 +53,8 @@ public class AMSchedulerEventTAEnded extends AMSchedulerEvent {
   public ContainerId getUsedContainerId() {
     return this.containerId;
   }
+
+  public int getSchedulerId() {
+    return schedulerId;
+  }
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/25980c1a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/LocalTaskSchedulerService.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/LocalTaskSchedulerService.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/LocalTaskSchedulerService.java
index 51d8b9d..72a074f 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/LocalTaskSchedulerService.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/LocalTaskSchedulerService.java
@@ -34,6 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.NodeId;
@@ -63,10 +64,11 @@ public class LocalTaskSchedulerService extends TaskSchedulerService {
   final int appHostPort;
   final String appTrackingUrl;
   final AppContext appContext;
+  final long customContainerAppId;
 
   public LocalTaskSchedulerService(TaskSchedulerAppCallback appClient,
       ContainerSignatureMatcher containerSignatureMatcher, String appHostName,
-      int appHostPort, String appTrackingUrl, AppContext appContext) {
+      int appHostPort, String appTrackingUrl, long customContainerAppId, AppContext appContext) {
     super(LocalTaskSchedulerService.class.getName());
     this.realAppClient = appClient;
     this.appCallbackExecutor = createAppCallbackExecutorService();
@@ -78,6 +80,7 @@ public class LocalTaskSchedulerService extends TaskSchedulerService {
     this.appContext = appContext;
     taskRequestQueue = new PriorityBlockingQueue<TaskRequest>();
     taskAllocations = new LinkedHashMap<Object, Container>();
+    this.customContainerAppId = customContainerAppId;
   }
 
   private ExecutorService createAppCallbackExecutorService() {
@@ -164,7 +167,7 @@ public class LocalTaskSchedulerService extends TaskSchedulerService {
 
   protected AsyncDelegateRequestHandler createRequestHandler(Configuration conf) {
     return new AsyncDelegateRequestHandler(taskRequestQueue,
-        new LocalContainerFactory(appContext),
+        new LocalContainerFactory(appContext, customContainerAppId),
         taskAllocations,
         appClientDelegate,
         conf);
@@ -195,17 +198,19 @@ public class LocalTaskSchedulerService extends TaskSchedulerService {
   }
 
   static class LocalContainerFactory {
-    final AppContext appContext;
     AtomicInteger nextId;
+    final ApplicationAttemptId customAppAttemptId;
 
-    public LocalContainerFactory(AppContext appContext) {
-      this.appContext = appContext;
+    public LocalContainerFactory(AppContext appContext, long appIdLong) {
       this.nextId = new AtomicInteger(1);
+      ApplicationId appId = ApplicationId
+          .newInstance(appIdLong, appContext.getApplicationAttemptId().getApplicationId().getId());
+      this.customAppAttemptId = ApplicationAttemptId
+          .newInstance(appId, appContext.getApplicationAttemptId().getAttemptId());
     }
 
     public Container createContainer(Resource capability, Priority priority) {
-      ApplicationAttemptId appAttemptId = appContext.getApplicationAttemptId();
-      ContainerId containerId = ContainerId.newInstance(appAttemptId, nextId.getAndIncrement());
+      ContainerId containerId = ContainerId.newInstance(customAppAttemptId, nextId.getAndIncrement());
       NodeId nodeId = NodeId.newInstance("127.0.0.1", 0);
       String nodeHttpAddress = "127.0.0.1:0";
 

http://git-wip-us.apache.org/repos/asf/tez/blob/25980c1a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/NMCommunicatorEvent.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/NMCommunicatorEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/NMCommunicatorEvent.java
index 8bdeb28..f86894f 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/NMCommunicatorEvent.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/NMCommunicatorEvent.java
@@ -28,13 +28,15 @@ public class NMCommunicatorEvent extends AbstractEvent<NMCommunicatorEventType>
   private final ContainerId containerId;
   private final NodeId nodeId;
   private final Token containerToken;
+  private final int launcherId;
 
   public NMCommunicatorEvent(ContainerId containerId, NodeId nodeId,
-      Token containerToken, NMCommunicatorEventType type) {
+      Token containerToken, NMCommunicatorEventType type, int launcherId) {
     super(type);
     this.containerId = containerId;
     this.nodeId = nodeId;
     this.containerToken = containerToken;
+    this.launcherId = launcherId;
   }
 
   public ContainerId getContainerId() {
@@ -48,10 +50,14 @@ public class NMCommunicatorEvent extends AbstractEvent<NMCommunicatorEventType>
   public Token getContainerToken() {
     return this.containerToken;
   }
-  
+
+  public int getLauncherId() {
+    return launcherId;
+  }
+
   public String toSrting() {
     return super.toString() + " for container " + containerId + ", nodeId: "
-        + nodeId;
+        + nodeId + ", launcherId: " + launcherId;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/tez/blob/25980c1a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/NMCommunicatorLaunchRequestEvent.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/NMCommunicatorLaunchRequestEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/NMCommunicatorLaunchRequestEvent.java
index c3b12c0..a38345c 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/NMCommunicatorLaunchRequestEvent.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/NMCommunicatorLaunchRequestEvent.java
@@ -25,13 +25,16 @@ public class NMCommunicatorLaunchRequestEvent extends NMCommunicatorEvent {
 
   private final ContainerLaunchContext clc;
   private final Container container;
+  // The task communicator index for the specific container being launched.
+  private final int taskCommId;
 
   public NMCommunicatorLaunchRequestEvent(ContainerLaunchContext clc,
-      Container container) {
+      Container container, int launcherId, int taskCommId) {
     super(container.getId(), container.getNodeId(), container
-        .getContainerToken(), NMCommunicatorEventType.CONTAINER_LAUNCH_REQUEST);
+        .getContainerToken(), NMCommunicatorEventType.CONTAINER_LAUNCH_REQUEST, launcherId);
     this.clc = clc;
     this.container = container;
+    this.taskCommId = taskCommId;
   }
 
   public ContainerLaunchContext getContainerLaunchContext() {
@@ -42,6 +45,10 @@ public class NMCommunicatorLaunchRequestEvent extends NMCommunicatorEvent {
     return container;
   }
 
+  public int getTaskCommId() {
+    return taskCommId;
+  }
+
   @Override
   public boolean equals(Object o) {
     if (this == o) {

http://git-wip-us.apache.org/repos/asf/tez/blob/25980c1a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/NMCommunicatorStopRequestEvent.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/NMCommunicatorStopRequestEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/NMCommunicatorStopRequestEvent.java
index 277d1e7..c9b5c44 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/NMCommunicatorStopRequestEvent.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/NMCommunicatorStopRequestEvent.java
@@ -25,9 +25,9 @@ import org.apache.hadoop.yarn.api.records.Token;
 public class NMCommunicatorStopRequestEvent extends NMCommunicatorEvent {
 
   public NMCommunicatorStopRequestEvent(ContainerId containerId, NodeId nodeId,
-      Token containerToken) {
+      Token containerToken, int launcherId) {
     super(containerId, nodeId, containerToken,
-        NMCommunicatorEventType.CONTAINER_STOP_REQUEST);
+        NMCommunicatorEventType.CONTAINER_STOP_REQUEST, launcherId);
   }
 
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/25980c1a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerEventHandler.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerEventHandler.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerEventHandler.java
index 72389e7..5a0ace8 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerEventHandler.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerEventHandler.java
@@ -108,9 +108,22 @@ public class TaskSchedulerEventHandler extends AbstractService
   private final String[] taskSchedulerClasses;
   protected final TaskSchedulerService []taskSchedulers;
 
+  private final boolean isPureLocalMode;
+  // If running in non local-only mode, the YARN task scheduler will always run to take care of
+  // registration with YARN and heartbeats to YARN.
+  // Splitting registration and heartbeats is not straigh-forward due to the taskScheduler being
+  // tied to a ContainerRequestType.
+  private final int yarnTaskSchedulerIndex;
+  // Custom AppIds to avoid container conflicts if there's multiple sources
+  private final long SCHEDULER_APP_ID_BASE = 111101111;
+  private final long SCHEDULER_APP_ID_INCREMENT = 111111111;
+
   BlockingQueue<AMSchedulerEvent> eventQueue
                               = new LinkedBlockingQueue<AMSchedulerEvent>();
 
+  // Not tracking container / task to schedulerId. Instead relying on everything flowing through
+  // the system and being propagated back via events.
+
   /**
    *
    * @param appContext
@@ -125,7 +138,7 @@ public class TaskSchedulerEventHandler extends AbstractService
   public TaskSchedulerEventHandler(AppContext appContext,
       DAGClientServer clientService, EventHandler eventHandler, 
       ContainerSignatureMatcher containerSignatureMatcher, WebUIService webUI,
-      String [] schedulerClasses) {
+      String [] schedulerClasses, boolean isPureLocalMode) {
     super(TaskSchedulerEventHandler.class.getName());
     this.appContext = appContext;
     this.eventHandler = eventHandler;
@@ -133,13 +146,39 @@ public class TaskSchedulerEventHandler extends AbstractService
     this.containerSignatureMatcher = containerSignatureMatcher;
     this.webUI = webUI;
     this.historyUrl = getHistoryUrl();
+    this.isPureLocalMode = isPureLocalMode;
     if (this.webUI != null) {
       this.webUI.setHistoryUrl(this.historyUrl);
     }
-    if (schedulerClasses == null || schedulerClasses.length == 0) {
-      this.taskSchedulerClasses = new String[] {TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT};
+
+    // Override everything for pure local mode
+    if (isPureLocalMode) {
+      this.taskSchedulerClasses = new String[] {TezConstants.TEZ_AM_SERVICE_PLUGINS_LOCAL_MODE_NAME_DEFAULT};
+      this.yarnTaskSchedulerIndex = -1;
     } else {
-      this.taskSchedulerClasses = schedulerClasses;
+      if (schedulerClasses == null || schedulerClasses.length ==0) {
+        this.taskSchedulerClasses = new String[] {TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT};
+        this.yarnTaskSchedulerIndex = 0;
+      } else {
+        // Ensure the YarnScheduler will be setup and note it's index. This will be responsible for heartbeats and YARN registration.
+        int foundYarnTaskSchedulerIndex = -1;
+        for (int i = 0 ; i < schedulerClasses.length ; i++) {
+          if (schedulerClasses[i].equals(TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT)) {
+            foundYarnTaskSchedulerIndex = i;
+            break;
+          }
+        }
+        if (foundYarnTaskSchedulerIndex == -1) { // Not found. Add at the end.
+          this.taskSchedulerClasses = new String[schedulerClasses.length+1];
+          foundYarnTaskSchedulerIndex = this.taskSchedulerClasses.length -1;
+          for (int i = 0 ; i < schedulerClasses.length ; i++) { // Copy over the rest.
+            this.taskSchedulerClasses[i] = schedulerClasses[i];
+          }
+        } else {
+          this.taskSchedulerClasses = schedulerClasses;
+        }
+        this.yarnTaskSchedulerIndex = foundYarnTaskSchedulerIndex;
+      }
     }
     taskSchedulers = new TaskSchedulerService[this.taskSchedulerClasses.length];
   }
@@ -157,12 +196,12 @@ public class TaskSchedulerEventHandler extends AbstractService
     return cachedNodeCount;
   }
   
-  public Resource getAvailableResources() {
-    return taskSchedulers[0].getAvailableResources();
+  public Resource getAvailableResources(int schedulerId) {
+    return taskSchedulers[schedulerId].getAvailableResources();
   }
 
-  public Resource getTotalResources() {
-    return taskSchedulers[0].getTotalResources();
+  public Resource getTotalResources(int schedulerId) {
+    return taskSchedulers[schedulerId].getTotalResources();
   }
 
   public synchronized void handleEvent(AMSchedulerEvent sEvent) {
@@ -176,7 +215,7 @@ public class TaskSchedulerEventHandler extends AbstractService
       switch(event.getState()) {
       case FAILED:
       case KILLED:
-        handleTAUnsuccessfulEnd((AMSchedulerEventTAEnded) sEvent);
+        handleTAUnsuccessfulEnd(event);
         break;
       case SUCCEEDED:
         handleTASucceeded(event);
@@ -228,9 +267,9 @@ public class TaskSchedulerEventHandler extends AbstractService
 
   private void handleNodeBlacklistUpdate(AMSchedulerEventNodeBlacklistUpdate event) {
     if (event.getType() == AMSchedulerEventType.S_NODE_BLACKLISTED) {
-      taskSchedulers[0].blacklistNode(event.getNodeId());
+      taskSchedulers[event.getSchedulerId()].blacklistNode(event.getNodeId());
     } else if (event.getType() == AMSchedulerEventType.S_NODE_UNBLACKLISTED) {
-      taskSchedulers[0].unblacklistNode(event.getNodeId());
+      taskSchedulers[event.getSchedulerId()].unblacklistNode(event.getNodeId());
     } else {
       throw new TezUncheckedException("Invalid event type: " + event.getType());
     }
@@ -242,14 +281,14 @@ public class TaskSchedulerEventHandler extends AbstractService
     // TODO what happens to the task that was connected to this container?
     // current assumption is that it will eventually call handleTaStopRequest
     //TaskAttempt taskAttempt = (TaskAttempt)
-    taskSchedulers[0].deallocateContainer(containerId);
+    taskSchedulers[event.getSchedulerId()].deallocateContainer(containerId);
     // TODO does this container need to be stopped via C_STOP_REQUEST
     sendEvent(new AMContainerEventStopRequest(containerId));
   }
 
   private void handleTAUnsuccessfulEnd(AMSchedulerEventTAEnded event) {
     TaskAttempt attempt = event.getAttempt();
-    boolean wasContainerAllocated = taskSchedulers[0].deallocateTask(attempt, false);
+    boolean wasContainerAllocated = taskSchedulers[event.getSchedulerId()].deallocateTask(attempt, false);
     // use stored value of container id in case the scheduler has removed this
     // assignment because the task has been deallocated earlier.
     // retroactive case
@@ -291,7 +330,8 @@ public class TaskSchedulerEventHandler extends AbstractService
           event.getAttemptID()));
     }
 
-    boolean wasContainerAllocated = taskSchedulers[0].deallocateTask(attempt, true);
+    boolean wasContainerAllocated = taskSchedulers[event.getSchedulerId()].deallocateTask(attempt,
+        true);
     if (!wasContainerAllocated) {
       LOG.error("De-allocated successful task: " + attempt.getID()
           + ", but TaskScheduler reported no container assigned to task");
@@ -316,7 +356,7 @@ public class TaskSchedulerEventHandler extends AbstractService
         TaskAttempt affinityAttempt = vertex.getTask(taskIndex).getSuccessfulAttempt();
         if (affinityAttempt != null) {
           Preconditions.checkNotNull(affinityAttempt.getAssignedContainerID(), affinityAttempt.getID());
-          taskSchedulers[0].allocateTask(taskAttempt,
+          taskSchedulers[event.getSchedulerId()].allocateTask(taskAttempt,
               event.getCapability(),
               affinityAttempt.getAssignedContainerID(),
               Priority.newInstance(event.getPriority()),
@@ -336,7 +376,7 @@ public class TaskSchedulerEventHandler extends AbstractService
       }
     }
 
-    taskSchedulers[0].allocateTask(taskAttempt,
+    taskSchedulers[event.getSchedulerId()].allocateTask(taskAttempt,
         event.getCapability(),
         hosts,
         racks,
@@ -347,7 +387,8 @@ public class TaskSchedulerEventHandler extends AbstractService
 
   private TaskSchedulerService createTaskScheduler(String host, int port, String trackingUrl,
                                                    AppContext appContext,
-                                                   String schedulerClassName) {
+                                                   String schedulerClassName,
+                                                   long customAppIdIdentifier) {
     if (schedulerClassName.equals(TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT)) {
       LOG.info("Creating TaskScheduler: YarnTaskSchedulerService");
       return new YarnTaskSchedulerService(this, this.containerSignatureMatcher,
@@ -355,7 +396,7 @@ public class TaskSchedulerEventHandler extends AbstractService
     } else if (schedulerClassName.equals(TezConstants.TEZ_AM_SERVICE_PLUGINS_LOCAL_MODE_NAME_DEFAULT)) {
       LOG.info("Creating TaskScheduler: Local TaskScheduler");
       return new LocalTaskSchedulerService(this, this.containerSignatureMatcher,
-          host, port, trackingUrl, appContext);
+          host, port, trackingUrl, customAppIdIdentifier, appContext);
     } else {
       LOG.info("Creating custom TaskScheduler: " + schedulerClassName);
       // TODO TEZ-2003 Temporary reflection with specific parameters. Remove once there is a clean interface.
@@ -364,9 +405,10 @@ public class TaskSchedulerEventHandler extends AbstractService
       try {
         Constructor<? extends TaskSchedulerService> ctor = taskSchedulerClazz
             .getConstructor(TaskSchedulerAppCallback.class, AppContext.class, String.class,
-                int.class, String.class, Configuration.class);
+                int.class, String.class, long.class, Configuration.class);
         ctor.setAccessible(true);
-        return ctor.newInstance(this, appContext, host, port, trackingUrl, getConfig());
+        return ctor.newInstance(this, appContext, host, port, trackingUrl, customAppIdIdentifier,
+            getConfig());
       } catch (NoSuchMethodException e) {
         throw new TezUncheckedException(e);
       } catch (InvocationTargetException e) {
@@ -381,10 +423,19 @@ public class TaskSchedulerEventHandler extends AbstractService
 
   @VisibleForTesting
   protected void instantiateScheduelrs(String host, int port, String trackingUrl, AppContext appContext) {
+    // TODO Add error checking for components being used in the Vertex when running in pure local mode.
     // Iterate over the list and create all the taskSchedulers
+    int j = 0;
     for (int i = 0; i < taskSchedulerClasses.length; i++) {
+      long customAppIdIdentifier;
+      if (isPureLocalMode || taskSchedulerClasses[i].equals(
+          TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT)) { // Use the app identifier from the appId.
+        customAppIdIdentifier = appContext.getApplicationID().getClusterTimestamp();
+      } else {
+        customAppIdIdentifier = SCHEDULER_APP_ID_BASE + (j++ * SCHEDULER_APP_ID_INCREMENT);
+      }
       taskSchedulers[i] = createTaskScheduler(host, port,
-          trackingUrl, appContext, taskSchedulerClasses[i]);
+          trackingUrl, appContext, taskSchedulerClasses[i], customAppIdIdentifier);
     }
   }
 
@@ -403,12 +454,12 @@ public class TaskSchedulerEventHandler extends AbstractService
     for (int i = 0 ; i < taskSchedulers.length ; i++) {
       taskSchedulers[i].init(getConfig());
       taskSchedulers[i].start();
-    }
-
-    // TODO TEZ-2118 Start using multiple task schedulers
-    if (shouldUnregisterFlag.get()) {
-      // Flag may have been set earlier when task scheduler was not initialized
-      taskSchedulers[0].setShouldUnregister();
+      if (shouldUnregisterFlag.get()) {
+        // Flag may have been set earlier when task scheduler was not initialized
+        // TODO TEZ-2003 Should setRegister / unregister be part of APIs when not YARN specific ?
+        // External services could need to talk to some other entity.
+        taskSchedulers[i].setShouldUnregister();
+      }
     }
 
     this.eventHandlingThread = new Thread("TaskSchedulerEventHandlerThread") {
@@ -457,8 +508,10 @@ public class TaskSchedulerEventHandler extends AbstractService
       if (eventHandlingThread != null)
         eventHandlingThread.interrupt();
     }
-    if (taskSchedulers[0] != null) {
-      ((AbstractService)taskSchedulers[0]).stop();
+    for (int i = 0 ; i < taskSchedulers.length ; i++) {
+      if (taskSchedulers[i] != null) {
+        taskSchedulers[i].stop();
+      }
     }
   }
 
@@ -467,15 +520,18 @@ public class TaskSchedulerEventHandler extends AbstractService
   public synchronized void taskAllocated(Object task,
                                            Object appCookie,
                                            Container container) {
+    AMSchedulerEventTALaunchRequest event =
+        (AMSchedulerEventTALaunchRequest) appCookie;
     ContainerId containerId = container.getId();
-    if (appContext.getAllContainers().addContainerIfNew(container)) {
+    if (appContext.getAllContainers()
+        .addContainerIfNew(container, event.getSchedulerId(), event.getLauncherId(),
+            event.getTaskCommId())) {
       appContext.getNodeTracker().nodeSeen(container.getNodeId());
       sendEvent(new AMNodeEventContainerAllocated(container
           .getNodeId(), container.getId()));
     }
 
-    AMSchedulerEventTALaunchRequest event =
-                         (AMSchedulerEventTALaunchRequest) appCookie;
+
     TaskAttempt taskAttempt = event.getTaskAttempt();
     // TODO - perhaps check if the task still needs this container
     // because the deallocateTask downcall may have raced with the
@@ -484,7 +540,7 @@ public class TaskSchedulerEventHandler extends AbstractService
  
     if (appContext.getAllContainers().get(containerId).getState() == AMContainerState.ALLOCATED) {
       sendEvent(new AMContainerEventLaunchRequest(containerId, taskAttempt.getVertexID(),
-          event.getContainerContext()));
+          event.getContainerContext(), event.getLauncherId(), event.getTaskCommId()));
     }
     sendEvent(new DAGEventSchedulerUpdateTAAssigned(taskAttempt, container));
     sendEvent(new AMContainerEventAssignTA(containerId, taskAttempt.getID(),
@@ -603,6 +659,9 @@ public class TaskSchedulerEventHandler extends AbstractService
   public float getProgress() {
     // at this point allocate has been called and so node count must be available
     // may change after YARN-1722
+    // This is a heartbeat in from the scheduler into the APP, and is being used to piggy-back and
+    // node updates from the cluster.
+    // TODO Handle this in TEZ-2124. Need a way to know which scheduler is calling in.
     int nodeCount = taskSchedulers[0].getClusterNodeCount();
     if (nodeCount != cachedNodeCount) {
       cachedNodeCount = nodeCount;
@@ -618,7 +677,9 @@ public class TaskSchedulerEventHandler extends AbstractService
   }
 
   public void dagCompleted() {
-    taskSchedulers[0].dagComplete();
+    for (int i = 0 ; i < taskSchedulers.length ; i++) {
+      taskSchedulers[i].dagComplete();
+    }
   }
 
   public void dagSubmitted() {
@@ -628,7 +689,10 @@ public class TaskSchedulerEventHandler extends AbstractService
 
   @Override
   public void preemptContainer(ContainerId containerId) {
-    taskSchedulers[0].deallocateContainer(containerId);
+    // TODO Why is this making a call back into the scheduler, when the call is originating from there.
+    // An AMContainer instance should already exist if an attempt is being made to preempt it
+    AMContainer amContainer = appContext.getAllContainers().get(containerId);
+    taskSchedulers[amContainer.getTaskSchedulerIdentifier()].deallocateContainer(containerId);
     // Inform the Containers about completion.
     sendEvent(new AMContainerEventCompleted(containerId, ContainerExitStatus.INVALID,
         "Container preempted internally", TaskAttemptTerminationCause.INTERNAL_PREEMPTION));
@@ -637,13 +701,24 @@ public class TaskSchedulerEventHandler extends AbstractService
   public void setShouldUnregisterFlag() {
     LOG.info("TaskScheduler notified that it should unregister from RM");
     this.shouldUnregisterFlag.set(true);
-    if (this.taskSchedulers[0] != null) {
-      this.taskSchedulers[0].setShouldUnregister();
+    for (int i = 0 ; i < taskSchedulers.length ; i++) {
+      if (this.taskSchedulers[i] != null) {
+        // TODO TEZ-2003 registration required for all schedulers ?
+        this.taskSchedulers[i].setShouldUnregister();
+      }
     }
   }
 
   public boolean hasUnregistered() {
-    return this.taskSchedulers[0].hasUnregistered();
+    boolean result = true;
+    for (int i = 0 ; i < taskSchedulers.length ; i++) {
+      // TODO TEZ-2003 registration required for all schedulers ?
+      result |= this.taskSchedulers[i].hasUnregistered();
+      if (result == false) {
+        return result;
+      }
+    }
+    return result;
   }
 
   @VisibleForTesting

http://git-wip-us.apache.org/repos/asf/tez/blob/25980c1a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainer.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainer.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainer.java
index 0fc2e12..6616896 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainer.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainer.java
@@ -34,4 +34,7 @@ public interface AMContainer extends EventHandler<AMContainerEvent>{
   public List<TezTaskAttemptID> getAllTaskAttempts();
   public TezTaskAttemptID getCurrentTaskAttempt();
 
+  public int getTaskSchedulerIdentifier();
+  public int getContainerLauncherIdentifier();
+  public int getTaskCommunicatorIdentifier();
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/25980c1a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerEventLaunchRequest.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerEventLaunchRequest.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerEventLaunchRequest.java
index d973264..92e5817 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerEventLaunchRequest.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerEventLaunchRequest.java
@@ -27,12 +27,17 @@ public class AMContainerEventLaunchRequest extends AMContainerEvent {
 
   private final TezVertexID vertexId;
   private final ContainerContext containerContext;
+  private final int launcherId;
+  private final int taskCommId;
 
   public AMContainerEventLaunchRequest(ContainerId containerId,
-      TezVertexID vertexId, ContainerContext containerContext) {
+      TezVertexID vertexId, ContainerContext containerContext,
+      int launcherId, int taskCommId) {
     super(containerId, AMContainerEventType.C_LAUNCH_REQUEST);
     this.vertexId = vertexId;
     this.containerContext = containerContext;
+    this.launcherId = launcherId;
+    this.taskCommId = taskCommId;
   }
 
   public TezDAGID getDAGId() {
@@ -46,4 +51,12 @@ public class AMContainerEventLaunchRequest extends AMContainerEvent {
   public ContainerContext getContainerContext() {
     return this.containerContext;
   }
+
+  public int getLauncherId() {
+    return launcherId;
+  }
+
+  public int getTaskCommId() {
+    return taskCommId;
+  }
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/25980c1a/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 1acec9c..39df2e8 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
@@ -81,6 +81,9 @@ public class AMContainerImpl implements AMContainer {
   private final TaskAttemptListener taskAttemptListener;
   protected final EventHandler eventHandler;
   private final ContainerSignatureMatcher signatureMatcher;
+  private final int schedulerId;
+  private final int launcherId;
+  private final int taskCommId;
 
   private final List<TezTaskAttemptID> completedAttempts =
       new LinkedList<TezTaskAttemptID>();
@@ -302,7 +305,7 @@ public class AMContainerImpl implements AMContainer {
   // additional change - JvmID, YarnChild, etc depend on TaskType.
   public AMContainerImpl(Container container, ContainerHeartbeatHandler chh,
       TaskAttemptListener tal, ContainerSignatureMatcher signatureMatcher,
-      AppContext appContext) {
+      AppContext appContext, int schedulerId, int launcherId, int taskCommId) {
     ReentrantReadWriteLock rwLock = new ReentrantReadWriteLock();
     this.readLock = rwLock.readLock();
     this.writeLock = rwLock.writeLock();
@@ -314,6 +317,9 @@ public class AMContainerImpl implements AMContainer {
     this.containerHeartbeatHandler = chh;
     this.taskAttemptListener = tal;
     this.failedAssignments = new LinkedList<TezTaskAttemptID>();
+    this.schedulerId = schedulerId;
+    this.launcherId = launcherId;
+    this.taskCommId = taskCommId;
     this.stateMachine = stateMachineFactory.make(this);
   }
 
@@ -363,6 +369,21 @@ public class AMContainerImpl implements AMContainer {
     }
   }
 
+  @Override
+  public int getTaskSchedulerIdentifier() {
+    return this.schedulerId;
+  }
+
+  @Override
+  public int getContainerLauncherIdentifier() {
+    return this.launcherId;
+  }
+
+  @Override
+  public int getTaskCommunicatorIdentifier() {
+    return this.taskCommId;
+  }
+
   public boolean isInErrorState() {
     return inError;
   }
@@ -432,7 +453,7 @@ public class AMContainerImpl implements AMContainer {
           containerContext.getLocalResources(),
           containerContext.getEnvironment(),
           containerContext.getJavaOpts(),
-          container.taskAttemptListener.getAddress(), containerContext.getCredentials(),
+          container.taskAttemptListener.getTaskCommunicator(container.taskCommId).getAddress(), containerContext.getCredentials(),
           container.appContext, container.container.getResource(),
           container.appContext.getAMConf());
 
@@ -1014,7 +1035,7 @@ public class AMContainerImpl implements AMContainer {
   }
   
   protected void deAllocate() {
-    sendEvent(new AMSchedulerEventDeallocateContainer(containerId));
+    sendEvent(new AMSchedulerEventDeallocateContainer(containerId, schedulerId));
   }
 
   protected void sendTerminatedToTaskAttempt(
@@ -1044,28 +1065,28 @@ public class AMContainerImpl implements AMContainer {
   }
 
   protected void sendStartRequestToNM(ContainerLaunchContext clc) {
-    sendEvent(new NMCommunicatorLaunchRequestEvent(clc, container));
+    sendEvent(new NMCommunicatorLaunchRequestEvent(clc, container, launcherId, taskCommId));
   }
 
   protected void sendStopRequestToNM() {
     sendEvent(new NMCommunicatorStopRequestEvent(containerId,
-        container.getNodeId(), container.getContainerToken()));
+        container.getNodeId(), container.getContainerToken(), launcherId));
   }
 
   protected void unregisterAttemptFromListener(TezTaskAttemptID attemptId) {
-    taskAttemptListener.unregisterTaskAttempt(attemptId);
+    taskAttemptListener.unregisterTaskAttempt(attemptId, taskCommId);
   }
 
   protected void registerAttemptWithListener(AMContainerTask amContainerTask) {
-    taskAttemptListener.registerTaskAttempt(amContainerTask, this.containerId);
+    taskAttemptListener.registerTaskAttempt(amContainerTask, this.containerId, taskCommId);
   }
 
   protected void registerWithTAListener() {
-    taskAttemptListener.registerRunningContainer(containerId);
+    taskAttemptListener.registerRunningContainer(containerId, taskCommId);
   }
 
   protected void unregisterFromTAListener() {
-    this.taskAttemptListener.unregisterRunningContainer(containerId);
+    this.taskAttemptListener.unregisterRunningContainer(containerId, taskCommId);
   }
 
   protected void registerWithContainerListener() {

http://git-wip-us.apache.org/repos/asf/tez/blob/25980c1a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerMap.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerMap.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerMap.java
index 574c38e..938096d 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerMap.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerMap.java
@@ -62,9 +62,9 @@ public class AMContainerMap extends AbstractService implements EventHandler<AMCo
     }
   }
 
-  public boolean addContainerIfNew(Container container) {
+  public boolean addContainerIfNew(Container container, int schedulerId, int launcherId, int taskCommId) {
     AMContainer amc = new AMContainerImpl(container, chh, tal,
-      containerSignatureMatcher, context);
+      containerSignatureMatcher, context, schedulerId, launcherId, taskCommId);
     return (containerMap.putIfAbsent(container.getId(), amc) == null);
   }
 

http://git-wip-us.apache.org/repos/asf/tez/blob/25980c1a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeImpl.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeImpl.java
index b93cab3..0d8e4cd 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeImpl.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeImpl.java
@@ -257,7 +257,8 @@ public class AMNodeImpl implements AMNode {
     // these containers are not useful anymore
     pastContainers.addAll(containers);
     containers.clear();
-    sendEvent(new AMSchedulerEventNodeBlacklistUpdate(getNodeId(), true));
+    // TODO TEZ-2124 node tracking per ext source
+    sendEvent(new AMSchedulerEventNodeBlacklistUpdate(getNodeId(), true, 0));
   }
 
   @SuppressWarnings("unchecked")
@@ -363,7 +364,8 @@ public class AMNodeImpl implements AMNode {
     public void transition(AMNodeImpl node, AMNodeEvent nEvent) {
       node.ignoreBlacklisting = ignore;
       if (node.getState() == AMNodeState.BLACKLISTED) {
-        node.sendEvent(new AMSchedulerEventNodeBlacklistUpdate(node.getNodeId(), false));
+        // TODO TEZ-2124 node tracking per ext source
+        node.sendEvent(new AMSchedulerEventNodeBlacklistUpdate(node.getNodeId(), false, 0));
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/tez/blob/25980c1a/tez-dag/src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java b/tez-dag/src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java
index 26fc1ab..a466bc6 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java
@@ -192,7 +192,7 @@ public class MockDAGAppMaster extends DAGAppMaster {
     @Override
     public void serviceStart() throws Exception {
       taListener = (TaskAttemptListenerImpTezDag) getTaskAttemptListener();
-      taskCommunicator = (TezTaskCommunicatorImpl) taListener.getTaskCommunicator();
+      taskCommunicator = (TezTaskCommunicatorImpl) taListener.getTaskCommunicator(0);
       eventHandlingThread = new Thread(this);
       eventHandlingThread.start();
       ExecutorService rawExecutor = Executors.newFixedThreadPool(handlerConcurrency,

http://git-wip-us.apache.org/repos/asf/tez/blob/25980c1a/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag.java b/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag.java
index 16bd1d3..bffb5b9 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag.java
@@ -108,9 +108,16 @@ public class TestTaskAttemptListenerImplTezDag {
     doReturn(dag).when(appContext).getCurrentDAG();
     doReturn(appAcls).when(appContext).getApplicationACLs();
     doReturn(amContainerMap).when(appContext).getAllContainers();
+    NodeId nodeId = NodeId.newInstance("localhost", 0);
+
+    AMContainer amContainer = mock(AMContainer.class);
+    Container container = mock(Container.class);
+    doReturn(nodeId).when(container).getNodeId();
+    doReturn(amContainer).when(amContainerMap).get(any(ContainerId.class));
+    doReturn(container).when(amContainer).getContainer();
 
     taskAttemptListener = new TaskAttemptListenerImplForTest(appContext,
-        mock(TaskHeartbeatHandler.class), mock(ContainerHeartbeatHandler.class), null);
+        mock(TaskHeartbeatHandler.class), mock(ContainerHeartbeatHandler.class), null, null);
 
     taskSpec = mock(TaskSpec.class);
     doReturn(taskAttemptID).when(taskSpec).getTaskAttemptID();
@@ -121,7 +128,7 @@ public class TestTaskAttemptListenerImplTezDag {
   @Test(timeout = 5000)
   public void testGetTask() throws IOException {
 
-    TezTaskCommunicatorImpl taskCommunicator = (TezTaskCommunicatorImpl)taskAttemptListener.getTaskCommunicator();
+    TezTaskCommunicatorImpl taskCommunicator = (TezTaskCommunicatorImpl)taskAttemptListener.getTaskCommunicator(0);
     TezTaskUmbilicalProtocol tezUmbilical = taskCommunicator.getUmbilical();
 
     ContainerId containerId1 = createContainerId(appId, 1);
@@ -131,55 +138,55 @@ public class TestTaskAttemptListenerImplTezDag {
 
     ContainerId containerId2 = createContainerId(appId, 2);
     ContainerContext containerContext2 = new ContainerContext(containerId2.toString());
-    taskAttemptListener.registerRunningContainer(containerId2);
+    taskAttemptListener.registerRunningContainer(containerId2, 0);
     containerTask = tezUmbilical.getTask(containerContext2);
     assertNull(containerTask);
 
     // Valid task registered
-    taskAttemptListener.registerTaskAttempt(amContainerTask, containerId2);
+    taskAttemptListener.registerTaskAttempt(amContainerTask, containerId2, 0);
     containerTask = tezUmbilical.getTask(containerContext2);
     assertFalse(containerTask.shouldDie());
     assertEquals(taskSpec, containerTask.getTaskSpec());
 
     // Task unregistered. Should respond to heartbeats
-    taskAttemptListener.unregisterTaskAttempt(taskAttemptId);
+    taskAttemptListener.unregisterTaskAttempt(taskAttemptId, 0);
     containerTask = tezUmbilical.getTask(containerContext2);
     assertNull(containerTask);
 
     // Container unregistered. Should send a shouldDie = true
-    taskAttemptListener.unregisterRunningContainer(containerId2);
+    taskAttemptListener.unregisterRunningContainer(containerId2, 0);
     containerTask = tezUmbilical.getTask(containerContext2);
     assertTrue(containerTask.shouldDie());
 
     ContainerId containerId3 = createContainerId(appId, 3);
     ContainerContext containerContext3 = new ContainerContext(containerId3.toString());
-    taskAttemptListener.registerRunningContainer(containerId3);
+    taskAttemptListener.registerRunningContainer(containerId3, 0);
 
     // Register task to container3, followed by unregistering container 3 all together
     TaskSpec taskSpec2 = mock(TaskSpec.class);
     TezTaskAttemptID taskAttemptId2 = mock(TezTaskAttemptID.class);
     doReturn(taskAttemptId2).when(taskSpec2).getTaskAttemptID();
     AMContainerTask amContainerTask2 = new AMContainerTask(taskSpec, null, null, false, 0);
-    taskAttemptListener.registerTaskAttempt(amContainerTask2, containerId3);
-    taskAttemptListener.unregisterRunningContainer(containerId3);
+    taskAttemptListener.registerTaskAttempt(amContainerTask2, containerId3, 0);
+    taskAttemptListener.unregisterRunningContainer(containerId3, 0);
     containerTask = tezUmbilical.getTask(containerContext3);
     assertTrue(containerTask.shouldDie());
   }
 
   @Test(timeout = 5000)
   public void testGetTaskMultiplePulls() throws IOException {
-    TezTaskCommunicatorImpl taskCommunicator = (TezTaskCommunicatorImpl)taskAttemptListener.getTaskCommunicator();
+    TezTaskCommunicatorImpl taskCommunicator = (TezTaskCommunicatorImpl)taskAttemptListener.getTaskCommunicator(0);
     TezTaskUmbilicalProtocol tezUmbilical = taskCommunicator.getUmbilical();
 
     ContainerId containerId1 = createContainerId(appId, 1);
     doReturn(mock(AMContainer.class)).when(amContainerMap).get(containerId1);
     ContainerContext containerContext1 = new ContainerContext(containerId1.toString());
-    taskAttemptListener.registerRunningContainer(containerId1);
+    taskAttemptListener.registerRunningContainer(containerId1, 0);
     containerTask = tezUmbilical.getTask(containerContext1);
     assertNull(containerTask);
 
     // Register task
-    taskAttemptListener.registerTaskAttempt(amContainerTask, containerId1);
+    taskAttemptListener.registerTaskAttempt(amContainerTask, containerId1, 0);
     containerTask = tezUmbilical.getTask(containerContext1);
     assertFalse(containerTask.shouldDie());
     assertEquals(taskSpec, containerTask.getTaskSpec());

http://git-wip-us.apache.org/repos/asf/tez/blob/25980c1a/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 60c4c88..9df225c 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
@@ -56,6 +56,7 @@ import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.util.Clock;
 import org.apache.hadoop.yarn.util.SystemClock;
 import org.apache.tez.common.MockDNSToSwitchMapping;
+import org.apache.tez.dag.api.TaskCommunicator;
 import org.apache.tez.dag.api.TaskLocationHint;
 import org.apache.tez.dag.api.TezConfiguration;
 import org.apache.tez.dag.api.oldrecords.TaskAttemptState;
@@ -273,8 +274,9 @@ public class TestTaskAttempt {
 
     MockEventHandler eventHandler = new MockEventHandler();
     TaskAttemptListener taListener = mock(TaskAttemptListener.class);
-    when(taListener.getAddress()).thenReturn(
-        new InetSocketAddress("localhost", 0));
+    TaskCommunicator taskComm = mock(TaskCommunicator.class);
+    doReturn(new InetSocketAddress("localhost", 0)).when(taskComm).getAddress();
+    doReturn(taskComm).when(taListener).getTaskCommunicator(0);
 
     Configuration taskConf = new Configuration();
     taskConf.setClass("fs.file.impl", StubbedFS.class, FileSystem.class);
@@ -323,8 +325,9 @@ public class TestTaskAttempt {
 
     MockEventHandler eventHandler = spy(new MockEventHandler());
     TaskAttemptListener taListener = mock(TaskAttemptListener.class);
-    when(taListener.getAddress()).thenReturn(
-        new InetSocketAddress("localhost", 0));
+    TaskCommunicator taskComm = mock(TaskCommunicator.class);
+    doReturn(new InetSocketAddress("localhost", 0)).when(taskComm).getAddress();
+    doReturn(taskComm).when(taListener).getTaskCommunicator(0);
 
     Configuration taskConf = new Configuration();
     taskConf.setClass("fs.file.impl", StubbedFS.class, FileSystem.class);
@@ -345,7 +348,7 @@ public class TestTaskAttempt {
     AMContainerMap containers = new AMContainerMap(
         mock(ContainerHeartbeatHandler.class), mock(TaskAttemptListener.class),
         new ContainerContextMatcher(), appCtx);
-    containers.addContainerIfNew(container);
+    containers.addContainerIfNew(container, 0, 0, 0);
 
     doReturn(new ClusterInfo()).when(appCtx).getClusterInfo();
     doReturn(containers).when(appCtx).getAllContainers();
@@ -424,8 +427,9 @@ public class TestTaskAttempt {
 
     MockEventHandler eventHandler = new MockEventHandler();
     TaskAttemptListener taListener = mock(TaskAttemptListener.class);
-    when(taListener.getAddress()).thenReturn(
-        new InetSocketAddress("localhost", 0));
+    TaskCommunicator taskComm = mock(TaskCommunicator.class);
+    doReturn(new InetSocketAddress("localhost", 0)).when(taskComm).getAddress();
+    doReturn(taskComm).when(taListener).getTaskCommunicator(0);
 
     Configuration taskConf = new Configuration();
     taskConf.setClass("fs.file.impl", StubbedFS.class, FileSystem.class);
@@ -446,7 +450,7 @@ public class TestTaskAttempt {
     AMContainerMap containers = new AMContainerMap(
         mock(ContainerHeartbeatHandler.class), mock(TaskAttemptListener.class),
         new ContainerContextMatcher(), appCtx);
-    containers.addContainerIfNew(container);
+    containers.addContainerIfNew(container, 0, 0, 0);
 
     doReturn(new ClusterInfo()).when(appCtx).getClusterInfo();
     doReturn(containers).when(appCtx).getAllContainers();
@@ -489,8 +493,9 @@ public class TestTaskAttempt {
 
     MockEventHandler eventHandler = spy(new MockEventHandler());
     TaskAttemptListener taListener = mock(TaskAttemptListener.class);
-    when(taListener.getAddress()).thenReturn(
-        new InetSocketAddress("localhost", 0));
+    TaskCommunicator taskComm = mock(TaskCommunicator.class);
+    doReturn(new InetSocketAddress("localhost", 0)).when(taskComm).getAddress();
+    doReturn(taskComm).when(taListener).getTaskCommunicator(0);
 
     Configuration taskConf = new Configuration();
     taskConf.setClass("fs.file.impl", StubbedFS.class, FileSystem.class);
@@ -511,7 +516,7 @@ public class TestTaskAttempt {
     AMContainerMap containers = new AMContainerMap(
         mock(ContainerHeartbeatHandler.class), mock(TaskAttemptListener.class),
         new ContainerContextMatcher(), appCtx);
-    containers.addContainerIfNew(container);
+    containers.addContainerIfNew(container, 0, 0, 0);
 
     doReturn(new ClusterInfo()).when(appCtx).getClusterInfo();
     doReturn(containers).when(appCtx).getAllContainers();
@@ -581,8 +586,9 @@ public class TestTaskAttempt {
 
     MockEventHandler eventHandler = spy(new MockEventHandler());
     TaskAttemptListener taListener = mock(TaskAttemptListener.class);
-    when(taListener.getAddress()).thenReturn(
-        new InetSocketAddress("localhost", 0));
+    TaskCommunicator taskComm = mock(TaskCommunicator.class);
+    doReturn(new InetSocketAddress("localhost", 0)).when(taskComm).getAddress();
+    doReturn(taskComm).when(taListener).getTaskCommunicator(0);
 
     Configuration taskConf = new Configuration();
     taskConf.setClass("fs.file.impl", StubbedFS.class, FileSystem.class);
@@ -604,7 +610,7 @@ public class TestTaskAttempt {
     AMContainerMap containers = new AMContainerMap(
         mock(ContainerHeartbeatHandler.class), mock(TaskAttemptListener.class),
         new ContainerContextMatcher(), appCtx);
-    containers.addContainerIfNew(container);
+    containers.addContainerIfNew(container, 0, 0, 0);
 
     doReturn(new ClusterInfo()).when(appCtx).getClusterInfo();
     doReturn(containers).when(appCtx).getAllContainers();
@@ -712,8 +718,9 @@ public class TestTaskAttempt {
 
     MockEventHandler eventHandler = spy(new MockEventHandler());
     TaskAttemptListener taListener = mock(TaskAttemptListener.class);
-    when(taListener.getAddress()).thenReturn(
-        new InetSocketAddress("localhost", 0));
+    TaskCommunicator taskComm = mock(TaskCommunicator.class);
+    doReturn(new InetSocketAddress("localhost", 0)).when(taskComm).getAddress();
+    doReturn(taskComm).when(taListener).getTaskCommunicator(0);
 
     Configuration taskConf = new Configuration();
     taskConf.setClass("fs.file.impl", StubbedFS.class, FileSystem.class);
@@ -735,7 +742,7 @@ public class TestTaskAttempt {
     AMContainerMap containers = new AMContainerMap(
         mock(ContainerHeartbeatHandler.class), mock(TaskAttemptListener.class),
         new ContainerContextMatcher(), appCtx);
-    containers.addContainerIfNew(container);
+    containers.addContainerIfNew(container, 0, 0, 0);
 
     doReturn(new ClusterInfo()).when(appCtx).getClusterInfo();
     doReturn(containers).when(appCtx).getAllContainers();
@@ -804,8 +811,9 @@ public class TestTaskAttempt {
 
     MockEventHandler eventHandler = spy(new MockEventHandler());
     TaskAttemptListener taListener = mock(TaskAttemptListener.class);
-    when(taListener.getAddress()).thenReturn(
-        new InetSocketAddress("localhost", 0));
+    TaskCommunicator taskComm = mock(TaskCommunicator.class);
+    doReturn(new InetSocketAddress("localhost", 0)).when(taskComm).getAddress();
+    doReturn(taskComm).when(taListener).getTaskCommunicator(0);
 
     Configuration taskConf = new Configuration();
     taskConf.setClass("fs.file.impl", StubbedFS.class, FileSystem.class);
@@ -826,7 +834,7 @@ public class TestTaskAttempt {
     AMContainerMap containers = new AMContainerMap(
         mock(ContainerHeartbeatHandler.class), mock(TaskAttemptListener.class),
         new ContainerContextMatcher(), appCtx);
-    containers.addContainerIfNew(container);
+    containers.addContainerIfNew(container, 0, 0, 0);
 
     doReturn(new ClusterInfo()).when(appCtx).getClusterInfo();
     doReturn(containers).when(appCtx).getAllContainers();
@@ -899,8 +907,9 @@ public class TestTaskAttempt {
 
     MockEventHandler eventHandler = spy(new MockEventHandler());
     TaskAttemptListener taListener = mock(TaskAttemptListener.class);
-    when(taListener.getAddress()).thenReturn(
-        new InetSocketAddress("localhost", 0));
+    TaskCommunicator taskComm = mock(TaskCommunicator.class);
+    doReturn(new InetSocketAddress("localhost", 0)).when(taskComm).getAddress();
+    doReturn(taskComm).when(taListener).getTaskCommunicator(0);
 
     Configuration taskConf = new Configuration();
     taskConf.setClass("fs.file.impl", StubbedFS.class, FileSystem.class);
@@ -921,7 +930,7 @@ public class TestTaskAttempt {
     AMContainerMap containers = new AMContainerMap(
         mock(ContainerHeartbeatHandler.class), mock(TaskAttemptListener.class),
         new ContainerContextMatcher(), appCtx);
-    containers.addContainerIfNew(container);
+    containers.addContainerIfNew(container, 0, 0, 0);
 
     doReturn(new ClusterInfo()).when(appCtx).getClusterInfo();
     doReturn(containers).when(appCtx).getAllContainers();
@@ -1002,8 +1011,9 @@ public class TestTaskAttempt {
 
     MockEventHandler eventHandler = spy(new MockEventHandler());
     TaskAttemptListener taListener = mock(TaskAttemptListener.class);
-    when(taListener.getAddress()).thenReturn(
-        new InetSocketAddress("localhost", 0));
+    TaskCommunicator taskComm = mock(TaskCommunicator.class);
+    doReturn(new InetSocketAddress("localhost", 0)).when(taskComm).getAddress();
+    doReturn(taskComm).when(taListener).getTaskCommunicator(0);
 
     Configuration taskConf = new Configuration();
     taskConf.setClass("fs.file.impl", StubbedFS.class, FileSystem.class);
@@ -1024,7 +1034,7 @@ public class TestTaskAttempt {
     AMContainerMap containers = new AMContainerMap(
         mock(ContainerHeartbeatHandler.class), mock(TaskAttemptListener.class),
         new ContainerContextMatcher(), appCtx);
-    containers.addContainerIfNew(container);
+    containers.addContainerIfNew(container, 0, 0, 0);
 
     doReturn(new ClusterInfo()).when(appCtx).getClusterInfo();
     doReturn(containers).when(appCtx).getAllContainers();
@@ -1102,8 +1112,9 @@ public class TestTaskAttempt {
     MockEventHandler mockEh = new MockEventHandler();
     MockEventHandler eventHandler = spy(mockEh);
     TaskAttemptListener taListener = mock(TaskAttemptListener.class);
-    when(taListener.getAddress()).thenReturn(
-        new InetSocketAddress("localhost", 0));
+    TaskCommunicator taskComm = mock(TaskCommunicator.class);
+    doReturn(new InetSocketAddress("localhost", 0)).when(taskComm).getAddress();
+    doReturn(taskComm).when(taListener).getTaskCommunicator(0);
 
     Configuration taskConf = new Configuration();
     taskConf.setClass("fs.file.impl", StubbedFS.class, FileSystem.class);
@@ -1124,7 +1135,7 @@ public class TestTaskAttempt {
     AMContainerMap containers = new AMContainerMap(
         mock(ContainerHeartbeatHandler.class), mock(TaskAttemptListener.class),
         new ContainerContextMatcher(), appCtx);
-    containers.addContainerIfNew(container);
+    containers.addContainerIfNew(container, 0, 0, 0);
 
     doReturn(new ClusterInfo()).when(appCtx).getClusterInfo();
     doReturn(containers).when(appCtx).getAllContainers();

http://git-wip-us.apache.org/repos/asf/tez/blob/25980c1a/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 99ec6cf..df29eaa 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
@@ -2176,7 +2176,7 @@ public class TestVertexImpl {
     doReturn(dagId).when(appContext).getCurrentDAGID();
     doReturn(dagId).when(dag).getID();
     doReturn(taskScheduler).when(appContext).getTaskScheduler();
-    doReturn(Resource.newInstance(102400, 60)).when(taskScheduler).getTotalResources();
+    doReturn(Resource.newInstance(102400, 60)).when(taskScheduler).getTotalResources(0);
     doReturn(historyEventHandler).when(appContext).getHistoryHandler();
     doReturn(dispatcher.getEventHandler()).when(appContext).getEventHandler();
 
@@ -2942,7 +2942,7 @@ public class TestVertexImpl {
     AMContainerMap containers = new AMContainerMap(
         mock(ContainerHeartbeatHandler.class), mock(TaskAttemptListener.class),
         new ContainerContextMatcher(), appContext);
-    containers.addContainerIfNew(container);
+    containers.addContainerIfNew(container, 0, 0, 0);
     doReturn(containers).when(appContext).getAllContainers();
 
     ta.handle(new TaskAttemptEventStartedRemotely(ta.getID(), contId, null));
@@ -2977,7 +2977,7 @@ public class TestVertexImpl {
     AMContainerMap containers = new AMContainerMap(
         mock(ContainerHeartbeatHandler.class), mock(TaskAttemptListener.class),
         new ContainerContextMatcher(), appContext);
-    containers.addContainerIfNew(container);
+    containers.addContainerIfNew(container, 0, 0, 0);
     doReturn(containers).when(appContext).getAllContainers();
 
     ta.handle(new TaskAttemptEventStartedRemotely(ta.getID(), contId, null));
@@ -3013,7 +3013,7 @@ public class TestVertexImpl {
     AMContainerMap containers = new AMContainerMap(
         mock(ContainerHeartbeatHandler.class), mock(TaskAttemptListener.class),
         new ContainerContextMatcher(), appContext);
-    containers.addContainerIfNew(container);
+    containers.addContainerIfNew(container, 0, 0, 0);
     doReturn(containers).when(appContext).getAllContainers();
 
     ta.handle(new TaskAttemptEventStartedRemotely(ta.getID(), contId, null));


[29/50] [abbrv] tez git commit: TEZ-2090. Add tests for jobs running in external services. (sseth)

Posted by ss...@apache.org.
http://git-wip-us.apache.org/repos/asf/tez/blob/8b40191d/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/ContainerRunnerImpl.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/ContainerRunnerImpl.java b/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/ContainerRunnerImpl.java
new file mode 100644
index 0000000..4a6ce33
--- /dev/null
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/ContainerRunnerImpl.java
@@ -0,0 +1,512 @@
+/*
+ * 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.
+ */
+
+package org.apache.tez.service.impl;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.nio.ByteBuffer;
+import java.security.PrivilegedExceptionAction;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+
+import com.google.common.base.Preconditions;
+import com.google.common.base.Stopwatch;
+import com.google.common.collect.HashMultimap;
+import com.google.common.collect.Multimap;
+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;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.DataInputBuffer;
+import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.security.Credentials;
+import org.apache.hadoop.security.SecurityUtil;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.service.AbstractService;
+import org.apache.hadoop.yarn.api.ApplicationConstants;
+import org.apache.hadoop.yarn.util.AuxiliaryServiceHelper;
+import org.apache.log4j.Logger;
+import org.apache.tez.common.TezCommonUtils;
+import org.apache.tez.common.TezTaskUmbilicalProtocol;
+import org.apache.tez.common.security.JobTokenIdentifier;
+import org.apache.tez.common.security.TokenCache;
+import org.apache.tez.dag.api.TezConfiguration;
+import org.apache.tez.dag.api.TezException;
+import org.apache.tez.runtime.task.TaskReporter;
+import org.apache.tez.runtime.task.TezTaskRunner;
+import org.apache.tez.service.ContainerRunner;
+import org.apache.tez.dag.api.TezConstants;
+import org.apache.tez.runtime.api.ExecutionContext;
+import org.apache.tez.runtime.api.impl.ExecutionContextImpl;
+import org.apache.tez.runtime.common.objectregistry.ObjectRegistryImpl;
+import org.apache.tez.runtime.task.TezChild;
+import org.apache.tez.runtime.task.TezChild.ContainerExecutionResult;
+import org.apache.tez.shufflehandler.ShuffleHandler;
+import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos.RunContainerRequestProto;
+import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos.SubmitWorkRequestProto;
+import org.apache.tez.util.ProtoConverters;
+
+public class ContainerRunnerImpl extends AbstractService implements ContainerRunner {
+
+  private static final Logger LOG = Logger.getLogger(ContainerRunnerImpl.class);
+
+  private final ListeningExecutorService executorService;
+  private final AtomicReference<InetSocketAddress> localAddress;
+  private final String[] localDirsBase;
+  private final Map<String, String> localEnv = new HashMap<String, String>();
+  private volatile FileSystem localFs;
+  private final long memoryPerExecutor;
+  // TODO Support for removing queued containers, interrupting / killing specific containers - when preemption is supported
+
+
+
+
+  public ContainerRunnerImpl(int numExecutors, String[] localDirsBase,
+                             AtomicReference<InetSocketAddress> localAddress,
+                             long totalMemoryAvailableBytes) {
+    super("ContainerRunnerImpl");
+    Preconditions.checkState(numExecutors > 0,
+        "Invalid number of executors: " + numExecutors + ". Must be > 0");
+    this.localDirsBase = localDirsBase;
+    this.localAddress = localAddress;
+
+    ExecutorService raw = Executors.newFixedThreadPool(numExecutors,
+        new ThreadFactoryBuilder().setNameFormat("ContainerExecutor %d").build());
+    this.executorService = MoreExecutors.listeningDecorator(raw);
+
+
+    // 80% of memory considered for accounted buffers. Rest for objects.
+    // TODO Tune this based on the available size.
+    this.memoryPerExecutor = (long)(totalMemoryAvailableBytes * 0.8 / (float) numExecutors);
+
+    LOG.info("ContainerRunnerImpl config: " +
+        "memoryPerExecutorDerived=" + memoryPerExecutor +
+        ", numExecutors=" + numExecutors
+    );
+  }
+
+  @Override
+  public void serviceInit(Configuration conf) {
+    try {
+      localFs = FileSystem.getLocal(conf);
+    } catch (IOException e) {
+      throw new RuntimeException("Failed to setup local filesystem instance", e);
+    }
+  }
+
+  @Override
+  public void serviceStart() {
+  }
+
+  public void setShufflePort(int shufflePort) {
+    AuxiliaryServiceHelper.setServiceDataIntoEnv(
+        TezConstants.TEZ_SHUFFLE_HANDLER_SERVICE_ID,
+        ByteBuffer.allocate(4).putInt(shufflePort), localEnv);
+  }
+
+  @Override
+  protected void serviceStop() throws Exception {
+    super.serviceStop();
+  }
+
+  // TODO Move this into a utilities class
+  private static String createAppSpecificLocalDir(String baseDir, String applicationIdString,
+                                                  String user) {
+    return baseDir + File.separator + "usercache" + File.separator + user + File.separator +
+        "appcache" + File.separator + applicationIdString;
+  }
+
+  /**
+   * Submit a container which is ready for running.
+   * The regular pull mechanism will be used to fetch work from the AM
+   * @param request
+   * @throws IOException
+   */
+  @Override
+  public void queueContainer(RunContainerRequestProto request) throws IOException {
+    LOG.info("Queuing container for execution: " + request);
+
+    Map<String, String> env = new HashMap<String, String>();
+    env.putAll(localEnv);
+    env.put(ApplicationConstants.Environment.USER.name(), request.getUser());
+
+    String[] localDirs = new String[localDirsBase.length];
+
+    // Setup up local dirs to be application specific, and create them.
+    for (int i = 0; i < localDirsBase.length; i++) {
+      localDirs[i] = createAppSpecificLocalDir(localDirsBase[i], request.getApplicationIdString(),
+          request.getUser());
+      localFs.mkdirs(new Path(localDirs[i]));
+    }
+    LOG.info("DEBUG: Dirs are: " + Arrays.toString(localDirs));
+
+
+    // Setup workingDir. This is otherwise setup as Environment.PWD
+    // Used for re-localization, to add the user specified configuration (conf_pb_binary_stream)
+    String workingDir = localDirs[0];
+
+    Credentials credentials = new Credentials();
+    DataInputBuffer dib = new DataInputBuffer();
+    byte[] tokenBytes = request.getCredentialsBinary().toByteArray();
+    dib.reset(tokenBytes, tokenBytes.length);
+    credentials.readTokenStorageStream(dib);
+
+    Token<JobTokenIdentifier> jobToken = TokenCache.getSessionToken(credentials);
+
+    // TODO Unregistering does not happen at the moment, since there's no signals on when an app completes.
+    LOG.info("DEBUG: Registering request with the ShuffleHandler");
+    ShuffleHandler.get().registerApplication(request.getApplicationIdString(), jobToken, request.getUser());
+
+
+    ContainerRunnerCallable callable = new ContainerRunnerCallable(request, new Configuration(getConfig()),
+        new ExecutionContextImpl(localAddress.get().getHostName()), env, localDirs,
+        workingDir, credentials, memoryPerExecutor);
+    ListenableFuture<ContainerExecutionResult> future = executorService
+        .submit(callable);
+    Futures.addCallback(future, new ContainerRunnerCallback(request, callable));
+  }
+
+  /**
+   * Submit an entire work unit - containerId + TaskSpec.
+   * This is intended for a task push from the AM
+   *
+   * @param request
+   * @throws IOException
+   */
+  @Override
+  public void submitWork(SubmitWorkRequestProto request) throws
+      IOException {
+    LOG.info("Queuing work for execution: " + request);
+
+    Map<String, String> env = new HashMap<String, String>();
+    env.putAll(localEnv);
+    env.put(ApplicationConstants.Environment.USER.name(), request.getUser());
+
+    String[] localDirs = new String[localDirsBase.length];
+
+    // Setup up local dirs to be application specific, and create them.
+    for (int i = 0; i < localDirsBase.length; i++) {
+      localDirs[i] = createAppSpecificLocalDir(localDirsBase[i], request.getApplicationIdString(),
+          request.getUser());
+      localFs.mkdirs(new Path(localDirs[i]));
+    }
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Dirs are: " + Arrays.toString(localDirs));
+    }
+
+    // Setup workingDir. This is otherwise setup as Environment.PWD
+    // Used for re-localization, to add the user specified configuration (conf_pb_binary_stream)
+    String workingDir = localDirs[0];
+
+    Credentials credentials = new Credentials();
+    DataInputBuffer dib = new DataInputBuffer();
+    byte[] tokenBytes = request.getCredentialsBinary().toByteArray();
+    dib.reset(tokenBytes, tokenBytes.length);
+    credentials.readTokenStorageStream(dib);
+
+    Token<JobTokenIdentifier> jobToken = TokenCache.getSessionToken(credentials);
+
+    // TODO Unregistering does not happen at the moment, since there's no signals on when an app completes.
+    LOG.info("DEBUG: Registering request with the ShuffleHandler");
+    ShuffleHandler.get().registerApplication(request.getApplicationIdString(), jobToken, request.getUser());
+    TaskRunnerCallable callable = new TaskRunnerCallable(request, new Configuration(getConfig()),
+        new ExecutionContextImpl(localAddress.get().getHostName()), env, localDirs,
+        workingDir, credentials, memoryPerExecutor);
+    ListenableFuture<ContainerExecutionResult> future = executorService.submit(callable);
+    Futures.addCallback(future, new TaskRunnerCallback(request, callable));
+  }
+
+
+  static class ContainerRunnerCallable implements Callable<ContainerExecutionResult> {
+
+    private final RunContainerRequestProto request;
+    private final Configuration conf;
+    private final String workingDir;
+    private final String[] localDirs;
+    private final Map<String, String> envMap;
+    private final String pid = null;
+    private final ObjectRegistryImpl objectRegistry;
+    private final ExecutionContext executionContext;
+    private final Credentials credentials;
+    private final long memoryAvailable;
+    private volatile TezChild tezChild;
+
+
+    ContainerRunnerCallable(RunContainerRequestProto request, Configuration conf,
+                            ExecutionContext executionContext, Map<String, String> envMap,
+                            String[] localDirs, String workingDir, Credentials credentials,
+                            long memoryAvailable) {
+      this.request = request;
+      this.conf = conf;
+      this.executionContext = executionContext;
+      this.envMap = envMap;
+      this.workingDir = workingDir;
+      this.localDirs = localDirs;
+      this.objectRegistry = new ObjectRegistryImpl();
+      this.credentials = credentials;
+      this.memoryAvailable = memoryAvailable;
+
+    }
+
+    @Override
+    public ContainerExecutionResult call() throws Exception {
+      Stopwatch sw = new Stopwatch().start();
+      tezChild =
+          new TezChild(conf, request.getAmHost(), request.getAmPort(),
+              request.getContainerIdString(),
+              request.getTokenIdentifier(), request.getAppAttemptNumber(), workingDir, localDirs,
+              envMap, objectRegistry, pid,
+              executionContext, credentials, memoryAvailable, request.getUser());
+      ContainerExecutionResult result = tezChild.run();
+      LOG.info("ExecutionTime for Container: " + request.getContainerIdString() + "=" +
+          sw.stop().elapsedMillis());
+      return result;
+    }
+
+    public TezChild getTezChild() {
+      return this.tezChild;
+    }
+  }
+
+
+  final class ContainerRunnerCallback implements FutureCallback<ContainerExecutionResult> {
+
+    private final RunContainerRequestProto request;
+    private final ContainerRunnerCallable containerRunnerCallable;
+
+    ContainerRunnerCallback(RunContainerRequestProto request,
+                            ContainerRunnerCallable containerRunnerCallable) {
+      this.request = request;
+      this.containerRunnerCallable = containerRunnerCallable;
+    }
+
+    // TODO Proper error handling
+    @Override
+    public void onSuccess(ContainerExecutionResult result) {
+      switch (result.getExitStatus()) {
+        case SUCCESS:
+          LOG.info("Successfully finished: " + request.getApplicationIdString() + ", containerId=" +
+              request.getContainerIdString());
+          break;
+        case EXECUTION_FAILURE:
+          LOG.info("Failed to run: " + request.getApplicationIdString() + ", containerId=" +
+              request.getContainerIdString(), result.getThrowable());
+          break;
+        case INTERRUPTED:
+          LOG.info(
+              "Interrupted while running: " + request.getApplicationIdString() + ", containerId=" +
+                  request.getContainerIdString(), result.getThrowable());
+          break;
+        case ASKED_TO_DIE:
+          LOG.info(
+              "Asked to die while running: " + request.getApplicationIdString() + ", containerId=" +
+                  request.getContainerIdString());
+          break;
+      }
+    }
+
+    @Override
+    public void onFailure(Throwable t) {
+      LOG.error(
+          "TezChild execution failed for : " + request.getApplicationIdString() + ", containerId=" +
+              request.getContainerIdString(), t);
+      TezChild tezChild = containerRunnerCallable.getTezChild();
+      if (tezChild != null) {
+        tezChild.shutdown();
+      }
+    }
+  }
+
+  static class TaskRunnerCallable implements Callable<ContainerExecutionResult> {
+
+    private final SubmitWorkRequestProto request;
+    private final Configuration conf;
+    private final String workingDir;
+    private final String[] localDirs;
+    private final Map<String, String> envMap;
+    private final String pid = null;
+    private final ObjectRegistryImpl objectRegistry;
+    private final ExecutionContext executionContext;
+    private final Credentials credentials;
+    private final long memoryAvailable;
+    private final ListeningExecutorService executor;
+    private volatile TezTaskRunner taskRunner;
+    private volatile TaskReporter taskReporter;
+    private TezTaskUmbilicalProtocol umbilical;
+
+
+    TaskRunnerCallable(SubmitWorkRequestProto request, Configuration conf,
+                             ExecutionContext executionContext, Map<String, String> envMap,
+                             String[] localDirs, String workingDir, Credentials credentials,
+                             long memoryAvailable) {
+      this.request = request;
+      this.conf = conf;
+      this.executionContext = executionContext;
+      this.envMap = envMap;
+      this.workingDir = workingDir;
+      this.localDirs = localDirs;
+      this.objectRegistry = new ObjectRegistryImpl();
+      this.credentials = credentials;
+      this.memoryAvailable = memoryAvailable;
+      // TODO This executor seems unnecessary. Here and TezChild
+      ExecutorService executorReal = Executors.newFixedThreadPool(1, new ThreadFactoryBuilder()
+          .setDaemon(true)
+          .setNameFormat("TezTaskRunner_" + request.getTaskSpec().getTaskAttemptIdString()).build());
+      executor = MoreExecutors.listeningDecorator(executorReal);
+    }
+
+    @Override
+    public ContainerExecutionResult call() throws Exception {
+
+      // TODO Consolidate this code with TezChild.
+      Stopwatch sw = new Stopwatch().start();
+      UserGroupInformation taskUgi = UserGroupInformation.createRemoteUser(request.getUser());
+      taskUgi.addCredentials(credentials);
+
+      Token<JobTokenIdentifier> jobToken = TokenCache.getSessionToken(credentials);
+      Map<String, ByteBuffer> serviceConsumerMetadata = new HashMap<String, ByteBuffer>();
+      serviceConsumerMetadata.put(TezConstants.TEZ_SHUFFLE_HANDLER_SERVICE_ID,
+          TezCommonUtils.convertJobTokenToBytes(jobToken));
+      Multimap<String, String> startedInputsMap = HashMultimap.create();
+
+      UserGroupInformation taskOwner =
+          UserGroupInformation.createRemoteUser(request.getTokenIdentifier());
+      final InetSocketAddress address =
+          NetUtils.createSocketAddrForHost(request.getAmHost(), request.getAmPort());
+      SecurityUtil.setTokenService(jobToken, address);
+      taskOwner.addToken(jobToken);
+      umbilical = taskOwner.doAs(new PrivilegedExceptionAction<TezTaskUmbilicalProtocol>() {
+        @Override
+        public TezTaskUmbilicalProtocol run() throws Exception {
+          return RPC.getProxy(TezTaskUmbilicalProtocol.class,
+              TezTaskUmbilicalProtocol.versionID, address, conf);
+        }
+      });
+      // TODO Stop reading this on each request.
+      taskReporter = new TaskReporter(
+          umbilical,
+          conf.getInt(TezConfiguration.TEZ_TASK_AM_HEARTBEAT_INTERVAL_MS,
+              TezConfiguration.TEZ_TASK_AM_HEARTBEAT_INTERVAL_MS_DEFAULT),
+          conf.getLong(
+              TezConfiguration.TEZ_TASK_AM_HEARTBEAT_COUNTER_INTERVAL_MS,
+              TezConfiguration.TEZ_TASK_AM_HEARTBEAT_COUNTER_INTERVAL_MS_DEFAULT),
+          conf.getInt(TezConfiguration.TEZ_TASK_MAX_EVENTS_PER_HEARTBEAT,
+              TezConfiguration.TEZ_TASK_MAX_EVENTS_PER_HEARTBEAT_DEFAULT),
+          new AtomicLong(0),
+          request.getContainerIdString());
+
+      taskRunner = new TezTaskRunner(conf, taskUgi, localDirs,
+          ProtoConverters.getTaskSpecfromProto(request.getTaskSpec()), umbilical,
+          request.getAppAttemptNumber(),
+          serviceConsumerMetadata, envMap, startedInputsMap, taskReporter, executor, objectRegistry,
+          pid,
+          executionContext, memoryAvailable);
+
+      boolean shouldDie;
+      try {
+        shouldDie = !taskRunner.run();
+        if (shouldDie) {
+          LOG.info("Got a shouldDie notification via hearbeats. Shutting down");
+          return new ContainerExecutionResult(ContainerExecutionResult.ExitStatus.SUCCESS, null,
+              "Asked to die by the AM");
+        }
+      } catch (IOException e) {
+        return new ContainerExecutionResult(ContainerExecutionResult.ExitStatus.EXECUTION_FAILURE,
+            e, "TaskExecutionFailure: " + e.getMessage());
+      } catch (TezException e) {
+        return new ContainerExecutionResult(ContainerExecutionResult.ExitStatus.EXECUTION_FAILURE,
+            e, "TaskExecutionFailure: " + e.getMessage());
+      } finally {
+        FileSystem.closeAllForUGI(taskUgi);
+      }
+      LOG.info("ExecutionTime for Container: " + request.getContainerIdString() + "=" +
+          sw.stop().elapsedMillis());
+      return new ContainerExecutionResult(ContainerExecutionResult.ExitStatus.SUCCESS, null,
+          null);
+    }
+
+    public void shutdown() {
+      executor.shutdownNow();
+      if (taskReporter != null) {
+        taskReporter.shutdown();
+      }
+      if (umbilical != null) {
+        RPC.stopProxy(umbilical);
+      }
+    }
+  }
+
+
+  final class TaskRunnerCallback implements FutureCallback<ContainerExecutionResult> {
+
+    private final SubmitWorkRequestProto request;
+    private final TaskRunnerCallable taskRunnerCallable;
+
+    TaskRunnerCallback(SubmitWorkRequestProto request,
+                            TaskRunnerCallable containerRunnerCallable) {
+      this.request = request;
+      this.taskRunnerCallable = containerRunnerCallable;
+    }
+
+    // TODO Proper error handling
+    @Override
+    public void onSuccess(ContainerExecutionResult result) {
+      switch (result.getExitStatus()) {
+        case SUCCESS:
+          LOG.info("Successfully finished: " + request.getApplicationIdString() + ", containerId=" +
+              request.getContainerIdString());
+          break;
+        case EXECUTION_FAILURE:
+          LOG.info("Failed to run: " + request.getApplicationIdString() + ", containerId=" +
+              request.getContainerIdString(), result.getThrowable());
+          break;
+        case INTERRUPTED:
+          LOG.info(
+              "Interrupted while running: " + request.getApplicationIdString() + ", containerId=" +
+                  request.getContainerIdString(), result.getThrowable());
+          break;
+        case ASKED_TO_DIE:
+          LOG.info(
+              "Asked to die while running: " + request.getApplicationIdString() + ", containerId=" +
+                  request.getContainerIdString());
+          break;
+      }
+      taskRunnerCallable.shutdown();
+    }
+
+    @Override
+    public void onFailure(Throwable t) {
+      LOG.error(
+          "TezTaskRunner execution failed for : " + request.getApplicationIdString() + ", containerId=" +
+              request.getContainerIdString(), t);
+      taskRunnerCallable.shutdown();
+    }
+  }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tez/blob/8b40191d/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/TezTestService.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/TezTestService.java b/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/TezTestService.java
new file mode 100644
index 0000000..012e352
--- /dev/null
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/TezTestService.java
@@ -0,0 +1,126 @@
+/*
+ * 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.
+ */
+
+package org.apache.tez.service.impl;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.Arrays;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicReference;
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.service.AbstractService;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.log4j.Logger;
+import org.apache.tez.service.ContainerRunner;
+import org.apache.tez.shufflehandler.ShuffleHandler;
+import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos;
+import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos.RunContainerRequestProto;
+
+public class TezTestService extends AbstractService implements ContainerRunner {
+
+  private static final Logger LOG = Logger.getLogger(TezTestService.class);
+
+  private final Configuration shuffleHandlerConf;
+  private final int numExecutors;
+
+  private final TezTestServiceProtocolServerImpl server;
+  private final ContainerRunnerImpl containerRunner;
+  private final String[] localDirs;
+
+  private final AtomicInteger numSubmissions = new AtomicInteger(0);
+
+
+  private final AtomicReference<InetSocketAddress> address = new AtomicReference<InetSocketAddress>();
+
+  public TezTestService(Configuration conf, int numExecutors, long memoryAvailable, String[] localDirs) {
+    super(TezTestService.class.getSimpleName());
+    this.numExecutors = numExecutors;
+    this.localDirs = localDirs;
+
+    long memoryAvailableBytes = memoryAvailable;
+    long jvmMax = Runtime.getRuntime().maxMemory();
+
+    LOG.info(TezTestService.class.getSimpleName() + " created with the following configuration: " +
+        "numExecutors=" + numExecutors +
+        ", workDirs=" + Arrays.toString(localDirs) +
+        ", memoryAvailable=" + memoryAvailable +
+        ", jvmMaxMemory=" + jvmMax);
+
+    Preconditions.checkArgument(this.numExecutors > 0);
+    Preconditions.checkArgument(this.localDirs != null && this.localDirs.length > 0,
+        "Work dirs must be specified");
+    Preconditions.checkState(jvmMax >= memoryAvailableBytes,
+        "Invalid configuration. Xmx value too small. maxAvailable=" + jvmMax + ", configured=" +
+            memoryAvailableBytes);
+
+    this.shuffleHandlerConf = new Configuration(conf);
+    // Start Shuffle on a random port
+    this.shuffleHandlerConf.setInt(ShuffleHandler.SHUFFLE_PORT_CONFIG_KEY, 0);
+    this.shuffleHandlerConf.set(ShuffleHandler.SHUFFLE_HANDLER_LOCAL_DIRS, StringUtils.arrayToString(localDirs));
+
+    this.server = new TezTestServiceProtocolServerImpl(this, address);
+    this.containerRunner = new ContainerRunnerImpl(numExecutors, localDirs, address,
+        memoryAvailableBytes);
+  }
+
+  @Override
+  public void serviceInit(Configuration conf) {
+    server.init(conf);
+    containerRunner.init(conf);
+  }
+
+  @Override
+  public void serviceStart() throws Exception {
+    ShuffleHandler.initializeAndStart(shuffleHandlerConf);
+    containerRunner.setShufflePort(ShuffleHandler.get().getPort());
+    server.start();
+    containerRunner.start();
+  }
+
+  public void serviceStop() throws Exception {
+    containerRunner.stop();
+    server.stop();
+    ShuffleHandler.get().stop();
+  }
+
+  public InetSocketAddress getListenerAddress() {
+    return server.getBindAddress();
+  }
+
+  public int getShufflePort() {
+    return ShuffleHandler.get().getPort();
+  }
+
+
+
+  @Override
+  public void queueContainer(RunContainerRequestProto request) throws IOException {
+    numSubmissions.incrementAndGet();
+    containerRunner.queueContainer(request);
+  }
+
+  @Override
+  public void submitWork(TezTestServiceProtocolProtos.SubmitWorkRequestProto request) throws
+      IOException {
+    numSubmissions.incrementAndGet();
+    containerRunner.submitWork(request);
+  }
+
+  public int getNumSubmissions() {
+    return numSubmissions.get();
+  }
+}

http://git-wip-us.apache.org/repos/asf/tez/blob/8b40191d/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/TezTestServiceProtocolClientImpl.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/TezTestServiceProtocolClientImpl.java b/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/TezTestServiceProtocolClientImpl.java
new file mode 100644
index 0000000..10d2952
--- /dev/null
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/TezTestServiceProtocolClientImpl.java
@@ -0,0 +1,82 @@
+/*
+ * 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.
+ */
+
+package org.apache.tez.service.impl;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+
+import com.google.protobuf.RpcController;
+import com.google.protobuf.ServiceException;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.ipc.ProtobufRpcEngine;
+import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.tez.service.TezTestServiceProtocolBlockingPB;
+import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos;
+import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos.RunContainerRequestProto;
+import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos.RunContainerResponseProto;
+
+
+public class TezTestServiceProtocolClientImpl implements TezTestServiceProtocolBlockingPB {
+
+  private final Configuration conf;
+  private final InetSocketAddress serverAddr;
+  TezTestServiceProtocolBlockingPB proxy;
+
+
+  public TezTestServiceProtocolClientImpl(Configuration conf, String hostname, int port) {
+    this.conf = conf;
+    this.serverAddr = NetUtils.createSocketAddr(hostname, port);
+  }
+
+  @Override
+  public RunContainerResponseProto runContainer(RpcController controller,
+                                                RunContainerRequestProto request) throws
+      ServiceException {
+    try {
+      return getProxy().runContainer(null, request);
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
+
+  @Override
+  public TezTestServiceProtocolProtos.SubmitWorkResponseProto submitWork(RpcController controller,
+                                                                         TezTestServiceProtocolProtos.SubmitWorkRequestProto request) throws
+      ServiceException {
+    try {
+      return getProxy().submitWork(null, request);
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
+
+
+  public TezTestServiceProtocolBlockingPB getProxy() throws IOException {
+    if (proxy == null) {
+      proxy = createProxy();
+    }
+    return proxy;
+  }
+
+  public TezTestServiceProtocolBlockingPB createProxy() throws IOException {
+    TezTestServiceProtocolBlockingPB p;
+    // TODO Fix security
+    RPC.setProtocolEngine(conf, TezTestServiceProtocolBlockingPB.class, ProtobufRpcEngine.class);
+    p = (TezTestServiceProtocolBlockingPB) RPC
+        .getProxy(TezTestServiceProtocolBlockingPB.class, 0, serverAddr, conf);
+    return p;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tez/blob/8b40191d/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/TezTestServiceProtocolServerImpl.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/TezTestServiceProtocolServerImpl.java b/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/TezTestServiceProtocolServerImpl.java
new file mode 100644
index 0000000..d7f8444
--- /dev/null
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/TezTestServiceProtocolServerImpl.java
@@ -0,0 +1,133 @@
+/*
+ * 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.
+ */
+
+package org.apache.tez.service.impl;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.concurrent.atomic.AtomicReference;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.protobuf.BlockingService;
+import com.google.protobuf.RpcController;
+import com.google.protobuf.ServiceException;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.ipc.ProtobufRpcEngine;
+import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.service.AbstractService;
+import org.apache.tez.service.ContainerRunner;
+import org.apache.tez.service.TezTestServiceProtocolBlockingPB;
+import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos;
+import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos.RunContainerRequestProto;
+import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos.RunContainerResponseProto;
+import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos.SubmitWorkResponseProto;
+
+public class TezTestServiceProtocolServerImpl extends AbstractService
+    implements TezTestServiceProtocolBlockingPB {
+
+  private static final Log LOG = LogFactory.getLog(TezTestServiceProtocolServerImpl.class);
+
+  private final ContainerRunner containerRunner;
+  private RPC.Server server;
+  private final AtomicReference<InetSocketAddress> bindAddress;
+
+
+  public TezTestServiceProtocolServerImpl(ContainerRunner containerRunner,
+                                          AtomicReference<InetSocketAddress> address) {
+    super(TezTestServiceProtocolServerImpl.class.getSimpleName());
+    this.containerRunner = containerRunner;
+    this.bindAddress = address;
+  }
+
+  @Override
+  public RunContainerResponseProto runContainer(RpcController controller,
+                                                RunContainerRequestProto request) throws
+      ServiceException {
+    LOG.info("Received request: " + request);
+    try {
+      containerRunner.queueContainer(request);
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+    return RunContainerResponseProto.getDefaultInstance();
+  }
+
+  @Override
+  public SubmitWorkResponseProto submitWork(RpcController controller, TezTestServiceProtocolProtos.SubmitWorkRequestProto request) throws
+      ServiceException {
+    LOG.info("Received submitWork request: " + request);
+    try {
+      containerRunner.submitWork(request);
+    } catch (IOException e) {
+      e.printStackTrace();
+    }
+    return SubmitWorkResponseProto.getDefaultInstance();
+  }
+
+
+  @Override
+  public void serviceStart() {
+    Configuration conf = getConfig();
+
+    int numHandlers = 3;
+    InetSocketAddress addr = new InetSocketAddress(0);
+
+    try {
+      server = createServer(TezTestServiceProtocolBlockingPB.class, addr, conf, numHandlers,
+          TezTestServiceProtocolProtos.TezTestServiceProtocol.newReflectiveBlockingService(this));
+      server.start();
+    } catch (IOException e) {
+      LOG.error("Failed to run RPC Server", e);
+      throw new RuntimeException(e);
+    }
+
+    InetSocketAddress serverBindAddress = NetUtils.getConnectAddress(server);
+    this.bindAddress.set(NetUtils.createSocketAddrForHost(
+        serverBindAddress.getAddress().getCanonicalHostName(),
+        serverBindAddress.getPort()));
+    LOG.info("Instantiated TestTestServiceListener at " + bindAddress);
+  }
+
+  @Override
+  public void serviceStop() {
+    if (server != null) {
+      server.stop();
+    }
+  }
+
+  @InterfaceAudience.Private
+  @VisibleForTesting
+  InetSocketAddress getBindAddress() {
+    return this.bindAddress.get();
+  }
+
+  private RPC.Server createServer(Class<?> pbProtocol, InetSocketAddress addr, Configuration conf,
+                                  int numHandlers, BlockingService blockingService) throws
+      IOException {
+    RPC.setProtocolEngine(conf, pbProtocol, ProtobufRpcEngine.class);
+    RPC.Server server = new RPC.Builder(conf)
+        .setProtocol(pbProtocol)
+        .setInstance(blockingService)
+        .setBindAddress(addr.getHostName())
+        .setPort(0)
+        .setNumHandlers(numHandlers)
+        .build();
+    // TODO Add security.
+    return server;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tez/blob/8b40191d/tez-ext-service-tests/src/test/java/org/apache/tez/shufflehandler/FadvisedChunkedFile.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/shufflehandler/FadvisedChunkedFile.java b/tez-ext-service-tests/src/test/java/org/apache/tez/shufflehandler/FadvisedChunkedFile.java
new file mode 100644
index 0000000..65588fe
--- /dev/null
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/shufflehandler/FadvisedChunkedFile.java
@@ -0,0 +1,78 @@
+/*
+ * 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.
+ */
+
+package org.apache.tez.shufflehandler;
+
+import java.io.FileDescriptor;
+import java.io.IOException;
+import java.io.RandomAccessFile;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.io.ReadaheadPool;
+import org.apache.hadoop.io.ReadaheadPool.ReadaheadRequest;
+import org.apache.hadoop.io.nativeio.NativeIO;
+import org.jboss.netty.handler.stream.ChunkedFile;
+
+public class FadvisedChunkedFile extends ChunkedFile {
+
+  private static final Log LOG = LogFactory.getLog(FadvisedChunkedFile.class);
+
+  private final boolean manageOsCache;
+  private final int readaheadLength;
+  private final ReadaheadPool readaheadPool;
+  private final FileDescriptor fd;
+  private final String identifier;
+
+  private ReadaheadRequest readaheadRequest;
+
+  public FadvisedChunkedFile(RandomAccessFile file, long position, long count,
+      int chunkSize, boolean manageOsCache, int readaheadLength,
+      ReadaheadPool readaheadPool, String identifier) throws IOException {
+    super(file, position, count, chunkSize);
+    this.manageOsCache = manageOsCache;
+    this.readaheadLength = readaheadLength;
+    this.readaheadPool = readaheadPool;
+    this.fd = file.getFD();
+    this.identifier = identifier;
+  }
+
+  @Override
+  public Object nextChunk() throws Exception {
+    if (manageOsCache && readaheadPool != null) {
+      readaheadRequest = readaheadPool
+          .readaheadStream(identifier, fd, getCurrentOffset(), readaheadLength,
+              getEndOffset(), readaheadRequest);
+    }
+    return super.nextChunk();
+  }
+
+  @Override
+  public void close() throws Exception {
+    if (readaheadRequest != null) {
+      readaheadRequest.cancel();
+    }
+    if (manageOsCache && getEndOffset() - getStartOffset() > 0) {
+      try {
+        NativeIO.POSIX.getCacheManipulator().posixFadviseIfPossible(identifier,
+            fd,
+            getStartOffset(), getEndOffset() - getStartOffset(),
+            NativeIO.POSIX.POSIX_FADV_DONTNEED);
+      } catch (Throwable t) {
+        LOG.warn("Failed to manage OS cache for " + identifier, t);
+      }
+    }
+    super.close();
+  }
+}

http://git-wip-us.apache.org/repos/asf/tez/blob/8b40191d/tez-ext-service-tests/src/test/java/org/apache/tez/shufflehandler/FadvisedFileRegion.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/shufflehandler/FadvisedFileRegion.java b/tez-ext-service-tests/src/test/java/org/apache/tez/shufflehandler/FadvisedFileRegion.java
new file mode 100644
index 0000000..bdffe52
--- /dev/null
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/shufflehandler/FadvisedFileRegion.java
@@ -0,0 +1,160 @@
+/*
+ * 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.
+ */
+
+package org.apache.tez.shufflehandler;
+
+import java.io.FileDescriptor;
+import java.io.IOException;
+import java.io.RandomAccessFile;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.channels.WritableByteChannel;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.io.ReadaheadPool;
+import org.apache.hadoop.io.ReadaheadPool.ReadaheadRequest;
+import org.apache.hadoop.io.nativeio.NativeIO;
+import org.jboss.netty.channel.DefaultFileRegion;
+
+public class FadvisedFileRegion extends DefaultFileRegion {
+
+  private static final Log LOG = LogFactory.getLog(FadvisedFileRegion.class);
+
+  private final boolean manageOsCache;
+  private final int readaheadLength;
+  private final ReadaheadPool readaheadPool;
+  private final FileDescriptor fd;
+  private final String identifier;
+  private final long count;
+  private final long position;
+  private final int shuffleBufferSize;
+  private final boolean shuffleTransferToAllowed;
+  private final FileChannel fileChannel;
+  
+  private ReadaheadRequest readaheadRequest;
+
+  public FadvisedFileRegion(RandomAccessFile file, long position, long count,
+      boolean manageOsCache, int readaheadLength, ReadaheadPool readaheadPool,
+      String identifier, int shuffleBufferSize, 
+      boolean shuffleTransferToAllowed) throws IOException {
+    super(file.getChannel(), position, count);
+    this.manageOsCache = manageOsCache;
+    this.readaheadLength = readaheadLength;
+    this.readaheadPool = readaheadPool;
+    this.fd = file.getFD();
+    this.identifier = identifier;
+    this.fileChannel = file.getChannel();
+    this.count = count;
+    this.position = position;
+    this.shuffleBufferSize = shuffleBufferSize;
+    this.shuffleTransferToAllowed = shuffleTransferToAllowed;
+  }
+
+  @Override
+  public long transferTo(WritableByteChannel target, long position)
+      throws IOException {
+    if (manageOsCache && readaheadPool != null) {
+      readaheadRequest = readaheadPool.readaheadStream(identifier, fd,
+          getPosition() + position, readaheadLength,
+          getPosition() + getCount(), readaheadRequest);
+    }
+    
+    if(this.shuffleTransferToAllowed) {
+      return super.transferTo(target, position);
+    } else {
+      return customShuffleTransfer(target, position);
+    } 
+  }
+
+  /**
+   * This method transfers data using local buffer. It transfers data from 
+   * a disk to a local buffer in memory, and then it transfers data from the 
+   * buffer to the target. This is used only if transferTo is disallowed in
+   * the configuration file. super.TransferTo does not perform well on Windows 
+   * due to a small IO request generated. customShuffleTransfer can control 
+   * the size of the IO requests by changing the size of the intermediate 
+   * buffer.
+   */
+  @VisibleForTesting
+  long customShuffleTransfer(WritableByteChannel target, long position)
+      throws IOException {
+    long actualCount = this.count - position;
+    if (actualCount < 0 || position < 0) {
+      throw new IllegalArgumentException(
+          "position out of range: " + position +
+          " (expected: 0 - " + (this.count - 1) + ')');
+    }
+    if (actualCount == 0) {
+      return 0L;
+    }
+    
+    long trans = actualCount;
+    int readSize;
+    ByteBuffer byteBuffer = ByteBuffer.allocate(this.shuffleBufferSize);
+    
+    while(trans > 0L &&
+        (readSize = fileChannel.read(byteBuffer, this.position+position)) > 0) {
+      //adjust counters and buffer limit
+      if(readSize < trans) {
+        trans -= readSize;
+        position += readSize;
+        byteBuffer.flip();
+      } else {
+        //We can read more than we need if the actualCount is not multiple 
+        //of the byteBuffer size and file is big enough. In that case we cannot
+        //use flip method but we need to set buffer limit manually to trans.
+        byteBuffer.limit((int)trans);
+        byteBuffer.position(0);
+        position += trans; 
+        trans = 0;
+      }
+      
+      //write data to the target
+      while(byteBuffer.hasRemaining()) {
+        target.write(byteBuffer);
+      }
+      
+      byteBuffer.clear();
+    }
+    
+    return actualCount - trans;
+  }
+
+  
+  @Override
+  public void releaseExternalResources() {
+    if (readaheadRequest != null) {
+      readaheadRequest.cancel();
+    }
+    super.releaseExternalResources();
+  }
+  
+  /**
+   * Call when the transfer completes successfully so we can advise the OS that
+   * we don't need the region to be cached anymore.
+   */
+  public void transferSuccessful() {
+    if (manageOsCache && getCount() > 0) {
+      try {
+        NativeIO.POSIX.getCacheManipulator().posixFadviseIfPossible(identifier,
+           fd, getPosition(), getCount(),
+           NativeIO.POSIX.POSIX_FADV_DONTNEED);
+      } catch (Throwable t) {
+        LOG.warn("Failed to manage OS cache for " + identifier, t);
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/tez/blob/8b40191d/tez-ext-service-tests/src/test/java/org/apache/tez/shufflehandler/IndexCache.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/shufflehandler/IndexCache.java b/tez-ext-service-tests/src/test/java/org/apache/tez/shufflehandler/IndexCache.java
new file mode 100644
index 0000000..9a51ca0
--- /dev/null
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/shufflehandler/IndexCache.java
@@ -0,0 +1,199 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.tez.shufflehandler;
+
+import java.io.IOException;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.LinkedBlockingQueue;
+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.Path;
+import org.apache.tez.runtime.library.common.Constants;
+import org.apache.tez.runtime.library.common.sort.impl.TezIndexRecord;
+import org.apache.tez.runtime.library.common.sort.impl.TezSpillRecord;
+
+class IndexCache {
+
+  private final Configuration conf;
+  private final int totalMemoryAllowed;
+  private AtomicInteger totalMemoryUsed = new AtomicInteger();
+  private static final Log LOG = LogFactory.getLog(IndexCache.class);
+
+  private final ConcurrentHashMap<String,IndexInformation> cache =
+      new ConcurrentHashMap<String,IndexInformation>();
+
+  private final LinkedBlockingQueue<String> queue =
+      new LinkedBlockingQueue<String>();
+
+  public IndexCache(Configuration conf) {
+    this.conf = conf;
+    totalMemoryAllowed = 10 * 1024 * 1024;
+    LOG.info("IndexCache created with max memory = " + totalMemoryAllowed);
+  }
+
+  /**
+   * This method gets the index information for the given mapId and reduce.
+   * It reads the index file into cache if it is not already present.
+   * @param mapId
+   * @param reduce
+   * @param fileName The file to read the index information from if it is not
+   *                 already present in the cache
+   * @param expectedIndexOwner The expected owner of the index file
+   * @return The Index Information
+   * @throws IOException
+   */
+  public TezIndexRecord getIndexInformation(String mapId, int reduce,
+                                         Path fileName, String expectedIndexOwner)
+      throws IOException {
+
+    IndexInformation info = cache.get(mapId);
+
+    if (info == null) {
+      info = readIndexFileToCache(fileName, mapId, expectedIndexOwner);
+    } else {
+      synchronized(info) {
+        while (isUnderConstruction(info)) {
+          try {
+            info.wait();
+          } catch (InterruptedException e) {
+            throw new IOException("Interrupted waiting for construction", e);
+          }
+        }
+      }
+      LOG.debug("IndexCache HIT: MapId " + mapId + " found");
+    }
+
+    if (info.mapSpillRecord.size() == 0 ||
+        info.mapSpillRecord.size() <= reduce) {
+      throw new IOException("Invalid request " +
+          " Map Id = " + mapId + " Reducer = " + reduce +
+          " Index Info Length = " + info.mapSpillRecord.size());
+    }
+    return info.mapSpillRecord.getIndex(reduce);
+  }
+
+  private boolean isUnderConstruction(IndexInformation info) {
+    synchronized(info) {
+      return (null == info.mapSpillRecord);
+    }
+  }
+
+  private IndexInformation readIndexFileToCache(Path indexFileName,
+                                                String mapId,
+                                                String expectedIndexOwner)
+      throws IOException {
+    IndexInformation info;
+    IndexInformation newInd = new IndexInformation();
+    if ((info = cache.putIfAbsent(mapId, newInd)) != null) {
+      synchronized(info) {
+        while (isUnderConstruction(info)) {
+          try {
+            info.wait();
+          } catch (InterruptedException e) {
+            throw new IOException("Interrupted waiting for construction", e);
+          }
+        }
+      }
+      LOG.debug("IndexCache HIT: MapId " + mapId + " found");
+      return info;
+    }
+    LOG.debug("IndexCache MISS: MapId " + mapId + " not found") ;
+    TezSpillRecord tmp = null;
+    try {
+      tmp = new TezSpillRecord(indexFileName, conf, expectedIndexOwner);
+    } catch (Throwable e) {
+      tmp = new TezSpillRecord(0);
+      cache.remove(mapId);
+      throw new IOException("Error Reading IndexFile", e);
+    } finally {
+      synchronized (newInd) {
+        newInd.mapSpillRecord = tmp;
+        newInd.notifyAll();
+      }
+    }
+    queue.add(mapId);
+
+    if (totalMemoryUsed.addAndGet(newInd.getSize()) > totalMemoryAllowed) {
+      freeIndexInformation();
+    }
+    return newInd;
+  }
+
+  /**
+   * This method removes the map from the cache if index information for this
+   * map is loaded(size>0), index information entry in cache will not be 
+   * removed if it is in the loading phrase(size=0), this prevents corruption  
+   * of totalMemoryUsed. It should be called when a map output on this tracker 
+   * is discarded.
+   * @param mapId The taskID of this map.
+   */
+  public void removeMap(String mapId) {
+    IndexInformation info = cache.get(mapId);
+    if (info == null || ((info != null) && isUnderConstruction(info))) {
+      return;
+    }
+    info = cache.remove(mapId);
+    if (info != null) {
+      totalMemoryUsed.addAndGet(-info.getSize());
+      if (!queue.remove(mapId)) {
+        LOG.warn("Map ID" + mapId + " not found in queue!!");
+      }
+    } else {
+      LOG.info("Map ID " + mapId + " not found in cache");
+    }
+  }
+
+  /**
+   * This method checks if cache and totolMemoryUsed is consistent.
+   * It is only used for unit test.
+   * @return True if cache and totolMemoryUsed is consistent
+   */
+  boolean checkTotalMemoryUsed() {
+    int totalSize = 0;
+    for (IndexInformation info : cache.values()) {
+      totalSize += info.getSize();
+    }
+    return totalSize == totalMemoryUsed.get();
+  }
+
+  /**
+   * Bring memory usage below totalMemoryAllowed.
+   */
+  private synchronized void freeIndexInformation() {
+    while (totalMemoryUsed.get() > totalMemoryAllowed) {
+      String s = queue.remove();
+      IndexInformation info = cache.remove(s);
+      if (info != null) {
+        totalMemoryUsed.addAndGet(-info.getSize());
+      }
+    }
+  }
+
+  private static class IndexInformation {
+    TezSpillRecord mapSpillRecord;
+
+    int getSize() {
+      return mapSpillRecord == null
+          ? 0
+          : mapSpillRecord.size() * Constants.MAP_OUTPUT_INDEX_RECORD_LENGTH;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/tez/blob/8b40191d/tez-ext-service-tests/src/test/java/org/apache/tez/shufflehandler/ShuffleHandler.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/shufflehandler/ShuffleHandler.java b/tez-ext-service-tests/src/test/java/org/apache/tez/shufflehandler/ShuffleHandler.java
new file mode 100644
index 0000000..cc82d74
--- /dev/null
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/shufflehandler/ShuffleHandler.java
@@ -0,0 +1,840 @@
+/*
+ * 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.
+ */
+
+package org.apache.tez.shufflehandler;
+
+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 javax.crypto.SecretKey;
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.RandomAccessFile;
+import java.net.InetSocketAddress;
+import java.net.URL;
+import java.nio.ByteBuffer;
+import java.nio.channels.ClosedChannelException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.regex.Pattern;
+
+import com.google.common.base.Charsets;
+import com.google.common.base.Preconditions;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.LocalDirAllocator;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.DataInputByteBuffer;
+import org.apache.hadoop.io.DataOutputBuffer;
+import org.apache.hadoop.io.ReadaheadPool;
+import org.apache.hadoop.io.SecureIOUtils;
+import org.apache.hadoop.metrics2.annotation.Metric;
+import org.apache.hadoop.metrics2.annotation.Metrics;
+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.util.Shell;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.util.ConverterUtils;
+import org.apache.tez.common.security.JobTokenIdentifier;
+import org.apache.tez.common.security.JobTokenSecretManager;
+import org.apache.tez.runtime.library.common.security.SecureShuffleUtils;
+import org.apache.tez.runtime.library.common.shuffle.orderedgrouped.ShuffleHeader;
+import org.apache.tez.runtime.library.common.sort.impl.TezIndexRecord;
+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.ChannelStateEvent;
+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.HttpHeaders;
+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.ChunkedWriteHandler;
+import org.jboss.netty.util.CharsetUtil;
+
+public class ShuffleHandler {
+
+  private static final Log LOG = LogFactory.getLog(ShuffleHandler.class);
+
+  public static final String SHUFFLE_HANDLER_LOCAL_DIRS = "tez.shuffle.handler.local-dirs";
+
+  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;
+  
+  // pattern to identify errors related to the client closing the socket early
+  // idea borrowed from Netty SslHandler
+  private static final Pattern IGNORABLE_ERROR_MESSAGE = Pattern.compile(
+      "^.*(?:connection.*reset|connection.*closed|broken.*pipe).*$",
+      Pattern.CASE_INSENSITIVE);
+
+  private int port;
+  private final ChannelFactory selector;
+  private final ChannelGroup accepted = new DefaultChannelGroup();
+  protected HttpPipelineFactory pipelineFact;
+  private final int sslFileBufferSize;
+  private final Configuration conf;
+
+  private final ConcurrentMap<String, Boolean> registeredApps = new ConcurrentHashMap<String, Boolean>();
+
+  /**
+   * Should the shuffle use posix_fadvise calls to manage the OS cache during
+   * sendfile
+   */
+  private final boolean manageOsCache;
+  private final int readaheadLength;
+  private final int maxShuffleConnections;
+  private final int shuffleBufferSize;
+  private final boolean shuffleTransferToAllowed;
+  private final ReadaheadPool readaheadPool = ReadaheadPool.getInstance();
+
+  private Map<String,String> userRsrc;
+  private JobTokenSecretManager secretManager;
+
+  // TODO Fix this for tez.
+  public static final String MAPREDUCE_SHUFFLE_SERVICEID =
+      "mapreduce_shuffle";
+
+  public static final String SHUFFLE_PORT_CONFIG_KEY = "tez.shuffle.port";
+  public static final int DEFAULT_SHUFFLE_PORT = 15551;
+
+  // TODO Change configs to remove mapreduce references.
+  public static final String SHUFFLE_CONNECTION_KEEP_ALIVE_ENABLED =
+      "mapreduce.shuffle.connection-keep-alive.enable";
+  public static final boolean DEFAULT_SHUFFLE_CONNECTION_KEEP_ALIVE_ENABLED = false;
+
+  public static final String SHUFFLE_CONNECTION_KEEP_ALIVE_TIME_OUT =
+      "mapreduce.shuffle.connection-keep-alive.timeout";
+  public static final int DEFAULT_SHUFFLE_CONNECTION_KEEP_ALIVE_TIME_OUT = 5; //seconds
+
+  public static final String SHUFFLE_MAPOUTPUT_META_INFO_CACHE_SIZE =
+      "mapreduce.shuffle.mapoutput-info.meta.cache.size";
+  public static final int DEFAULT_SHUFFLE_MAPOUTPUT_META_INFO_CACHE_SIZE =
+      1000;
+
+  public static final String CONNECTION_CLOSE = "close";
+
+  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;
+
+  public static final String MAX_SHUFFLE_CONNECTIONS = "mapreduce.shuffle.max.connections";
+  public static final int DEFAULT_MAX_SHUFFLE_CONNECTIONS = 0; // 0 implies no limit
+  
+  public static final String MAX_SHUFFLE_THREADS = "mapreduce.shuffle.max.threads";
+  // 0 implies Netty default of 2 * number of available processors
+  public static final int DEFAULT_MAX_SHUFFLE_THREADS = 0;
+  
+  public static final String SHUFFLE_BUFFER_SIZE = 
+      "mapreduce.shuffle.transfer.buffer.size";
+  public static final int DEFAULT_SHUFFLE_BUFFER_SIZE = 128 * 1024;
+  
+  public static final String  SHUFFLE_TRANSFERTO_ALLOWED = 
+      "mapreduce.shuffle.transferTo.allowed";
+  public static final boolean DEFAULT_SHUFFLE_TRANSFERTO_ALLOWED = true;
+  public static final boolean WINDOWS_DEFAULT_SHUFFLE_TRANSFERTO_ALLOWED = 
+      false;
+
+  final boolean connectionKeepAliveEnabled;
+  final int connectionKeepAliveTimeOut;
+  final int mapOutputMetaInfoCacheSize;
+  private static final AtomicBoolean started = new AtomicBoolean(false);
+  private static final AtomicBoolean initing = new AtomicBoolean(false);
+  private static ShuffleHandler INSTANCE;
+
+  @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();
+    }
+  }
+
+  public ShuffleHandler(Configuration conf) {
+    this.conf = conf;
+    manageOsCache = conf.getBoolean(SHUFFLE_MANAGE_OS_CACHE,
+        DEFAULT_SHUFFLE_MANAGE_OS_CACHE);
+
+    readaheadLength = conf.getInt(SHUFFLE_READAHEAD_BYTES,
+        DEFAULT_SHUFFLE_READAHEAD_BYTES);
+
+    maxShuffleConnections = conf.getInt(MAX_SHUFFLE_CONNECTIONS,
+        DEFAULT_MAX_SHUFFLE_CONNECTIONS);
+    int maxShuffleThreads = conf.getInt(MAX_SHUFFLE_THREADS,
+        DEFAULT_MAX_SHUFFLE_THREADS);
+    if (maxShuffleThreads == 0) {
+      maxShuffleThreads = 2 * Runtime.getRuntime().availableProcessors();
+    }
+
+    shuffleBufferSize = conf.getInt(SHUFFLE_BUFFER_SIZE,
+        DEFAULT_SHUFFLE_BUFFER_SIZE);
+
+    shuffleTransferToAllowed = conf.getBoolean(SHUFFLE_TRANSFERTO_ALLOWED,
+        (Shell.WINDOWS)?WINDOWS_DEFAULT_SHUFFLE_TRANSFERTO_ALLOWED:
+            DEFAULT_SHUFFLE_TRANSFERTO_ALLOWED);
+
+    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),
+        maxShuffleThreads);
+
+    sslFileBufferSize = conf.getInt(SUFFLE_SSL_FILE_BUFFER_SIZE_KEY,
+        DEFAULT_SUFFLE_SSL_FILE_BUFFER_SIZE);
+    connectionKeepAliveEnabled =
+        conf.getBoolean(SHUFFLE_CONNECTION_KEEP_ALIVE_ENABLED,
+            DEFAULT_SHUFFLE_CONNECTION_KEEP_ALIVE_ENABLED);
+    connectionKeepAliveTimeOut =
+        Math.max(1, conf.getInt(SHUFFLE_CONNECTION_KEEP_ALIVE_TIME_OUT,
+            DEFAULT_SHUFFLE_CONNECTION_KEEP_ALIVE_TIME_OUT));
+    mapOutputMetaInfoCacheSize =
+        Math.max(1, conf.getInt(SHUFFLE_MAPOUTPUT_META_INFO_CACHE_SIZE,
+            DEFAULT_SHUFFLE_MAPOUTPUT_META_INFO_CACHE_SIZE));
+
+    userRsrc = new ConcurrentHashMap<String,String>();
+    secretManager = new JobTokenSecretManager();
+  }
+
+
+  public void start() throws Exception {
+    ServerBootstrap bootstrap = new ServerBootstrap(selector);
+    try {
+      pipelineFact = new HttpPipelineFactory(conf);
+    } catch (Exception ex) {
+      throw new RuntimeException(ex);
+    }
+    bootstrap.setPipelineFactory(pipelineFact);
+    port = conf.getInt(SHUFFLE_PORT_CONFIG_KEY, DEFAULT_SHUFFLE_PORT);
+    Channel ch = bootstrap.bind(new InetSocketAddress(port));
+    accepted.add(ch);
+    port = ((InetSocketAddress)ch.getLocalAddress()).getPort();
+    conf.set(SHUFFLE_PORT_CONFIG_KEY, Integer.toString(port));
+    pipelineFact.SHUFFLE.setPort(port);
+    LOG.info("TezShuffleHandler" + " listening on port " + port);
+  }
+
+  public static void initializeAndStart(Configuration conf) throws Exception {
+    if (!initing.getAndSet(true)) {
+      INSTANCE = new ShuffleHandler(conf);
+      INSTANCE.start();
+      started.set(true);
+    }
+  }
+
+  public static ShuffleHandler get() {
+    Preconditions.checkState(started.get(), "ShuffleHandler must be started before invoking started");
+    return INSTANCE;
+  }
+
+  /**
+   * 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;
+  }
+
+  public int getPort() {
+    return port;
+  }
+
+  public void registerApplication(String applicationIdString, Token<JobTokenIdentifier> appToken,
+                                  String user) {
+    Boolean registered = registeredApps.putIfAbsent(applicationIdString, Boolean.valueOf(true));
+    if (registered == null) {
+      recordJobShuffleInfo(applicationIdString, user, appToken);
+    }
+  }
+
+  public void unregisterApplication(String applicationIdString) {
+    removeJobShuffleInfo(applicationIdString);
+  }
+
+
+  public void stop() throws Exception {
+    accepted.close().awaitUninterruptibly(10, TimeUnit.SECONDS);
+    if (selector != null) {
+      ServerBootstrap bootstrap = new ServerBootstrap(selector);
+      bootstrap.releaseExternalResources();
+    }
+    if (pipelineFact != null) {
+      pipelineFact.destroy();
+    }
+  }
+
+  protected Shuffle getShuffle(Configuration conf) {
+    return new Shuffle(conf);
+  }
+
+
+  private void addJobToken(String appIdString, String user,
+      Token<JobTokenIdentifier> jobToken) {
+    String jobIdString = appIdString.replace("application", "job");
+    userRsrc.put(jobIdString, user);
+    secretManager.addTokenForJob(jobIdString, jobToken);
+    LOG.info("Added token for " + jobIdString);
+  }
+
+  private void recordJobShuffleInfo(String appIdString, String user,
+      Token<JobTokenIdentifier> jobToken) {
+    addJobToken(appIdString, user, jobToken);
+  }
+
+  private void removeJobShuffleInfo(String appIdString) {
+    secretManager.removeTokenForJob(appIdString);
+    userRsrc.remove(appIdString);
+  }
+
+  class HttpPipelineFactory implements ChannelPipelineFactory {
+
+    final Shuffle SHUFFLE;
+    private SSLFactory sslFactory;
+
+    public HttpPipelineFactory(Configuration conf) throws Exception {
+      SHUFFLE = getShuffle(conf);
+      // TODO Setup SSL Shuffle
+//      if (conf.getBoolean(MRConfig.SHUFFLE_SSL_ENABLED_KEY,
+//                          MRConfig.SHUFFLE_SSL_ENABLED_DEFAULT)) {
+//        LOG.info("Encrypted shuffle is enabled.");
+//        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 final IndexCache indexCache;
+    private final LocalDirAllocator lDirAlloc =
+      new LocalDirAllocator(SHUFFLE_HANDLER_LOCAL_DIRS);
+    private int port;
+
+    public Shuffle(Configuration conf) {
+      this.conf = conf;
+      indexCache = new IndexCache(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 channelOpen(ChannelHandlerContext ctx, ChannelStateEvent evt) 
+        throws Exception {
+      if ((maxShuffleConnections > 0) && (accepted.size() >= maxShuffleConnections)) {
+        LOG.info(String.format("Current number of shuffle connections (%d) is " + 
+            "greater than or equal to the max allowed shuffle connections (%d)", 
+            accepted.size(), maxShuffleConnections));
+        evt.getChannel().close();
+        return;
+      }
+      accepted.add(evt.getChannel());
+      super.channelOpen(ctx, evt);
+     
+    }
+
+    @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> keepAliveList = q.get("keepAlive");
+      boolean keepAliveParam = false;
+      if (keepAliveList != null && keepAliveList.size() == 1) {
+        keepAliveParam = Boolean.valueOf(keepAliveList.get(0));
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("KeepAliveParam : " + keepAliveList
+            + " : " + keepAliveParam);
+        }
+      }
+      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 +
+            "\n  keepAlive: " + keepAliveParam);
+      }
+
+      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;
+      }
+
+      Map<String, MapOutputInfo> mapOutputInfoMap =
+          new HashMap<String, MapOutputInfo>();
+      Channel ch = evt.getChannel();
+      String user = userRsrc.get(jobId);
+
+      // $x/$user/appcache/$appId/output/$mapId
+      // TODO: Once Shuffle is out of NM, this can use MR APIs to convert
+      // between App and Job
+      String outputBasePathStr = getBaseLocation(jobId, user);
+
+      try {
+        populateHeaders(mapIds, outputBasePathStr, user, reduceId, request,
+          response, keepAliveParam, mapOutputInfoMap);
+      } catch(IOException e) {
+        ch.write(response);
+        LOG.error("Shuffle error in populating headers :", e);
+        String errorMessage = getErrorMessage(e);
+        sendError(ctx,errorMessage , INTERNAL_SERVER_ERROR);
+        return;
+      }
+      ch.write(response);
+      // TODO refactor the following into the pipeline
+      ChannelFuture lastMap = null;
+      for (String mapId : mapIds) {
+        try {
+          MapOutputInfo info = mapOutputInfoMap.get(mapId);
+          if (info == null) {
+            info = getMapOutputInfo(outputBasePathStr, mapId, reduceId, user);
+          }
+          lastMap =
+              sendMapOutput(ctx, ch, user, mapId,
+                reduceId, info);
+          if (null == lastMap) {
+            sendError(ctx, NOT_FOUND);
+            return;
+          }
+        } catch (IOException e) {
+          LOG.error("Shuffle error :", e);
+          String errorMessage = getErrorMessage(e);
+          sendError(ctx,errorMessage , INTERNAL_SERVER_ERROR);
+          return;
+        }
+      }
+      lastMap.addListener(ChannelFutureListener.CLOSE);
+    }
+
+    private String getErrorMessage(Throwable t) {
+      StringBuffer sb = new StringBuffer(t.getMessage());
+      while (t.getCause() != null) {
+        sb.append(t.getCause().getMessage());
+        t = t.getCause();
+      }
+      return sb.toString();
+    }
+
+    private final String USERCACHE_CONSTANT = "usercache";
+    private final String APPCACHE_CONSTANT = "appcache";
+
+    private String getBaseLocation(String jobIdString, String user) {
+      String parts[] = jobIdString.split("_");
+      Preconditions.checkArgument(parts.length == 3, "Invalid jobId. Expecting 3 parts");
+      final ApplicationId appID =
+          ApplicationId.newInstance(Long.parseLong(parts[1]), Integer.parseInt(parts[2]));
+      final String baseStr =
+          USERCACHE_CONSTANT + "/" + user + "/"
+              + APPCACHE_CONSTANT + "/"
+              + ConverterUtils.toString(appID) + "/output" + "/";
+      return baseStr;
+    }
+
+    protected MapOutputInfo getMapOutputInfo(String base, String mapId,
+        int reduce, String user) throws IOException {
+      // Index file
+      Path indexFileName =
+          lDirAlloc.getLocalPathToRead(base + "/file.out.index", conf);
+      TezIndexRecord info =
+          indexCache.getIndexInformation(mapId, reduce, indexFileName, user);
+
+      Path mapOutputFileName =
+          lDirAlloc.getLocalPathToRead(base + "/file.out", conf);
+      if (LOG.isDebugEnabled()) {
+        LOG.debug(base + " : " + mapOutputFileName + " : " + indexFileName);
+      }
+      MapOutputInfo outputInfo = new MapOutputInfo(mapOutputFileName, info);
+      return outputInfo;
+    }
+
+    protected void populateHeaders(List<String> mapIds, String outputBaseStr,
+        String user, int reduce, HttpRequest request, HttpResponse response,
+        boolean keepAliveParam, Map<String, MapOutputInfo> mapOutputInfoMap)
+        throws IOException {
+
+      long contentLength = 0;
+      for (String mapId : mapIds) {
+        String base = outputBaseStr + mapId;
+        MapOutputInfo outputInfo = getMapOutputInfo(base, mapId, reduce, user);
+        if (mapOutputInfoMap.size() < mapOutputMetaInfoCacheSize) {
+          mapOutputInfoMap.put(mapId, outputInfo);
+        }
+        // Index file
+        Path indexFileName =
+            lDirAlloc.getLocalPathToRead(base + "/file.out.index", conf);
+        TezIndexRecord info =
+            indexCache.getIndexInformation(mapId, reduce, indexFileName, user);
+        ShuffleHeader header =
+            new ShuffleHeader(mapId, info.getPartLength(), info.getRawLength(), reduce);
+        DataOutputBuffer dob = new DataOutputBuffer();
+        header.write(dob);
+
+        contentLength += info.getPartLength();
+        contentLength += dob.getLength();
+      }
+
+      // Now set the response headers.
+      setResponseHeaders(response, keepAliveParam, contentLength);
+    }
+
+    protected void setResponseHeaders(HttpResponse response,
+        boolean keepAliveParam, long contentLength) {
+      if (!connectionKeepAliveEnabled && !keepAliveParam) {
+        LOG.info("Setting connection close header...");
+        response.setHeader(HttpHeaders.Names.CONNECTION, CONNECTION_CLOSE);
+      } else {
+        response.setHeader(HttpHeaders.Names.CONTENT_LENGTH,
+          String.valueOf(contentLength));
+        response.setHeader(HttpHeaders.Names.CONNECTION, HttpHeaders.Values.KEEP_ALIVE);
+        response.setHeader(HttpHeaders.Values.KEEP_ALIVE, "timeout="
+            + connectionKeepAliveTimeOut);
+        LOG.info("Content Length in shuffle : " + contentLength);
+      }
+    }
+
+    class MapOutputInfo {
+      final Path mapOutputFileName;
+      final TezIndexRecord indexRecord;
+
+      MapOutputInfo(Path mapOutputFileName, TezIndexRecord indexRecord) {
+        this.mapOutputFileName = mapOutputFileName;
+        this.indexRecord = indexRecord;
+      }
+    }
+
+    protected void verifyRequest(String appid, ChannelHandlerContext ctx,
+        HttpRequest request, HttpResponse response, URL requestUri)
+        throws IOException {
+      SecretKey tokenSecret = secretManager.retrieveTokenSecret(appid);
+      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(Charsets.UTF_8), 
+            tokenSecret);
+      response.setHeader(SecureShuffleUtils.HTTP_HEADER_REPLY_URL_HASH, reply);
+      // 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);
+      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 mapId, int reduce, MapOutputInfo mapOutputInfo)
+        throws IOException {
+      final TezIndexRecord info = mapOutputInfo.indexRecord;
+      final ShuffleHeader header =
+        new ShuffleHeader(mapId, info.getPartLength(), info.getRawLength(), reduce);
+      final DataOutputBuffer dob = new DataOutputBuffer();
+      header.write(dob);
+      ch.write(wrappedBuffer(dob.getData(), 0, dob.getLength()));
+      final File spillfile =
+          new File(mapOutputInfo.mapOutputFileName.toString());
+      RandomAccessFile spill;
+      try {
+        spill = SecureIOUtils.openForRandomRead(spillfile, "r", user, null);
+      } catch (FileNotFoundException e) {
+        LOG.info(spillfile + " not found");
+        return null;
+      }
+      ChannelFuture writeFuture;
+      if (ch.getPipeline().get(SslHandler.class) == null) {
+        final FadvisedFileRegion partition = new FadvisedFileRegion(spill,
+            info.getStartOffset(), info.getPartLength(), manageOsCache, readaheadLength,
+            readaheadPool, spillfile.getAbsolutePath(), 
+            shuffleBufferSize, shuffleTransferToAllowed);
+        writeFuture = ch.write(partition);
+        writeFuture.addListener(new ChannelFutureListener() {
+            // TODO error handling; distinguish IO/connection failures,
+            //      attribute to appropriate spill output
+          @Override
+          public void operationComplete(ChannelFuture future) {
+            if (future.isSuccess()) {
+              partition.transferSuccessful();
+            }
+            partition.releaseExternalResources();
+          }
+        });
+      } else {
+        // HTTPS cannot be done with zero copy.
+        final FadvisedChunkedFile chunk = new FadvisedChunkedFile(spill,
+            info.getStartOffset(), info.getPartLength(), sslFileBufferSize,
+            manageOsCache, readaheadLength, readaheadPool,
+            spillfile.getAbsolutePath());
+        writeFuture = ch.write(chunk);
+      }
+      return writeFuture;
+    }
+
+    protected void sendError(ChannelHandlerContext ctx,
+        HttpResponseStatus status) {
+      sendError(ctx, "", status);
+    }
+
+    protected 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");
+      // 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);
+      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);
+    }
+
+    @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;
+      } else if (cause instanceof IOException) {
+        if (cause instanceof ClosedChannelException) {
+          LOG.debug("Ignoring closed channel error", cause);
+          return;
+        }
+        String message = String.valueOf(cause.getMessage());
+        if (IGNORABLE_ERROR_MESSAGE.matcher(message).matches()) {
+          LOG.debug("Ignoring client socket close", cause);
+          return;
+        }
+      }
+
+      LOG.error("Shuffle error: ", cause);
+      if (ch.isConnected()) {
+        LOG.error("Shuffle error " + e);
+        sendError(ctx, INTERNAL_SERVER_ERROR);
+      }
+    }
+  }
+}


[46/50] [abbrv] tez git commit: TEZ-2284. Separate TaskReporter into an interface. (sseth)

Posted by ss...@apache.org.
TEZ-2284. Separate TaskReporter into an interface. (sseth)


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

Branch: refs/heads/TEZ-2003
Commit: 854a43972d6dc1895423b4dfcb07a625d76b3db4
Parents: d99aae5
Author: Siddharth Seth <ss...@apache.org>
Authored: Tue Apr 7 13:21:35 2015 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Wed May 6 00:18:08 2015 -0700

----------------------------------------------------------------------
 TEZ-2003-CHANGES.txt                            |  1 +
 .../internals/api/TaskReporterInterface.java    | 46 ++++++++++++++++++++
 .../apache/tez/runtime/task/TaskReporter.java   | 12 ++++-
 .../org/apache/tez/runtime/task/TezChild.java   |  3 +-
 .../apache/tez/runtime/task/TezTaskRunner.java  |  5 ++-
 5 files changed, 62 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/854a4397/TEZ-2003-CHANGES.txt
----------------------------------------------------------------------
diff --git a/TEZ-2003-CHANGES.txt b/TEZ-2003-CHANGES.txt
index 6a4399c..e2c428d 100644
--- a/TEZ-2003-CHANGES.txt
+++ b/TEZ-2003-CHANGES.txt
@@ -13,5 +13,6 @@ ALL CHANGES:
   TEZ-2187. Allow TaskCommunicators to report failed / killed attempts.
   TEZ-2241. Miscellaneous fixes after last reabse.
   TEZ-2283. Fixes after rebase 04/07.
+  TEZ-2284. Separate TaskReporter into an interface.
 
 INCOMPATIBLE CHANGES:

http://git-wip-us.apache.org/repos/asf/tez/blob/854a4397/tez-runtime-internals/src/main/java/org/apache/tez/runtime/internals/api/TaskReporterInterface.java
----------------------------------------------------------------------
diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/internals/api/TaskReporterInterface.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/internals/api/TaskReporterInterface.java
new file mode 100644
index 0000000..47a61ab
--- /dev/null
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/internals/api/TaskReporterInterface.java
@@ -0,0 +1,46 @@
+/*
+ * 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.
+ */
+
+package org.apache.tez.runtime.internals.api;
+
+import java.io.IOException;
+import java.util.Collection;
+
+import org.apache.tez.dag.api.TezException;
+import org.apache.tez.dag.records.TezTaskAttemptID;
+import org.apache.tez.runtime.RuntimeTask;
+import org.apache.tez.runtime.api.impl.EventMetaData;
+import org.apache.tez.runtime.api.impl.TezEvent;
+import org.apache.tez.runtime.task.ErrorReporter;
+
+public interface TaskReporterInterface {
+
+  // TODO TEZ-2003 Consolidate private API usage if making this public
+
+  void registerTask(RuntimeTask task, ErrorReporter errorReporter);
+
+  void unregisterTask(TezTaskAttemptID taskAttemptId);
+
+  boolean taskSucceeded(TezTaskAttemptID taskAttemptId) throws IOException, TezException;
+
+  boolean taskFailed(TezTaskAttemptID taskAttemptId, Throwable cause, String diagnostics, EventMetaData srcMeta) throws IOException,
+      TezException;
+
+  void addEvents(TezTaskAttemptID taskAttemptId, Collection<TezEvent> events);
+
+  boolean canCommit(TezTaskAttemptID taskAttemptId) throws IOException;
+
+  void shutdown();
+
+}

http://git-wip-us.apache.org/repos/asf/tez/blob/854a4397/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TaskReporter.java
----------------------------------------------------------------------
diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TaskReporter.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TaskReporter.java
index 7324abd..98aa55a 100644
--- a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TaskReporter.java
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TaskReporter.java
@@ -47,6 +47,7 @@ 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.EventMetaData.EventProducerConsumerType;
+import org.apache.tez.runtime.internals.api.TaskReporterInterface;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -65,7 +66,7 @@ import com.google.common.util.concurrent.ThreadFactoryBuilder;
  * retrieve events specific to this task.
  * 
  */
-public class TaskReporter {
+public class TaskReporter implements TaskReporterInterface {
 
   private static final Logger LOG = LoggerFactory.getLogger(TaskReporter.class);
 
@@ -97,6 +98,7 @@ public class TaskReporter {
   /**
    * Register a task to be tracked. Heartbeats will be sent out for this task to fetch events, etc.
    */
+  @Override
   public synchronized void registerTask(RuntimeTask task,
       ErrorReporter errorReporter) {
     currentCallable = new HeartbeatCallable(task, umbilical, pollInterval, sendCounterInterval,
@@ -109,11 +111,13 @@ public class TaskReporter {
    * This method should always be invoked before setting up heartbeats for another task running in
    * the same container.
    */
+  @Override
   public synchronized void unregisterTask(TezTaskAttemptID taskAttemptID) {
     currentCallable.markComplete();
     currentCallable = null;
   }
-  
+
+  @Override
   public void shutdown() {
     heartbeatExecutor.shutdownNow();
   }
@@ -388,19 +392,23 @@ public class TaskReporter {
     }
   }
 
+  @Override
   public synchronized boolean taskSucceeded(TezTaskAttemptID taskAttemptID) throws IOException, TezException {
     return currentCallable.taskSucceeded(taskAttemptID);
   }
 
+  @Override
   public synchronized boolean taskFailed(TezTaskAttemptID taskAttemptID, Throwable t, String diagnostics,
       EventMetaData srcMeta) throws IOException, TezException {
     return currentCallable.taskFailed(taskAttemptID, t, diagnostics, srcMeta);
   }
 
+  @Override
   public synchronized void addEvents(TezTaskAttemptID taskAttemptID, Collection<TezEvent> events) {
     currentCallable.addEvents(taskAttemptID, events);
   }
 
+  @Override
   public boolean canCommit(TezTaskAttemptID taskAttemptID) throws IOException {
     return umbilical.canCommit(taskAttemptID);
   }

http://git-wip-us.apache.org/repos/asf/tez/blob/854a4397/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezChild.java
----------------------------------------------------------------------
diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezChild.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezChild.java
index 7615f08..c4fd64c 100644
--- a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezChild.java
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezChild.java
@@ -66,6 +66,7 @@ import org.apache.tez.runtime.api.impl.ExecutionContextImpl;
 import org.apache.tez.runtime.api.impl.TaskSpec;
 import org.apache.tez.runtime.api.impl.TezUmbilical;
 import org.apache.tez.runtime.common.objectregistry.ObjectRegistryImpl;
+import org.apache.tez.runtime.internals.api.TaskReporterInterface;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -111,7 +112,7 @@ public class TezChild {
   private final boolean ownUmbilical;
 
   private final TezTaskUmbilicalProtocol umbilical;
-  private TaskReporter taskReporter;
+  private TaskReporterInterface taskReporter;
   private int taskCount = 0;
   private TezVertexID lastVertexID;
 

http://git-wip-us.apache.org/repos/asf/tez/blob/854a4397/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezTaskRunner.java
----------------------------------------------------------------------
diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezTaskRunner.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezTaskRunner.java
index f54814b..33a7f4a 100644
--- a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezTaskRunner.java
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezTaskRunner.java
@@ -41,6 +41,7 @@ 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.apache.tez.runtime.api.impl.TezUmbilical;
+import org.apache.tez.runtime.internals.api.TaskReporterInterface;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -56,7 +57,7 @@ public class TezTaskRunner implements TezUmbilical, ErrorReporter {
   private final LogicalIOProcessorRuntimeTask task;
   private final UserGroupInformation ugi;
 
-  private final TaskReporter taskReporter;
+  private final TaskReporterInterface taskReporter;
   private final ListeningExecutorService executor;
   private volatile ListenableFuture<Void> taskFuture;
   private volatile Thread waitingThread;
@@ -70,7 +71,7 @@ public class TezTaskRunner implements TezUmbilical, ErrorReporter {
   public TezTaskRunner(Configuration tezConf, UserGroupInformation ugi, String[] localDirs,
       TaskSpec taskSpec, int appAttemptNumber,
       Map<String, ByteBuffer> serviceConsumerMetadata, Map<String, String> serviceProviderEnvMap,
-      Multimap<String, String> startedInputsMap, TaskReporter taskReporter,
+      Multimap<String, String> startedInputsMap, TaskReporterInterface taskReporter,
       ListeningExecutorService executor, ObjectRegistry objectRegistry, String pid,
       ExecutionContext executionContext, long memAvailable)
           throws IOException {


[04/50] [abbrv] tez git commit: TEZ-1988. Tez UI: does not work when using file:// in a browser (pramachandran)

Posted by ss...@apache.org.
TEZ-1988. Tez UI: does not work when using file:// in a browser (pramachandran)


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

Branch: refs/heads/TEZ-2003
Commit: 96efae01f5e7e2847e78a0b7a7a2dc458e3ff9bc
Parents: a02a5ea
Author: Prakash Ramachandran <pr...@hortonworks.com>
Authored: Fri May 1 13:26:46 2015 +0530
Committer: Prakash Ramachandran <pr...@hortonworks.com>
Committed: Fri May 1 13:26:46 2015 +0530

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../main/webapp/app/scripts/default-configs.js  | 23 +++++++++++++-------
 2 files changed, 16 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/96efae01/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 5a5c21f..e7079bf 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -155,6 +155,7 @@ Release 0.6.1: Unreleased
 INCOMPATIBLE CHANGES
 
 ALL CHANGES:
+  TEZ-1988. Tez UI: does not work when using file:// in a browser
   TEZ-2390. tez-tools swimlane tool fails to parse large jobs >8K containers
   TEZ-2256. Avoid use of BufferTooSmallException to signal end of buffer in UnorderedPartitionedKVWriter
   TEZ-2380. Disable fall back to reading from timeline if timeline disabled.

http://git-wip-us.apache.org/repos/asf/tez/blob/96efae01/tez-ui/src/main/webapp/app/scripts/default-configs.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/scripts/default-configs.js b/tez-ui/src/main/webapp/app/scripts/default-configs.js
index e438bad..d0ffa97 100644
--- a/tez-ui/src/main/webapp/app/scripts/default-configs.js
+++ b/tez-ui/src/main/webapp/app/scripts/default-configs.js
@@ -16,18 +16,25 @@
  * limitations under the License.
  */
 
+function constructDefaultUrl(location, port) {
+  var protocol,
+      hostname;
+  if (App.env.isStandalone && location.protocol != 'file:') {
+    protocol = location.protocol;
+    hostname = location.hostname;
+  } else {
+    protocol = 'http:';
+    hostname = 'localhost';
+  }
+  return '%@//%@:%@'.fmt(protocol, hostname, port);
+}
+
 var getDefaultTimelineUrl = function() {
-  var location = window.location;
-  var protocol = App.env.isStandalone ? location.protocol : 'http:';
-  var hostname = App.env.isStandalone ? location.hostname : 'localhost';
-  return '%@//%@:8188'.fmt(protocol, hostname);
+  return constructDefaultUrl(window.location, 8188);
 };
 
 var getDefaultRMWebUrl = function() {
-  var location = window.location;
-  var protocol = App.env.isStandalone ? location.protocol : 'http:';
-  var hostname = App.env.isStandalone ? location.hostname : 'localhost';
-  return '%@//%@:8088'.fmt(protocol, hostname);
+  return constructDefaultUrl(window.location, 8088);
 };
 
 $.extend(true, App.Configs, {


[36/50] [abbrv] tez git commit: TEZ-2123. Fix component managers to use pluggable components. Enable hybrid mode. (sseth)

Posted by ss...@apache.org.
http://git-wip-us.apache.org/repos/asf/tez/blob/25980c1a/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 54b9adb..c1169ef 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
@@ -223,7 +223,7 @@ public class TestContainerReuse {
 
     taskSchedulerEventHandler.handleEvent(
       new AMSchedulerEventTAEnded(
-        ta11, containerHost1.getId(), TaskAttemptState.SUCCEEDED));
+        ta11, containerHost1.getId(), TaskAttemptState.SUCCEEDED, 0));
     drainableAppCallback.drain();
     verify(taskScheduler).deallocateTask(eq(ta11), eq(true));
     verify(taskSchedulerEventHandler, times(1)).taskAllocated(
@@ -235,7 +235,7 @@ public class TestContainerReuse {
 
     taskSchedulerEventHandler.handleEvent(
       new AMSchedulerEventTAEnded(ta21, containerHost2.getId(),
-        TaskAttemptState.SUCCEEDED));
+        TaskAttemptState.SUCCEEDED, 0));
 
     long currentTs = System.currentTimeMillis();
     Throwable exception = null;
@@ -356,7 +356,7 @@ public class TestContainerReuse {
 
     taskSchedulerEventHandler.handleEvent(
       new AMSchedulerEventTAEnded(ta21, containerHost2.getId(),
-        TaskAttemptState.SUCCEEDED));
+        TaskAttemptState.SUCCEEDED, 0));
     drainableAppCallback.drain();
     verify(taskScheduler).deallocateTask(eq(ta21), eq(true));
     verify(taskSchedulerEventHandler, times(0)).taskAllocated(
@@ -459,7 +459,7 @@ public class TestContainerReuse {
     verify(taskSchedulerEventHandler).taskAllocated(eq(ta11), any(Object.class), eq(container1));
 
     // Task assigned to container completed successfully. Container should be re-used.
-    taskSchedulerEventHandler.handleEvent(new AMSchedulerEventTAEnded(ta11, container1.getId(), TaskAttemptState.SUCCEEDED));
+    taskSchedulerEventHandler.handleEvent(new AMSchedulerEventTAEnded(ta11, container1.getId(), TaskAttemptState.SUCCEEDED, 0));
     drainableAppCallback.drain();
     verify(taskScheduler).deallocateTask(eq(ta11), eq(true));
     verify(taskSchedulerEventHandler).taskAllocated(eq(ta12), any(Object.class), eq(container1));
@@ -469,7 +469,7 @@ public class TestContainerReuse {
 
     // Task assigned to container completed successfully.
     // Verify reuse across hosts.
-    taskSchedulerEventHandler.handleEvent(new AMSchedulerEventTAEnded(ta12, container1.getId(), TaskAttemptState.SUCCEEDED));
+    taskSchedulerEventHandler.handleEvent(new AMSchedulerEventTAEnded(ta12, container1.getId(), TaskAttemptState.SUCCEEDED, 0));
     drainableAppCallback.drain();
     verify(taskScheduler).deallocateTask(eq(ta12), eq(true));
     verify(taskSchedulerEventHandler).taskAllocated(eq(ta13), any(Object.class), eq(container1));
@@ -478,7 +478,7 @@ public class TestContainerReuse {
     eventHandler.reset();
 
     // Verify no re-use if a previous task fails.
-    taskSchedulerEventHandler.handleEvent(new AMSchedulerEventTAEnded(ta13, container1.getId(), TaskAttemptState.FAILED));
+    taskSchedulerEventHandler.handleEvent(new AMSchedulerEventTAEnded(ta13, container1.getId(), TaskAttemptState.FAILED, 0));
     drainableAppCallback.drain();
     verify(taskSchedulerEventHandler, times(0)).taskAllocated(eq(ta14), any(Object.class), eq(container1));
     verify(taskScheduler).deallocateTask(eq(ta13), eq(false));
@@ -496,7 +496,7 @@ public class TestContainerReuse {
     verify(taskSchedulerEventHandler).taskAllocated(eq(ta14), any(Object.class), eq(container2));
 
     // Task assigned to container completed successfully. No pending requests. Container should be released.
-    taskSchedulerEventHandler.handleEvent(new AMSchedulerEventTAEnded(ta14, container2.getId(), TaskAttemptState.SUCCEEDED));
+    taskSchedulerEventHandler.handleEvent(new AMSchedulerEventTAEnded(ta14, container2.getId(), TaskAttemptState.SUCCEEDED, 0));
     drainableAppCallback.drain();
     verify(taskScheduler).deallocateTask(eq(ta14), eq(true));
     verify(rmClient).releaseAssignedContainer(eq(container2.getId()));
@@ -607,7 +607,7 @@ public class TestContainerReuse {
 
     // First task had profiling on. This container can not be reused further.
     taskSchedulerEventHandler.handleEvent(
-        new AMSchedulerEventTAEnded(ta11, container1.getId(), TaskAttemptState.SUCCEEDED));
+        new AMSchedulerEventTAEnded(ta11, container1.getId(), TaskAttemptState.SUCCEEDED, 0));
     drainableAppCallback.drain();
     verify(taskScheduler).deallocateTask(eq(ta11), eq(true));
     verify(taskSchedulerEventHandler, times(0)).taskAllocated(eq(ta12), any(Object.class),
@@ -653,7 +653,7 @@ public class TestContainerReuse {
 
     // Verify that the container can not be reused when profiling option is turned on
     // Even for 2 tasks having same profiling option can have container reusability.
-    taskSchedulerEventHandler.handleEvent(new AMSchedulerEventTAEnded(ta13, container2.getId(), TaskAttemptState.SUCCEEDED));
+    taskSchedulerEventHandler.handleEvent(new AMSchedulerEventTAEnded(ta13, container2.getId(), TaskAttemptState.SUCCEEDED, 0));
     drainableAppCallback.drain();
     verify(taskScheduler).deallocateTask(eq(ta13), eq(true));
     verify(taskSchedulerEventHandler, times(0)).taskAllocated(eq(ta14), any(Object.class),
@@ -698,7 +698,7 @@ public class TestContainerReuse {
     verify(taskSchedulerEventHandler).taskAllocated(eq(ta15), any(Object.class), eq(container3));
 
     //Ensure task 6 (of vertex 1) is allocated to same container
-    taskSchedulerEventHandler.handleEvent(new AMSchedulerEventTAEnded(ta15, container3.getId(), TaskAttemptState.SUCCEEDED));
+    taskSchedulerEventHandler.handleEvent(new AMSchedulerEventTAEnded(ta15, container3.getId(), TaskAttemptState.SUCCEEDED, 0));
     drainableAppCallback.drain();
     verify(taskScheduler).deallocateTask(eq(ta15), eq(true));
     verify(taskSchedulerEventHandler).taskAllocated(eq(ta16), any(Object.class), eq(container3));
@@ -811,7 +811,7 @@ public class TestContainerReuse {
     // until delay expires.
     taskSchedulerEventHandler.handleEvent(
       new AMSchedulerEventTAEnded(ta11, container1.getId(),
-        TaskAttemptState.SUCCEEDED));
+        TaskAttemptState.SUCCEEDED, 0));
     drainableAppCallback.drain();
     verify(taskScheduler).deallocateTask(eq(ta11), eq(true));
     verify(taskSchedulerEventHandler, times(0)).taskAllocated(
@@ -828,7 +828,7 @@ public class TestContainerReuse {
     // TA12 completed.
     taskSchedulerEventHandler.handleEvent(
       new AMSchedulerEventTAEnded(ta12, container1.getId(),
-        TaskAttemptState.SUCCEEDED));
+        TaskAttemptState.SUCCEEDED, 0));
     drainableAppCallback.drain();
     LOG.info("Sleeping to ensure that the scheduling loop runs");
     Thread.sleep(3000l);
@@ -946,7 +946,7 @@ public class TestContainerReuse {
     // Container should  be assigned to task21.
     taskSchedulerEventHandler.handleEvent(
       new AMSchedulerEventTAEnded(ta11, container1.getId(),
-        TaskAttemptState.SUCCEEDED));
+        TaskAttemptState.SUCCEEDED, 0));
     drainableAppCallback.drain();
     verify(taskScheduler).deallocateTask(eq(ta11), eq(true));
     verify(taskSchedulerEventHandler).taskAllocated(
@@ -956,7 +956,7 @@ public class TestContainerReuse {
     // Task 2 completes.
     taskSchedulerEventHandler.handleEvent(
       new AMSchedulerEventTAEnded(ta21, container1.getId(),
-        TaskAttemptState.SUCCEEDED));
+        TaskAttemptState.SUCCEEDED, 0));
     verify(rmClient, times(0)).releaseAssignedContainer(eq(container1.getId()));
 
     LOG.info("Sleeping to ensure that the scheduling loop runs");
@@ -1065,7 +1065,7 @@ public class TestContainerReuse {
     assertEquals(1, assignEvent.getRemoteTaskLocalResources().size());
     
     // Task assigned to container completed successfully. Container should be re-used.
-    taskSchedulerEventHandler.handleEvent(new AMSchedulerEventTAEnded(ta111, container1.getId(), TaskAttemptState.SUCCEEDED));
+    taskSchedulerEventHandler.handleEvent(new AMSchedulerEventTAEnded(ta111, container1.getId(), TaskAttemptState.SUCCEEDED, 0));
     drainableAppCallback.drain();
     verify(taskScheduler).deallocateTask(eq(ta111), eq(true));
     verify(taskSchedulerEventHandler).taskAllocated(eq(ta112), any(Object.class), eq(container1));
@@ -1077,7 +1077,7 @@ public class TestContainerReuse {
 
     // Task assigned to container completed successfully.
     // Verify reuse across hosts.
-    taskSchedulerEventHandler.handleEvent(new AMSchedulerEventTAEnded(ta112, container1.getId(), TaskAttemptState.SUCCEEDED));
+    taskSchedulerEventHandler.handleEvent(new AMSchedulerEventTAEnded(ta112, container1.getId(), TaskAttemptState.SUCCEEDED, 0));
     drainableAppCallback.drain();
     verify(taskScheduler).deallocateTask(eq(ta112), eq(true));
     verify(rmClient, times(0)).releaseAssignedContainer(eq(container1.getId()));
@@ -1118,7 +1118,7 @@ public class TestContainerReuse {
     assertEquals(2, assignEvent.getRemoteTaskLocalResources().size());
     eventHandler.reset();
 
-    taskSchedulerEventHandler.handleEvent(new AMSchedulerEventTAEnded(ta211, container1.getId(), TaskAttemptState.SUCCEEDED));
+    taskSchedulerEventHandler.handleEvent(new AMSchedulerEventTAEnded(ta211, container1.getId(), TaskAttemptState.SUCCEEDED, 0));
     drainableAppCallback.drain();
     verify(taskScheduler).deallocateTask(eq(ta211), eq(true));
     verify(taskSchedulerEventHandler).taskAllocated(eq(ta212), any(Object.class), eq(container1));

http://git-wip-us.apache.org/repos/asf/tez/blob/25980c1a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestLocalTaskScheduler.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestLocalTaskScheduler.java b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestLocalTaskScheduler.java
index 60782e6..12390b2 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestLocalTaskScheduler.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestLocalTaskScheduler.java
@@ -59,7 +59,7 @@ public class TestLocalTaskScheduler {
     TezConfiguration tezConf = new TezConfiguration();
     tezConf.setInt(TezConfiguration.TEZ_AM_INLINE_TASK_EXECUTION_MAX_TASKS, MAX_TASKS);
 
-    LocalContainerFactory containerFactory = new LocalContainerFactory(createMockAppContext());
+    LocalContainerFactory containerFactory = new LocalContainerFactory(createMockAppContext(), 1000);
     HashMap<Object, Container> taskAllocations = new LinkedHashMap<Object, Container>();
     PriorityBlockingQueue<TaskRequest> taskRequestQueue = new PriorityBlockingQueue<TaskRequest>();
     TaskSchedulerAppCallback appClientDelegate = mock(TaskSchedulerAppCallback.class);

http://git-wip-us.apache.org/repos/asf/tez/blob/25980c1a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestLocalTaskSchedulerService.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestLocalTaskSchedulerService.java b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestLocalTaskSchedulerService.java
index 3cf4f6c..25cf4b5 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestLocalTaskSchedulerService.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestLocalTaskSchedulerService.java
@@ -22,6 +22,8 @@ import java.util.HashMap;
 import java.util.concurrent.BlockingQueue;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Resource;
@@ -81,8 +83,12 @@ public class TestLocalTaskSchedulerService {
    */
   @Test(timeout = 5000)
   public void testDeallocationBeforeAllocation() {
+    AppContext appContext = mock(AppContext.class);
+    ApplicationAttemptId appAttemptId =
+        ApplicationAttemptId.newInstance(ApplicationId.newInstance(10000l, 1), 1);
+    doReturn(appAttemptId).when(appContext).getApplicationAttemptId();
     MockLocalTaskSchedulerSerivce taskSchedulerService = new MockLocalTaskSchedulerSerivce
-        (mock(TaskSchedulerAppCallback.class), mock(ContainerSignatureMatcher.class), "", 0, "", mock(AppContext.class));
+        (mock(TaskSchedulerAppCallback.class), mock(ContainerSignatureMatcher.class), "", 0, "", appContext);
     taskSchedulerService.init(new Configuration());
     taskSchedulerService.start();
 
@@ -105,8 +111,12 @@ public class TestLocalTaskSchedulerService {
    */
   @Test(timeout = 5000)
   public void testDeallocationAfterAllocation() {
+    AppContext appContext = mock(AppContext.class);
+    ApplicationAttemptId appAttemptId =
+        ApplicationAttemptId.newInstance(ApplicationId.newInstance(10000l, 1), 1);
+    doReturn(appAttemptId).when(appContext).getApplicationAttemptId();
     MockLocalTaskSchedulerSerivce taskSchedulerService = new MockLocalTaskSchedulerSerivce
-        (mock(TaskSchedulerAppCallback.class), mock(ContainerSignatureMatcher.class), "", 0, "", mock(AppContext.class));
+        (mock(TaskSchedulerAppCallback.class), mock(ContainerSignatureMatcher.class), "", 0, "", appContext);
     taskSchedulerService.init(new Configuration());
     taskSchedulerService.start();
 
@@ -132,13 +142,13 @@ public class TestLocalTaskSchedulerService {
         String appHostName, int appHostPort, String appTrackingUrl,
         AppContext appContext) {
       super(appClient, containerSignatureMatcher, appHostName, appHostPort,
-          appTrackingUrl, appContext);
+          appTrackingUrl, 10000l, appContext);
     }
 
     @Override
     public AsyncDelegateRequestHandler createRequestHandler(Configuration conf) {
       requestHandler = new MockAsyncDelegateRequestHandler(taskRequestQueue,
-          new LocalContainerFactory(appContext),
+          new LocalContainerFactory(appContext, customContainerAppId),
           taskAllocations,
           appClientDelegate,
           conf);

http://git-wip-us.apache.org/repos/asf/tez/blob/25980c1a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerEventHandler.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerEventHandler.java b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerEventHandler.java
index 291e786..4ee05cc 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerEventHandler.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerEventHandler.java
@@ -89,7 +89,7 @@ public class TestTaskSchedulerEventHandler {
     public MockTaskSchedulerEventHandler(AppContext appContext,
         DAGClientServer clientService, EventHandler eventHandler,
         ContainerSignatureMatcher containerSignatureMatcher, WebUIService webUI) {
-      super(appContext, clientService, eventHandler, containerSignatureMatcher, webUI, new String[] {});
+      super(appContext, clientService, eventHandler, containerSignatureMatcher, webUI, new String[] {}, false);
     }
 
     @Override
@@ -162,7 +162,7 @@ public class TestTaskSchedulerEventHandler {
 
     AMSchedulerEventTALaunchRequest lr =
         new AMSchedulerEventTALaunchRequest(mockAttemptId, resource, null, mockTaskAttempt, locHint,
-            priority, containerContext);
+            priority, containerContext, 0, 0, 0);
     schedulerHandler.taskAllocated(mockTaskAttempt, lr, container);
     assertEquals(2, mockEventHandler.events.size());
     assertTrue(mockEventHandler.events.get(1) instanceof AMContainerEventAssignTA);
@@ -249,9 +249,14 @@ public class TestTaskSchedulerEventHandler {
     Configuration conf = new Configuration(false);
     schedulerHandler.init(conf);
     schedulerHandler.start();
-    
+
+    AMContainer mockAmContainer = mock(AMContainer.class);
+    when(mockAmContainer.getTaskSchedulerIdentifier()).thenReturn(0);
+    when(mockAmContainer.getContainerLauncherIdentifier()).thenReturn(0);
+    when(mockAmContainer.getTaskCommunicatorIdentifier()).thenReturn(0);
     ContainerId mockCId = mock(ContainerId.class);
     verify(mockTaskScheduler, times(0)).deallocateContainer((ContainerId)any());
+    when(mockAMContainerMap.get(mockCId)).thenReturn(mockAmContainer);
     schedulerHandler.preemptContainer(mockCId);
     verify(mockTaskScheduler, times(1)).deallocateContainer(mockCId);
     assertEquals(1, mockEventHandler.events.size());

http://git-wip-us.apache.org/repos/asf/tez/blob/25980c1a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerHelpers.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerHelpers.java b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerHelpers.java
index d775300..ffab769 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerHelpers.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerHelpers.java
@@ -127,7 +127,7 @@ class TestTaskSchedulerHelpers {
         EventHandler eventHandler,
         TezAMRMClientAsync<CookieContainerRequest> amrmClientAsync,
         ContainerSignatureMatcher containerSignatureMatcher) {
-      super(appContext, null, eventHandler, containerSignatureMatcher, null, new String[]{});
+      super(appContext, null, eventHandler, containerSignatureMatcher, null, new String[]{}, false);
       this.amrmClientAsync = amrmClientAsync;
       this.containerSignatureMatcher = containerSignatureMatcher;
     }

http://git-wip-us.apache.org/repos/asf/tez/blob/25980c1a/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 fafbba6..bdd0f61 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
@@ -63,6 +63,7 @@ import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.util.SystemClock;
 import org.apache.tez.common.security.JobTokenIdentifier;
 import org.apache.tez.common.security.TokenCache;
+import org.apache.tez.dag.api.TaskCommunicator;
 import org.apache.tez.dag.app.AppContext;
 import org.apache.tez.dag.app.ContainerHeartbeatHandler;
 import org.apache.tez.dag.app.ContainerContext;
@@ -104,7 +105,7 @@ public class TestAMContainer {
     wc.verifyState(AMContainerState.LAUNCHING);
     // 1 Launch request.
     wc.verifyCountAndGetOutgoingEvents(1);
-    verify(wc.tal).registerRunningContainer(wc.containerID);
+    verify(wc.tal).registerRunningContainer(wc.containerID, 0);
     assertNull(wc.amContainer.getCurrentTaskAttempt());
 
     // Assign task.
@@ -121,7 +122,7 @@ public class TestAMContainer {
     // Once for the previous NO_TASKS, one for the actual task.
     verify(wc.chh).register(wc.containerID);
     ArgumentCaptor<AMContainerTask> argumentCaptor = ArgumentCaptor.forClass(AMContainerTask.class);
-    verify(wc.tal, times(1)).registerTaskAttempt(argumentCaptor.capture(), eq(wc.containerID));
+    verify(wc.tal, times(1)).registerTaskAttempt(argumentCaptor.capture(), eq(wc.containerID), eq(0));
     assertEquals(1, argumentCaptor.getAllValues().size());
     assertEquals(wc.taskAttemptID, argumentCaptor.getAllValues().get(0).getTask().getTaskAttemptID());
     assertEquals(WrappedContainer.taskPriority, argumentCaptor.getAllValues().get(0).getPriority());
@@ -131,14 +132,14 @@ public class TestAMContainer {
     wc.verifyState(AMContainerState.IDLE);
     wc.verifyNoOutgoingEvents();
     assertNull(wc.amContainer.getCurrentTaskAttempt());
-    verify(wc.tal).unregisterTaskAttempt(wc.taskAttemptID);
+    verify(wc.tal).unregisterTaskAttempt(wc.taskAttemptID, 0);
 
     // Container completed
     wc.containerCompleted();
     wc.verifyHistoryStopEvent();
     wc.verifyState(AMContainerState.COMPLETED);
     wc.verifyNoOutgoingEvents();
-    verify(wc.tal).unregisterRunningContainer(wc.containerID);
+    verify(wc.tal).unregisterRunningContainer(wc.containerID, 0);
     verify(wc.chh).unregister(wc.containerID);
 
     assertEquals(1, wc.amContainer.getAllTaskAttempts().size());
@@ -157,7 +158,7 @@ public class TestAMContainer {
     wc.verifyState(AMContainerState.LAUNCHING);
     // 1 Launch request.
     wc.verifyCountAndGetOutgoingEvents(1);
-    verify(wc.tal).registerRunningContainer(wc.containerID);
+    verify(wc.tal).registerRunningContainer(wc.containerID, 0);
 
     // Container Launched
     wc.containerLaunched();
@@ -172,7 +173,7 @@ public class TestAMContainer {
     wc.verifyNoOutgoingEvents();
     assertEquals(wc.taskAttemptID, wc.amContainer.getCurrentTaskAttempt());
     ArgumentCaptor<AMContainerTask> argumentCaptor = ArgumentCaptor.forClass(AMContainerTask.class);
-    verify(wc.tal, times(1)).registerTaskAttempt(argumentCaptor.capture(), eq(wc.containerID));
+    verify(wc.tal, times(1)).registerTaskAttempt(argumentCaptor.capture(), eq(wc.containerID), eq(0));
     assertEquals(1, argumentCaptor.getAllValues().size());
     assertEquals(wc.taskAttemptID, argumentCaptor.getAllValues().get(0).getTask().getTaskAttemptID());
 
@@ -180,13 +181,13 @@ public class TestAMContainer {
     wc.verifyState(AMContainerState.IDLE);
     wc.verifyNoOutgoingEvents();
     assertNull(wc.amContainer.getCurrentTaskAttempt());
-    verify(wc.tal).unregisterTaskAttempt(wc.taskAttemptID);
+    verify(wc.tal).unregisterTaskAttempt(wc.taskAttemptID, 0);
 
     wc.containerCompleted();
     wc.verifyHistoryStopEvent();
     wc.verifyState(AMContainerState.COMPLETED);
     wc.verifyNoOutgoingEvents();
-    verify(wc.tal).unregisterRunningContainer(wc.containerID);
+    verify(wc.tal).unregisterRunningContainer(wc.containerID, 0);
     verify(wc.chh).unregister(wc.containerID);
 
     assertEquals(1, wc.amContainer.getAllTaskAttempts().size());
@@ -205,7 +206,7 @@ public class TestAMContainer {
     wc.verifyState(AMContainerState.LAUNCHING);
     // 1 Launch request.
     wc.verifyCountAndGetOutgoingEvents(1);
-    verify(wc.tal).registerRunningContainer(wc.containerID);
+    verify(wc.tal).registerRunningContainer(wc.containerID, 0);
     assertNull(wc.amContainer.getCurrentTaskAttempt());
 
     // Assign task.
@@ -222,7 +223,7 @@ public class TestAMContainer {
     // Once for the previous NO_TASKS, one for the actual task.
     verify(wc.chh).register(wc.containerID);
     ArgumentCaptor<AMContainerTask> argumentCaptor = ArgumentCaptor.forClass(AMContainerTask.class);
-    verify(wc.tal, times(1)).registerTaskAttempt(argumentCaptor.capture(), eq(wc.containerID));
+    verify(wc.tal, times(1)).registerTaskAttempt(argumentCaptor.capture(), eq(wc.containerID), eq(0));
     assertEquals(1, argumentCaptor.getAllValues().size());
     assertEquals(wc.taskAttemptID, argumentCaptor.getAllValues().get(0).getTask().getTaskAttemptID());
 
@@ -231,13 +232,13 @@ public class TestAMContainer {
     wc.verifyState(AMContainerState.IDLE);
     wc.verifyNoOutgoingEvents();
     assertNull(wc.amContainer.getCurrentTaskAttempt());
-    verify(wc.tal).unregisterTaskAttempt(wc.taskAttemptID);
+    verify(wc.tal).unregisterTaskAttempt(wc.taskAttemptID, 0);
 
     TezTaskAttemptID taId2 = TezTaskAttemptID.getInstance(wc.taskID, 2);
     wc.assignTaskAttempt(taId2);
     wc.verifyState(AMContainerState.RUNNING);
     argumentCaptor = ArgumentCaptor.forClass(AMContainerTask.class);
-    verify(wc.tal, times(2)).registerTaskAttempt(argumentCaptor.capture(), eq(wc.containerID));
+    verify(wc.tal, times(2)).registerTaskAttempt(argumentCaptor.capture(), eq(wc.containerID), eq(0));
     assertEquals(2, argumentCaptor.getAllValues().size());
     assertEquals(taId2, argumentCaptor.getAllValues().get(1).getTask().getTaskAttemptID());
 
@@ -246,14 +247,14 @@ public class TestAMContainer {
     wc.verifyState(AMContainerState.IDLE);
     wc.verifyNoOutgoingEvents();
     assertNull(wc.amContainer.getCurrentTaskAttempt());
-    verify(wc.tal).unregisterTaskAttempt(taId2);
+    verify(wc.tal).unregisterTaskAttempt(taId2, 0);
 
     // Container completed
     wc.containerCompleted();
     wc.verifyHistoryStopEvent();
     wc.verifyState(AMContainerState.COMPLETED);
     wc.verifyNoOutgoingEvents();
-    verify(wc.tal).unregisterRunningContainer(wc.containerID);
+    verify(wc.tal).unregisterRunningContainer(wc.containerID, 0);
     verify(wc.chh).unregister(wc.containerID);
 
     assertEquals(2, wc.amContainer.getAllTaskAttempts().size());
@@ -286,7 +287,7 @@ public class TestAMContainer {
     wc.verifyHistoryStopEvent();
     wc.verifyState(AMContainerState.COMPLETED);
     wc.verifyNoOutgoingEvents();
-    verify(wc.tal).unregisterRunningContainer(wc.containerID);
+    verify(wc.tal).unregisterRunningContainer(wc.containerID, 0);
     verify(wc.chh).unregister(wc.containerID);
 
     assertNull(wc.amContainer.getCurrentTaskAttempt());
@@ -323,7 +324,7 @@ public class TestAMContainer {
     wc.verifyHistoryStopEvent();
     wc.verifyState(AMContainerState.COMPLETED);
     wc.verifyNoOutgoingEvents();
-    verify(wc.tal).unregisterRunningContainer(wc.containerID);
+    verify(wc.tal).unregisterRunningContainer(wc.containerID, 0);
     verify(wc.chh).unregister(wc.containerID);
 
     assertNull(wc.amContainer.getCurrentTaskAttempt());
@@ -346,7 +347,7 @@ public class TestAMContainer {
     wc.assignTaskAttempt(taID2);
 
     wc.verifyState(AMContainerState.STOP_REQUESTED);
-    verify(wc.tal).unregisterRunningContainer(wc.containerID);
+    verify(wc.tal).unregisterRunningContainer(wc.containerID, 0);
     verify(wc.chh).unregister(wc.containerID);
     // 1 for NM stop request. 2 TERMINATING to TaskAttempt.
     outgoingEvents = wc.verifyCountAndGetOutgoingEvents(3);
@@ -378,13 +379,13 @@ public class TestAMContainer {
     wc.launchContainer();
     wc.assignTaskAttempt(wc.taskAttemptID);
     wc.verifyState(AMContainerState.LAUNCHING);
-    verify(wc.tal).registerRunningContainer(wc.containerID);
+    verify(wc.tal).registerRunningContainer(wc.containerID, 0);
 
     TezTaskAttemptID taID2 = TezTaskAttemptID.getInstance(wc.taskID, 2);
     wc.assignTaskAttempt(taID2);
 
     wc.verifyState(AMContainerState.STOP_REQUESTED);
-    verify(wc.tal).unregisterRunningContainer(wc.containerID);
+    verify(wc.tal).unregisterRunningContainer(wc.containerID, 0);
     verify(wc.chh).unregister(wc.containerID);
     // 1 for NM stop request. 2 TERMINATING to TaskAttempt.
     outgoingEvents = wc.verifyCountAndGetOutgoingEvents(3);
@@ -420,7 +421,7 @@ public class TestAMContainer {
 
     wc.containerTimedOut();
     wc.verifyState(AMContainerState.STOP_REQUESTED);
-    verify(wc.tal).unregisterRunningContainer(wc.containerID);
+    verify(wc.tal).unregisterRunningContainer(wc.containerID, 0);
     verify(wc.chh).unregister(wc.containerID);
     // 1 to TA, 1 for RM de-allocate.
     outgoingEvents = wc.verifyCountAndGetOutgoingEvents(2);
@@ -454,7 +455,7 @@ public class TestAMContainer {
 
     wc.stopRequest();
     wc.verifyState(AMContainerState.STOP_REQUESTED);
-    verify(wc.tal).unregisterRunningContainer(wc.containerID);
+    verify(wc.tal).unregisterRunningContainer(wc.containerID, 0);
     verify(wc.chh).unregister(wc.containerID);
     // 1 to TA, 1 for RM de-allocate.
     outgoingEvents = wc.verifyCountAndGetOutgoingEvents(2);
@@ -484,11 +485,11 @@ public class TestAMContainer {
     wc.launchContainer();
     wc.assignTaskAttempt(wc.taskAttemptID);
     wc.verifyState(AMContainerState.LAUNCHING);
-    verify(wc.tal).registerRunningContainer(wc.containerID);
+    verify(wc.tal).registerRunningContainer(wc.containerID, 0);
     wc.launchFailed();
     wc.verifyState(AMContainerState.STOPPING);
-    verify(wc.tal).registerRunningContainer(wc.containerID);
-    verify(wc.tal).unregisterRunningContainer(wc.containerID);
+    verify(wc.tal).registerRunningContainer(wc.containerID, 0);
+    verify(wc.tal).unregisterRunningContainer(wc.containerID, 0);
 
     outgoingEvents = wc.verifyCountAndGetOutgoingEvents(2);
     verifyUnOrderedOutgoingEventTypes(outgoingEvents,
@@ -537,8 +538,8 @@ public class TestAMContainer {
 
     wc.containerCompleted();
     wc.verifyState(AMContainerState.COMPLETED);
-    verify(wc.tal).registerRunningContainer(wc.containerID);
-    verify(wc.tal).unregisterRunningContainer(wc.containerID);
+    verify(wc.tal).registerRunningContainer(wc.containerID, 0);
+    verify(wc.tal).unregisterRunningContainer(wc.containerID, 0);
 
     outgoingEvents = wc.verifyCountAndGetOutgoingEvents(1);
     verifyUnOrderedOutgoingEventTypes(outgoingEvents,
@@ -567,8 +568,8 @@ public class TestAMContainer {
 
     wc.containerCompleted(ContainerExitStatus.DISKS_FAILED, TaskAttemptTerminationCause.NODE_DISK_ERROR);
     wc.verifyState(AMContainerState.COMPLETED);
-    verify(wc.tal).registerRunningContainer(wc.containerID);
-    verify(wc.tal).unregisterRunningContainer(wc.containerID);
+    verify(wc.tal).registerRunningContainer(wc.containerID, 0);
+    verify(wc.tal).unregisterRunningContainer(wc.containerID, 0);
 
     outgoingEvents = wc.verifyCountAndGetOutgoingEvents(1);
     verifyUnOrderedOutgoingEventTypes(outgoingEvents,
@@ -598,8 +599,8 @@ public class TestAMContainer {
 
     wc.containerCompleted(ContainerExitStatus.ABORTED, TaskAttemptTerminationCause.NODE_FAILED);
     wc.verifyState(AMContainerState.COMPLETED);
-    verify(wc.tal).registerRunningContainer(wc.containerID);
-    verify(wc.tal).unregisterRunningContainer(wc.containerID);
+    verify(wc.tal).registerRunningContainer(wc.containerID, 0);
+    verify(wc.tal).unregisterRunningContainer(wc.containerID, 0);
 
     outgoingEvents = wc.verifyCountAndGetOutgoingEvents(1);
     verifyUnOrderedOutgoingEventTypes(outgoingEvents,
@@ -629,8 +630,8 @@ public class TestAMContainer {
 
     wc.containerCompleted();
     wc.verifyState(AMContainerState.COMPLETED);
-    verify(wc.tal).registerRunningContainer(wc.containerID);
-    verify(wc.tal).unregisterRunningContainer(wc.containerID);
+    verify(wc.tal).registerRunningContainer(wc.containerID, 0);
+    verify(wc.tal).unregisterRunningContainer(wc.containerID, 0);
     verify(wc.chh).register(wc.containerID);
     verify(wc.chh).unregister(wc.containerID);
 
@@ -658,8 +659,8 @@ public class TestAMContainer {
 
     wc.containerCompleted();
     wc.verifyState(AMContainerState.COMPLETED);
-    verify(wc.tal).registerRunningContainer(wc.containerID);
-    verify(wc.tal).unregisterRunningContainer(wc.containerID);
+    verify(wc.tal).registerRunningContainer(wc.containerID, 0);
+    verify(wc.tal).unregisterRunningContainer(wc.containerID, 0);
     verify(wc.chh).register(wc.containerID);
     verify(wc.chh).unregister(wc.containerID);
 
@@ -693,8 +694,8 @@ public class TestAMContainer {
 
     wc.containerCompleted(ContainerExitStatus.PREEMPTED, TaskAttemptTerminationCause.EXTERNAL_PREEMPTION);
     wc.verifyState(AMContainerState.COMPLETED);
-    verify(wc.tal).registerRunningContainer(wc.containerID);
-    verify(wc.tal).unregisterRunningContainer(wc.containerID);
+    verify(wc.tal).registerRunningContainer(wc.containerID, 0);
+    verify(wc.tal).unregisterRunningContainer(wc.containerID, 0);
     verify(wc.chh).register(wc.containerID);
     verify(wc.chh).unregister(wc.containerID);
 
@@ -730,8 +731,8 @@ public class TestAMContainer {
 
     wc.containerCompleted(ContainerExitStatus.INVALID, TaskAttemptTerminationCause.INTERNAL_PREEMPTION);
     wc.verifyState(AMContainerState.COMPLETED);
-    verify(wc.tal).registerRunningContainer(wc.containerID);
-    verify(wc.tal).unregisterRunningContainer(wc.containerID);
+    verify(wc.tal).registerRunningContainer(wc.containerID, 0);
+    verify(wc.tal).unregisterRunningContainer(wc.containerID, 0);
     verify(wc.chh).register(wc.containerID);
     verify(wc.chh).unregister(wc.containerID);
 
@@ -767,8 +768,8 @@ public class TestAMContainer {
 
     wc.containerCompleted(ContainerExitStatus.DISKS_FAILED, TaskAttemptTerminationCause.NODE_DISK_ERROR);
     wc.verifyState(AMContainerState.COMPLETED);
-    verify(wc.tal).registerRunningContainer(wc.containerID);
-    verify(wc.tal).unregisterRunningContainer(wc.containerID);
+    verify(wc.tal).registerRunningContainer(wc.containerID, 0);
+    verify(wc.tal).unregisterRunningContainer(wc.containerID, 0);
     verify(wc.chh).register(wc.containerID);
     verify(wc.chh).unregister(wc.containerID);
 
@@ -1011,7 +1012,7 @@ public class TestAMContainer {
     wc.containerLaunched();
     wc.assignTaskAttempt(wc.taskAttemptID);
     ArgumentCaptor<AMContainerTask> argumentCaptor = ArgumentCaptor.forClass(AMContainerTask.class);
-    verify(wc.tal, times(1)).registerTaskAttempt(argumentCaptor.capture(), eq(wc.containerID));
+    verify(wc.tal, times(1)).registerTaskAttempt(argumentCaptor.capture(), eq(wc.containerID), eq(0));
     AMContainerTask task1 = argumentCaptor.getAllValues().get(0);
     assertEquals(0, task1.getAdditionalResources().size());
     wc.taskAttemptSucceeded(wc.taskAttemptID);
@@ -1024,7 +1025,7 @@ public class TestAMContainer {
     TezTaskAttemptID taID2 = TezTaskAttemptID.getInstance(wc.taskID, 2);
     wc.assignTaskAttempt(taID2, additionalResources, new Credentials());
     argumentCaptor = ArgumentCaptor.forClass(AMContainerTask.class);
-    verify(wc.tal, times(2)).registerTaskAttempt(argumentCaptor.capture(), eq(wc.containerID));
+    verify(wc.tal, times(2)).registerTaskAttempt(argumentCaptor.capture(), eq(wc.containerID), eq(0));
     AMContainerTask task2 = argumentCaptor.getAllValues().get(1);
     Map<String, LocalResource> pullTaskAdditionalResources = task2.getAdditionalResources();
     assertEquals(2, pullTaskAdditionalResources.size());
@@ -1047,7 +1048,7 @@ public class TestAMContainer {
     TezTaskAttemptID taID3 = TezTaskAttemptID.getInstance(wc.taskID, 3);
     wc.assignTaskAttempt(taID3, new HashMap<String, LocalResource>(), new Credentials());
     argumentCaptor = ArgumentCaptor.forClass(AMContainerTask.class);
-    verify(wc.tal, times(3)).registerTaskAttempt(argumentCaptor.capture(), eq(wc.containerID));
+    verify(wc.tal, times(3)).registerTaskAttempt(argumentCaptor.capture(), eq(wc.containerID), eq(0));
     AMContainerTask task3 = argumentCaptor.getAllValues().get(2);
     assertEquals(0, task3.getAdditionalResources().size());
     wc.taskAttemptSucceeded(taID3);
@@ -1100,7 +1101,7 @@ public class TestAMContainer {
     wc.containerLaunched();
     wc.assignTaskAttempt(attempt11, LRs, dag1Credentials);
     argumentCaptor = ArgumentCaptor.forClass(AMContainerTask.class);
-    verify(wc.tal, times(1)).registerTaskAttempt(argumentCaptor.capture(), eq(wc.containerID));
+    verify(wc.tal, times(1)).registerTaskAttempt(argumentCaptor.capture(), eq(wc.containerID), eq(0));
     fetchedTask = argumentCaptor.getAllValues().get(0);
     assertTrue(fetchedTask.haveCredentialsChanged());
     assertNotNull(fetchedTask.getCredentials());
@@ -1109,7 +1110,7 @@ public class TestAMContainer {
 
     wc.assignTaskAttempt(attempt12, LRs, dag1Credentials);
     argumentCaptor = ArgumentCaptor.forClass(AMContainerTask.class);
-    verify(wc.tal, times(2)).registerTaskAttempt(argumentCaptor.capture(), eq(wc.containerID));
+    verify(wc.tal, times(2)).registerTaskAttempt(argumentCaptor.capture(), eq(wc.containerID), eq(0));
     fetchedTask = argumentCaptor.getAllValues().get(1);
     assertFalse(fetchedTask.haveCredentialsChanged());
     assertNull(fetchedTask.getCredentials());
@@ -1119,7 +1120,7 @@ public class TestAMContainer {
     wc.setNewDAGID(dagID2);
     wc.assignTaskAttempt(attempt21, LRs, null);
     argumentCaptor = ArgumentCaptor.forClass(AMContainerTask.class);
-    verify(wc.tal, times(3)).registerTaskAttempt(argumentCaptor.capture(), eq(wc.containerID));
+    verify(wc.tal, times(3)).registerTaskAttempt(argumentCaptor.capture(), eq(wc.containerID), eq(0));
     fetchedTask = argumentCaptor.getAllValues().get(2);
     assertTrue(fetchedTask.haveCredentialsChanged());
     assertNull(fetchedTask.getCredentials());
@@ -1127,7 +1128,7 @@ public class TestAMContainer {
 
     wc.assignTaskAttempt(attempt22, LRs, null);
     argumentCaptor = ArgumentCaptor.forClass(AMContainerTask.class);
-    verify(wc.tal, times(4)).registerTaskAttempt(argumentCaptor.capture(), eq(wc.containerID));
+    verify(wc.tal, times(4)).registerTaskAttempt(argumentCaptor.capture(), eq(wc.containerID), eq(0));
     fetchedTask = argumentCaptor.getAllValues().get(3);
     assertFalse(fetchedTask.haveCredentialsChanged());
     assertNull(fetchedTask.getCredentials());
@@ -1137,7 +1138,7 @@ public class TestAMContainer {
     wc.setNewDAGID(dagID3);
     wc.assignTaskAttempt(attempt31, LRs , dag3Credentials);
     argumentCaptor = ArgumentCaptor.forClass(AMContainerTask.class);
-    verify(wc.tal, times(5)).registerTaskAttempt(argumentCaptor.capture(), eq(wc.containerID));
+    verify(wc.tal, times(5)).registerTaskAttempt(argumentCaptor.capture(), eq(wc.containerID), eq(0));
     fetchedTask = argumentCaptor.getAllValues().get(4);
     assertTrue(fetchedTask.haveCredentialsChanged());
     assertNotNull(fetchedTask.getCredentials());
@@ -1147,7 +1148,7 @@ public class TestAMContainer {
 
     wc.assignTaskAttempt(attempt32, LRs, dag1Credentials);
     argumentCaptor = ArgumentCaptor.forClass(AMContainerTask.class);
-    verify(wc.tal, times(6)).registerTaskAttempt(argumentCaptor.capture(), eq(wc.containerID));
+    verify(wc.tal, times(6)).registerTaskAttempt(argumentCaptor.capture(), eq(wc.containerID), eq(0));
     fetchedTask = argumentCaptor.getAllValues().get(5);
     assertFalse(fetchedTask.haveCredentialsChanged());
     assertNull(fetchedTask.getCredentials());
@@ -1200,9 +1201,10 @@ public class TestAMContainer {
 
       chh = mock(ContainerHeartbeatHandler.class);
 
-      InetSocketAddress addr = new InetSocketAddress("localhost", 0);
       tal = mock(TaskAttemptListener.class);
-      doReturn(addr).when(tal).getAddress();
+      TaskCommunicator taskComm = mock(TaskCommunicator.class);
+      doReturn(new InetSocketAddress("localhost", 0)).when(taskComm).getAddress();
+      doReturn(taskComm).when(tal).getTaskCommunicator(0);
 
       dagID = TezDAGID.getInstance(applicationID, 1);
       vertexID = TezVertexID.getInstance(dagID, 1);
@@ -1228,7 +1230,7 @@ public class TestAMContainer {
       doReturn(taskAttemptID).when(taskSpec).getTaskAttemptID();
 
       amContainer = new AMContainerImpl(container, chh, tal,
-          new ContainerContextMatcher(), appContext);
+          new ContainerContextMatcher(), appContext, 0, 0, 0);
     }
 
     public WrappedContainer() {
@@ -1278,7 +1280,7 @@ public class TestAMContainer {
       Token<JobTokenIdentifier> jobToken = mock(Token.class);
       TokenCache.setSessionToken(jobToken, credentials);
       amContainer.handle(new AMContainerEventLaunchRequest(containerID, vertexID,
-          new ContainerContext(localResources, credentials, new HashMap<String, String>(), "")));
+          new ContainerContext(localResources, credentials, new HashMap<String, String>(), ""), 0, 0));
     }
 
     public void assignTaskAttempt(TezTaskAttemptID taID) {

http://git-wip-us.apache.org/repos/asf/tez/blob/25980c1a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/container/TestAMContainerMap.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/container/TestAMContainerMap.java b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/container/TestAMContainerMap.java
index 61371e8..dee4541 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/container/TestAMContainerMap.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/container/TestAMContainerMap.java
@@ -31,6 +31,7 @@ import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.Token;
+import org.apache.tez.dag.api.TaskCommunicator;
 import org.apache.tez.dag.app.AppContext;
 import org.apache.tez.dag.app.ContainerHeartbeatHandler;
 import org.apache.tez.dag.app.TaskAttemptListener;
@@ -43,8 +44,9 @@ public class TestAMContainerMap {
 
   private TaskAttemptListener mockTaskAttemptListener() {
     TaskAttemptListener tal = mock(TaskAttemptListener.class);
-    InetSocketAddress socketAddr = new InetSocketAddress("localhost", 21000);
-    doReturn(socketAddr).when(tal).getAddress();
+    TaskCommunicator taskComm = mock(TaskCommunicator.class);
+    doReturn(new InetSocketAddress("localhost", 21000)).when(taskComm).getAddress();
+    doReturn(taskComm).when(tal).getTaskCommunicator(0);
     return tal;
   }
 

http://git-wip-us.apache.org/repos/asf/tez/blob/25980c1a/tez-examples/src/main/java/org/apache/tez/examples/JoinValidate.java
----------------------------------------------------------------------
diff --git a/tez-examples/src/main/java/org/apache/tez/examples/JoinValidate.java b/tez-examples/src/main/java/org/apache/tez/examples/JoinValidate.java
index d7fc5ac..52643c5 100644
--- a/tez-examples/src/main/java/org/apache/tez/examples/JoinValidate.java
+++ b/tez-examples/src/main/java/org/apache/tez/examples/JoinValidate.java
@@ -19,6 +19,7 @@
 package org.apache.tez.examples;
 
 import java.io.IOException;
+import java.util.Map;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -130,7 +131,7 @@ public class JoinValidate extends TezExampleBase {
 
   private DAG createDag(TezConfiguration tezConf, Path lhs, Path rhs, int numPartitions)
       throws IOException {
-    DAG dag = DAG.create("JoinValidate");
+    DAG dag = DAG.create(getDagName());
 
     // Configuration for intermediate output - shared by Vertex1 and Vertex2
     // This should only be setting selective keys from the underlying conf. Fix after there's a
@@ -147,15 +148,18 @@ public class JoinValidate extends TezExampleBase {
         MRInput
             .createConfigBuilder(new Configuration(tezConf), TextInputFormat.class,
                 lhs.toUri().toString()).groupSplits(!isDisableSplitGrouping()).build());
+    setVertexProperties(lhsVertex, getLhsVertexProperties());
 
     Vertex rhsVertex = Vertex.create(RHS_INPUT_NAME, ProcessorDescriptor.create(
         ForwardingProcessor.class.getName())).addDataSource("rhs",
         MRInput
             .createConfigBuilder(new Configuration(tezConf), TextInputFormat.class,
                 rhs.toUri().toString()).groupSplits(!isDisableSplitGrouping()).build());
+    setVertexProperties(rhsVertex, getRhsVertexProperties());
 
     Vertex joinValidateVertex = Vertex.create("joinvalidate", ProcessorDescriptor.create(
         JoinValidateProcessor.class.getName()), numPartitions);
+    setVertexProperties(joinValidateVertex, getValidateVertexProperties());
 
     Edge e1 = Edge.create(lhsVertex, joinValidateVertex, edgeConf.createDefaultEdgeProperty());
     Edge e2 = Edge.create(rhsVertex, joinValidateVertex, edgeConf.createDefaultEdgeProperty());
@@ -165,6 +169,30 @@ public class JoinValidate extends TezExampleBase {
     return dag;
   }
 
+  private void setVertexProperties(Vertex vertex, Map<String, String> properties) {
+    if (properties != null) {
+      for (Map.Entry<String, String> entry : properties.entrySet()) {
+        vertex.setConf(entry.getKey(), entry.getValue());
+      }
+    }
+  }
+
+  protected Map<String, String> getLhsVertexProperties() {
+    return null;
+  }
+
+  protected Map<String, String> getRhsVertexProperties() {
+    return null;
+  }
+
+  protected Map<String, String> getValidateVertexProperties() {
+    return null;
+  }
+
+  protected String getDagName() {
+    return "JoinValidate";
+  }
+
   public static class JoinValidateProcessor extends SimpleProcessor {
 
     private static final Logger LOG = LoggerFactory.getLogger(JoinValidateProcessor.class);

http://git-wip-us.apache.org/repos/asf/tez/blob/25980c1a/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/launcher/TezTestServiceContainerLauncher.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/launcher/TezTestServiceContainerLauncher.java b/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/launcher/TezTestServiceContainerLauncher.java
index e83165b..27356bc 100644
--- a/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/launcher/TezTestServiceContainerLauncher.java
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/launcher/TezTestServiceContainerLauncher.java
@@ -14,6 +14,8 @@
 
 package org.apache.tez.dag.app.launcher;
 
+import java.net.InetSocketAddress;
+
 import com.google.common.base.Preconditions;
 import com.google.protobuf.ByteString;
 import org.apache.commons.logging.Log;
@@ -124,7 +126,8 @@ public class TezTestServiceContainerLauncher extends AbstractService implements
 
   private RunContainerRequestProto constructRunContainerRequest(NMCommunicatorLaunchRequestEvent event) {
     RunContainerRequestProto.Builder builder = RunContainerRequestProto.newBuilder();
-    builder.setAmHost(tal.getAddress().getHostName()).setAmPort(tal.getAddress().getPort());
+    InetSocketAddress address = tal.getTaskCommunicator(event.getTaskCommId()).getAddress();
+    builder.setAmHost(address.getHostName()).setAmPort(address.getPort());
     builder.setAppAttemptNumber(event.getContainer().getId().getApplicationAttemptId().getAttemptId());
     builder.setApplicationIdString(
         event.getContainer().getId().getApplicationAttemptId().getApplicationId().toString());

http://git-wip-us.apache.org/repos/asf/tez/blob/25980c1a/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/rm/TezTestServiceTaskSchedulerService.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/rm/TezTestServiceTaskSchedulerService.java b/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/rm/TezTestServiceTaskSchedulerService.java
index e3c18bf..5657f86 100644
--- a/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/rm/TezTestServiceTaskSchedulerService.java
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/rm/TezTestServiceTaskSchedulerService.java
@@ -14,7 +14,6 @@
 
 package org.apache.tez.dag.app.rm;
 
-import java.io.IOException;
 import java.util.Arrays;
 import java.util.LinkedList;
 import java.util.List;
@@ -32,25 +31,17 @@ import com.google.common.util.concurrent.ThreadFactoryBuilder;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
-import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.NodeId;
-import org.apache.hadoop.yarn.api.records.NodeReport;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Resource;
-import org.apache.hadoop.yarn.client.api.AMRMClient;
-import org.apache.hadoop.yarn.client.api.async.AMRMClientAsync;
-import org.apache.hadoop.yarn.exceptions.YarnException;
-import org.apache.tez.dag.api.TezUncheckedException;
 import org.apache.tez.dag.app.AppContext;
 import org.apache.tez.service.TezTestServiceConfConstants;
 
 
-// TODO Registration with RM - so that the AM is considered dead and restarted in the expiry interval - 10 minutes.
-
 public class TezTestServiceTaskSchedulerService extends TaskSchedulerService {
 
   private static final Log LOG = LogFactory.getLog(TezTestServiceTaskSchedulerService.class);
@@ -71,7 +62,7 @@ public class TezTestServiceTaskSchedulerService extends TaskSchedulerService {
   private final ConcurrentMap<Object, ContainerId> runningTasks =
       new ConcurrentHashMap<Object, ContainerId>();
 
-  private final AMRMClientAsync<AMRMClient.ContainerRequest> amRmClient;
+  // AppIdIdentifier to avoid conflicts with other containres in the system.
 
   // Per instance
   private final int memoryPerInstance;
@@ -82,10 +73,13 @@ public class TezTestServiceTaskSchedulerService extends TaskSchedulerService {
   private final Resource resourcePerContainer;
 
 
+  // Not registering with the RM. Assuming the main TezScheduler will always run (except local mode),
+  // and take care of YARN registration.
   public TezTestServiceTaskSchedulerService(TaskSchedulerAppCallback appClient,
                                             AppContext appContext,
                                             String clientHostname, int clientPort,
                                             String trackingUrl,
+                                            long customAppIdIdentifier,
                                             Configuration conf) {
     // Accepting configuration here to allow setting up fields as final
     super(TezTestServiceTaskSchedulerService.class.getName());
@@ -93,7 +87,7 @@ public class TezTestServiceTaskSchedulerService extends TaskSchedulerService {
     this.appClientDelegate = createAppCallbackDelegate(appClient);
     this.appContext = appContext;
     this.serviceHosts = new LinkedList<String>();
-    this.containerFactory = new ContainerFactory(appContext);
+    this.containerFactory = new ContainerFactory(appContext, customAppIdIdentifier);
 
     this.memoryPerInstance = conf
         .getInt(TezTestServiceConfConstants.TEZ_TEST_SERVICE_MEMORY_PER_INSTANCE_MB, -1);
@@ -123,7 +117,6 @@ public class TezTestServiceTaskSchedulerService extends TaskSchedulerService {
     int memoryPerContainer = (int) (memoryPerInstance / (float) executorsPerInstance);
     int coresPerContainer = (int) (coresPerInstance / (float) executorsPerInstance);
     this.resourcePerContainer = Resource.newInstance(memoryPerContainer, coresPerContainer);
-    this.amRmClient = TezAMRMClientAsync.createAMRMClientAsync(5000, new FakeAmRmCallbackHandler());
 
     String[] hosts = conf.getTrimmedStrings(TezTestServiceConfConstants.TEZ_TEST_SERVICE_HOSTS);
     if (hosts == null || hosts.length == 0) {
@@ -143,36 +136,8 @@ public class TezTestServiceTaskSchedulerService extends TaskSchedulerService {
   }
 
   @Override
-  public void serviceInit(Configuration conf) {
-    amRmClient.init(conf);
-  }
-
-  @Override
-  public void serviceStart() {
-    amRmClient.start();
-    RegisterApplicationMasterResponse response;
-    try {
-      amRmClient.registerApplicationMaster(clientHostname, clientPort, trackingUrl);
-    } catch (YarnException e) {
-      throw new TezUncheckedException(e);
-    } catch (IOException e) {
-      throw new TezUncheckedException(e);
-    }
-  }
-
-  @Override
   public void serviceStop() {
     if (!this.isStopped.getAndSet(true)) {
-
-      try {
-        TaskSchedulerAppCallback.AppFinalStatus status = appClientDelegate.getFinalAppStatus();
-        amRmClient.unregisterApplicationMaster(status.exitStatus, status.exitMessage,
-            status.postCompletionTrackingUrl);
-      } catch (YarnException e) {
-        throw new TezUncheckedException(e);
-      } catch (IOException e) {
-        throw new TezUncheckedException(e);
-      }
       appCallbackExecutor.shutdownNow();
     }
   }
@@ -264,7 +229,7 @@ public class TezTestServiceTaskSchedulerService extends TaskSchedulerService {
 
   private ExecutorService createAppCallbackExecutorService() {
     return Executors.newSingleThreadExecutor(new ThreadFactoryBuilder()
-        .setNameFormat("TaskSchedulerAppCaller #%d").setDaemon(true).build());
+        .setNameFormat("TezTestTaskSchedulerAppCaller").setDaemon(true).build());
   }
 
   private TaskSchedulerAppCallback createAppCallbackDelegate(
@@ -274,7 +239,7 @@ public class TezTestServiceTaskSchedulerService extends TaskSchedulerService {
   }
 
   private String selectHost(String[] requestedHosts) {
-    String host = null;
+    String host;
     if (requestedHosts != null && requestedHosts.length > 0) {
       Arrays.sort(requestedHosts);
       host = requestedHosts[0];
@@ -287,17 +252,19 @@ public class TezTestServiceTaskSchedulerService extends TaskSchedulerService {
   }
 
   static class ContainerFactory {
-    final AppContext appContext;
     AtomicInteger nextId;
-
-    public ContainerFactory(AppContext appContext) {
-      this.appContext = appContext;
-      this.nextId = new AtomicInteger(2);
+    final ApplicationAttemptId customAppAttemptId;
+
+    public ContainerFactory(AppContext appContext, long appIdLong) {
+      this.nextId = new AtomicInteger(1);
+      ApplicationId appId = ApplicationId
+          .newInstance(appIdLong, appContext.getApplicationAttemptId().getApplicationId().getId());
+      this.customAppAttemptId = ApplicationAttemptId
+          .newInstance(appId, appContext.getApplicationAttemptId().getAttemptId());
     }
 
     public Container createContainer(Resource capability, Priority priority, String hostname, int port) {
-      ApplicationAttemptId appAttemptId = appContext.getApplicationAttemptId();
-      ContainerId containerId = ContainerId.newInstance(appAttemptId, nextId.getAndIncrement());
+      ContainerId containerId = ContainerId.newInstance(customAppAttemptId, nextId.getAndIncrement());
       NodeId nodeId = NodeId.newInstance(hostname, port);
       String nodeHttpAddress = "hostname:0";
 
@@ -311,37 +278,4 @@ public class TezTestServiceTaskSchedulerService extends TaskSchedulerService {
       return container;
     }
   }
-
-  private static class FakeAmRmCallbackHandler implements AMRMClientAsync.CallbackHandler {
-
-    @Override
-    public void onContainersCompleted(List<ContainerStatus> statuses) {
-
-    }
-
-    @Override
-    public void onContainersAllocated(List<Container> containers) {
-
-    }
-
-    @Override
-    public void onShutdownRequest() {
-
-    }
-
-    @Override
-    public void onNodesUpdated(List<NodeReport> updatedNodes) {
-
-    }
-
-    @Override
-    public float getProgress() {
-      return 0;
-    }
-
-    @Override
-    public void onError(Throwable e) {
-
-    }
-  }
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/25980c1a/tez-ext-service-tests/src/test/java/org/apache/tez/examples/JoinValidateConfigured.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/examples/JoinValidateConfigured.java b/tez-ext-service-tests/src/test/java/org/apache/tez/examples/JoinValidateConfigured.java
new file mode 100644
index 0000000..e5d2e3b
--- /dev/null
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/examples/JoinValidateConfigured.java
@@ -0,0 +1,53 @@
+/*
+ * 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.
+ */
+
+package org.apache.tez.examples;
+
+import java.util.Map;
+
+public class JoinValidateConfigured extends JoinValidate {
+
+  private final Map<String, String> lhsProps;
+  private final Map<String, String> rhsProps;
+  private final Map<String, String> validateProps;
+  private final String dagNameSuffix;
+
+  public JoinValidateConfigured(Map<String, String> lhsProps, Map<String, String> rhsProps,
+                                Map<String, String> validateProps, String dagNameSuffix) {
+    this.lhsProps = lhsProps;
+    this.rhsProps = rhsProps;
+    this.validateProps = validateProps;
+    this.dagNameSuffix = dagNameSuffix;
+  }
+
+  @Override
+  protected Map<String, String> getLhsVertexProperties() {
+    return this.lhsProps;
+  }
+
+  @Override
+  protected Map<String, String> getRhsVertexProperties() {
+    return this.rhsProps;
+  }
+
+  @Override
+  protected Map<String, String> getValidateVertexProperties() {
+    return this.validateProps;
+  }
+
+  @Override
+  protected String getDagName() {
+    return "JoinValidate_" + dagNameSuffix;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tez/blob/25980c1a/tez-ext-service-tests/src/test/java/org/apache/tez/tests/TestExternalTezServices.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/tests/TestExternalTezServices.java b/tez-ext-service-tests/src/test/java/org/apache/tez/tests/TestExternalTezServices.java
index ae7e7f8..9c149c6 100644
--- a/tez-ext-service-tests/src/test/java/org/apache/tez/tests/TestExternalTezServices.java
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/tests/TestExternalTezServices.java
@@ -15,11 +15,11 @@
 package org.apache.tez.tests;
 
 import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
 import java.util.Map;
 
+import com.google.common.collect.Maps;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -28,13 +28,14 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.tez.client.TezClient;
 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.app.launcher.TezTestServiceNoOpContainerLauncher;
 import org.apache.tez.dag.app.rm.TezTestServiceTaskSchedulerService;
 import org.apache.tez.dag.app.taskcomm.TezTestServiceTaskCommunicatorImpl;
 import org.apache.tez.examples.HashJoinExample;
 import org.apache.tez.examples.JoinDataGen;
-import org.apache.tez.examples.JoinValidate;
+import org.apache.tez.examples.JoinValidateConfigured;
 import org.apache.tez.service.MiniTezTestServiceCluster;
 import org.apache.tez.test.MiniTezCluster;
 import org.junit.AfterClass;
@@ -47,23 +48,31 @@ public class TestExternalTezServices {
 
   private static final String EXT_PUSH_ENTITY_NAME = "ExtServiceTestPush";
 
-  private static MiniTezCluster tezCluster;
-  private static MiniDFSCluster dfsCluster;
-  private static MiniTezTestServiceCluster tezTestServiceCluster;
+  private static volatile MiniTezCluster tezCluster;
+  private static volatile MiniDFSCluster dfsCluster;
+  private static volatile MiniTezTestServiceCluster tezTestServiceCluster;
 
-  private static Configuration clusterConf = new Configuration();
-  private static Configuration confForJobs;
+  private static volatile Configuration clusterConf = new Configuration();
+  private static volatile Configuration confForJobs;
 
-  private static FileSystem remoteFs;
-  private static FileSystem localFs;
+  private static volatile FileSystem remoteFs;
+  private static volatile FileSystem localFs;
 
-  private static TezClient sharedTezClient;
+  private static volatile TezClient sharedTezClient;
+
+  private static final Path SRC_DATA_DIR = new Path("/tmp/" + TestExternalTezServices.class.getSimpleName());
+  private static final Path HASH_JOIN_EXPECTED_RESULT_PATH = new Path(SRC_DATA_DIR, "expectedOutputPath");
+  private static final Path HASH_JOIN_OUTPUT_PATH = new Path(SRC_DATA_DIR, "outPath");
+
+  private static final Map<String, String> PROPS_EXT_SERVICE_PUSH = Maps.newHashMap();
+  private static final Map<String, String> PROPS_REGULAR_CONTAINERS = Maps.newHashMap();
+  private static final Map<String, String> PROPS_IN_AM = Maps.newHashMap();
 
   private static String TEST_ROOT_DIR = "target" + Path.SEPARATOR + TestExternalTezServices.class.getName()
       + "-tmpDir";
 
   @BeforeClass
-  public static void setup() throws IOException, TezException, InterruptedException {
+  public static void setup() throws Exception {
 
     localFs = FileSystem.getLocal(clusterConf);
 
@@ -108,27 +117,79 @@ public class TestExternalTezServices {
     remoteFs.mkdirs(stagingDirPath);
     // This is currently configured to push tasks into the Service, and then use the standard RPC
     confForJobs.set(TezConfiguration.TEZ_AM_STAGING_DIR, stagingDirPath.toString());
-    confForJobs.set(TezConfiguration.TEZ_AM_TASK_SCHEDULERS,
+
+    confForJobs.setStrings(TezConfiguration.TEZ_AM_TASK_SCHEDULERS,
+        TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT,
+//        TezConstants.TEZ_AM_SERVICE_PLUGINS_LOCAL_MODE_NAME_DEFAULT,
         EXT_PUSH_ENTITY_NAME + ":" + TezTestServiceTaskSchedulerService.class.getName());
-    confForJobs.set(TezConfiguration.TEZ_AM_CONTAINER_LAUNCHERS,
-        EXT_PUSH_ENTITY_NAME + ":" + TezTestServiceNoOpContainerLauncher.class.getName());
-    confForJobs.set(TezConfiguration.TEZ_AM_TASK_COMMUNICATORS,
-        EXT_PUSH_ENTITY_NAME + ":" + TezTestServiceTaskCommunicatorImpl.class.getName());
 
-    confForJobs.set(TezConfiguration.TEZ_AM_VERTEX_TASK_SCHEDULER_NAME, EXT_PUSH_ENTITY_NAME);
-    confForJobs.set(TezConfiguration.TEZ_AM_VERTEX_CONTAINER_LAUNCHER_NAME, EXT_PUSH_ENTITY_NAME);
-    confForJobs.set(TezConfiguration.TEZ_AM_VERTEX_TASK_COMMUNICATOR_NAME, EXT_PUSH_ENTITY_NAME);
+    confForJobs.setStrings(TezConfiguration.TEZ_AM_CONTAINER_LAUNCHERS,
+        TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT,
+//        TezConstants.TEZ_AM_SERVICE_PLUGINS_LOCAL_MODE_NAME_DEFAULT,
+        EXT_PUSH_ENTITY_NAME + ":" + TezTestServiceNoOpContainerLauncher.class.getName());
 
+    confForJobs.setStrings(TezConfiguration.TEZ_AM_TASK_COMMUNICATORS,
+        TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT,
+//        TezConstants.TEZ_AM_SERVICE_PLUGINS_LOCAL_MODE_NAME_DEFAULT,
+        EXT_PUSH_ENTITY_NAME + ":" + TezTestServiceTaskCommunicatorImpl.class.getName());
 
-    TezConfiguration tezConf = new TezConfiguration(confForJobs);
+    // Default all jobs to run via the service. Individual tests override this on a per vertex/dag level.
+    confForJobs.set(TezConfiguration.TEZ_AM_VERTEX_TASK_SCHEDULER_NAME,
+        TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT);
+    confForJobs.set(TezConfiguration.TEZ_AM_VERTEX_CONTAINER_LAUNCHER_NAME,
+        TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT);
+    confForJobs.set(TezConfiguration.TEZ_AM_VERTEX_TASK_COMMUNICATOR_NAME,
+        TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT);
+
+    // Setup various executor sets
+    PROPS_REGULAR_CONTAINERS.put(TezConfiguration.TEZ_AM_VERTEX_TASK_SCHEDULER_NAME,
+        TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT);
+    PROPS_REGULAR_CONTAINERS.put(TezConfiguration.TEZ_AM_VERTEX_CONTAINER_LAUNCHER_NAME,
+        TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT);
+    PROPS_REGULAR_CONTAINERS.put(TezConfiguration.TEZ_AM_VERTEX_TASK_COMMUNICATOR_NAME,
+        TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT);
+
+    PROPS_EXT_SERVICE_PUSH.put(TezConfiguration.TEZ_AM_VERTEX_TASK_SCHEDULER_NAME, EXT_PUSH_ENTITY_NAME);
+    PROPS_EXT_SERVICE_PUSH.put(TezConfiguration.TEZ_AM_VERTEX_CONTAINER_LAUNCHER_NAME, EXT_PUSH_ENTITY_NAME);
+    PROPS_EXT_SERVICE_PUSH.put(TezConfiguration.TEZ_AM_VERTEX_TASK_COMMUNICATOR_NAME, EXT_PUSH_ENTITY_NAME);
+
+    PROPS_IN_AM.put(TezConfiguration.TEZ_AM_VERTEX_TASK_SCHEDULER_NAME,
+        TezConstants.TEZ_AM_SERVICE_PLUGINS_LOCAL_MODE_NAME_DEFAULT);
+    PROPS_IN_AM.put(TezConfiguration.TEZ_AM_VERTEX_CONTAINER_LAUNCHER_NAME,
+        TezConstants.TEZ_AM_SERVICE_PLUGINS_LOCAL_MODE_NAME_DEFAULT);
+    PROPS_IN_AM.put(TezConfiguration.TEZ_AM_VERTEX_TASK_COMMUNICATOR_NAME,
+        TezConstants.TEZ_AM_SERVICE_PLUGINS_LOCAL_MODE_NAME_DEFAULT);
+
+
+    // Create a session to use for all tests.
+    TezConfiguration tezClientConf = new TezConfiguration(confForJobs);
 
     sharedTezClient = TezClient.create(TestExternalTezServices.class.getSimpleName() + "_session",
-        tezConf, true);
+        tezClientConf, true);
     sharedTezClient.start();
     LOG.info("Shared TezSession started");
     sharedTezClient.waitTillReady();
     LOG.info("Shared TezSession ready for submission");
 
+    // Generate the join data set used for each run.
+    // Can a timeout be enforced here ?
+    remoteFs.mkdirs(SRC_DATA_DIR);
+    Path dataPath1 = new Path(SRC_DATA_DIR, "inPath1");
+    Path dataPath2 = new Path(SRC_DATA_DIR, "inPath2");
+    TezConfiguration tezConf = new TezConfiguration(confForJobs);
+    //   Generate join data - with 2 tasks.
+    JoinDataGen dataGen = new JoinDataGen();
+    String[] dataGenArgs = new String[]{
+        dataPath1.toString(), "1048576", dataPath2.toString(), "524288",
+        HASH_JOIN_EXPECTED_RESULT_PATH.toString(), "2"};
+    assertEquals(0, dataGen.run(tezConf, dataGenArgs, sharedTezClient));
+    //    Run the actual join - with 2 reducers
+    HashJoinExample joinExample = new HashJoinExample();
+    String[] args = new String[]{
+        dataPath1.toString(), dataPath2.toString(), "2", HASH_JOIN_OUTPUT_PATH.toString()};
+    assertEquals(0, joinExample.run(tezConf, args, sharedTezClient));
+
+    LOG.info("Completed generating Data - Expected Hash Result and Actual Join Result");
   }
 
   @AfterClass
@@ -156,35 +217,50 @@ public class TestExternalTezServices {
 
 
   @Test(timeout = 60000)
-  public void test1() throws Exception {
-    Path testDir = new Path("/tmp/testHashJoinExample");
+  public void testAllInService() throws Exception {
+    int expectedExternalSubmissions = 4 + 3; //4 for 4 src files, 3 for num reducers.
+    runJoinValidate("AllInService", expectedExternalSubmissions, PROPS_EXT_SERVICE_PUSH,
+        PROPS_EXT_SERVICE_PUSH, PROPS_EXT_SERVICE_PUSH);
+  }
 
-    remoteFs.mkdirs(testDir);
+  @Test(timeout = 60000)
+  public void testAllInContainers() throws Exception {
+    int expectedExternalSubmissions = 0; // All in containers
+    runJoinValidate("AllInContainers", expectedExternalSubmissions, PROPS_REGULAR_CONTAINERS,
+        PROPS_REGULAR_CONTAINERS, PROPS_REGULAR_CONTAINERS);
+  }
 
-    Path dataPath1 = new Path(testDir, "inPath1");
-    Path dataPath2 = new Path(testDir, "inPath2");
-    Path expectedOutputPath = new Path(testDir, "expectedOutputPath");
-    Path outPath = new Path(testDir, "outPath");
+  @Test(timeout = 60000)
+  public void testMixed1() throws Exception { // M-ExtService, R-containers
+    int expectedExternalSubmissions = 4 + 0; //4 for 4 src files, 3 for num reducers.
+    runJoinValidate("Mixed1", expectedExternalSubmissions, PROPS_EXT_SERVICE_PUSH,
+        PROPS_EXT_SERVICE_PUSH, PROPS_REGULAR_CONTAINERS);
+  }
 
-    TezConfiguration tezConf = new TezConfiguration(confForJobs);
+  @Test(timeout = 60000)
+  public void testMixed2() throws Exception { // M-Containers, R-ExtService
+    int expectedExternalSubmissions = 0 + 3; //4 for 4 src files, 3 for num reducers.
+    runJoinValidate("Mixed2", expectedExternalSubmissions, PROPS_REGULAR_CONTAINERS,
+        PROPS_REGULAR_CONTAINERS, PROPS_EXT_SERVICE_PUSH);
+  }
 
-    JoinDataGen dataGen = new JoinDataGen();
-    String[] dataGenArgs = new String[]{
-        dataPath1.toString(), "1048576", dataPath2.toString(), "524288",
-        expectedOutputPath.toString(), "2"};
-    assertEquals(0, dataGen.run(tezConf, dataGenArgs, sharedTezClient));
 
-    HashJoinExample joinExample = new HashJoinExample();
-    String[] args = new String[]{
-        dataPath1.toString(), dataPath2.toString(), "2", outPath.toString()};
-    assertEquals(0, joinExample.run(tezConf, args, sharedTezClient));
+  private void runJoinValidate(String name, int extExpectedCount, Map<String, String> lhsProps,
+                               Map<String, String> rhsProps,
+                               Map<String, String> validateProps) throws
+      Exception {
+    int externalSubmissionCount = tezTestServiceCluster.getNumSubmissions();
 
-    JoinValidate joinValidate = new JoinValidate();
-    String[] validateArgs = new String[]{
-        expectedOutputPath.toString(), outPath.toString(), "3"};
+    TezConfiguration tezConf = new TezConfiguration(confForJobs);
+    JoinValidateConfigured joinValidate =
+        new JoinValidateConfigured(lhsProps, rhsProps,
+            validateProps, name);
+    String[] validateArgs = new String[]{"-disableSplitGrouping",
+        HASH_JOIN_EXPECTED_RESULT_PATH.toString(), HASH_JOIN_OUTPUT_PATH.toString(), "3"};
     assertEquals(0, joinValidate.run(tezConf, validateArgs, sharedTezClient));
 
     // Ensure this was actually submitted to the external cluster
-    assertTrue(tezTestServiceCluster.getNumSubmissions() > 0);
+    assertEquals(extExpectedCount,
+        (tezTestServiceCluster.getNumSubmissions() - externalSubmissionCount));
   }
 }


[31/50] [abbrv] tez git commit: TEZ-2125. Create a task communicator for local mode. Allow tasks to run in the AM. (sseth)

Posted by ss...@apache.org.
TEZ-2125. Create a task communicator for local mode. Allow tasks to run
in the AM. (sseth)


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

Branch: refs/heads/TEZ-2003
Commit: f7862e840f8e9ceefda4928c81a636ab6589bdab
Parents: 25980c1
Author: Siddharth Seth <ss...@apache.org>
Authored: Fri Feb 20 16:12:52 2015 -0800
Committer: Siddharth Seth <ss...@apache.org>
Committed: Wed May 6 00:13:29 2015 -0700

----------------------------------------------------------------------
 TEZ-2003-CHANGES.txt                            |  1 +
 .../org/apache/tez/dag/app/DAGAppMaster.java    | 25 +++++---
 .../dag/app/TaskAttemptListenerImpTezDag.java   | 18 ++++--
 .../dag/app/TezLocalTaskCommunicatorImpl.java   | 46 ++++++++++++++
 .../tez/dag/app/TezTaskCommunicatorImpl.java    | 67 ++++++++------------
 .../app/launcher/ContainerLauncherRouter.java   | 17 +++--
 .../app/launcher/LocalContainerLauncher.java    | 31 ++++++---
 .../dag/app/rm/TaskSchedulerEventHandler.java   |  2 +
 .../apache/tez/dag/app/MockDAGAppMaster.java    |  3 +-
 .../app/TestTaskAttemptListenerImplTezDag.java  |  2 +-
 .../tez/service/impl/ContainerRunnerImpl.java   |  2 +-
 .../tez/tests/TestExternalTezServices.java      | 57 +++++++++++++----
 .../org/apache/tez/runtime/task/TezChild.java   | 34 +++++-----
 13 files changed, 204 insertions(+), 101 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/f7862e84/TEZ-2003-CHANGES.txt
----------------------------------------------------------------------
diff --git a/TEZ-2003-CHANGES.txt b/TEZ-2003-CHANGES.txt
index 1a2264c..76496c9 100644
--- a/TEZ-2003-CHANGES.txt
+++ b/TEZ-2003-CHANGES.txt
@@ -5,5 +5,6 @@ ALL CHANGES:
   TEZ-2117. Add a manager for ContainerLaunchers running in the AM.
   TEZ-2122. Setup pluggable components at AM/Vertex level.
   TEZ-2123. Fix component managers to use pluggable components. (Enable hybrid mode)
+  TEZ-2125. Create a task communicator for local mode. Allow tasks to run in the AM.
 
 INCOMPATIBLE CHANGES:

http://git-wip-us.apache.org/repos/asf/tez/blob/f7862e84/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 89b6506..701eca8 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
@@ -465,7 +465,7 @@ public class DAGAppMaster extends AbstractService {
 
     //service to handle requests to TaskUmbilicalProtocol
     taskAttemptListener = createTaskAttemptListener(context,
-        taskHeartbeatHandler, containerHeartbeatHandler, taskCommunicatorClassIdentifiers);
+        taskHeartbeatHandler, containerHeartbeatHandler, taskCommunicatorClassIdentifiers, isLocal);
     addIfService(taskAttemptListener, true);
 
     containerSignatureMatcher = createContainerSignatureMatcher();
@@ -531,7 +531,7 @@ public class DAGAppMaster extends AbstractService {
         taskSchedulerEventHandler);
     addIfServiceDependency(taskSchedulerEventHandler, clientRpcServer);
 
-    this.containerLauncherRouter = createContainerLauncherRouter(conf, containerLauncherClassIdentifiers);
+    this.containerLauncherRouter = createContainerLauncherRouter(conf, containerLauncherClassIdentifiers, isLocal);
     addIfService(containerLauncherRouter, true);
     dispatcher.register(NMCommunicatorEventType.class, containerLauncherRouter);
 
@@ -1038,9 +1038,13 @@ public class DAGAppMaster extends AbstractService {
   }
 
   protected TaskAttemptListener createTaskAttemptListener(AppContext context,
-      TaskHeartbeatHandler thh, ContainerHeartbeatHandler chh, String[] taskCommunicatorClasses) {
+                                                          TaskHeartbeatHandler thh,
+                                                          ContainerHeartbeatHandler chh,
+                                                          String[] taskCommunicatorClasses,
+                                                          boolean isLocal) {
     TaskAttemptListener lis =
-        new TaskAttemptListenerImpTezDag(context, thh, chh,jobTokenSecretManager, taskCommunicatorClasses);
+        new TaskAttemptListenerImpTezDag(context, thh, chh, jobTokenSecretManager,
+            taskCommunicatorClasses, isLocal);
     return lis;
   }
 
@@ -1061,10 +1065,12 @@ public class DAGAppMaster extends AbstractService {
     return chh;
   }
 
-  protected ContainerLauncherRouter createContainerLauncherRouter(Configuration conf, String []containerLauncherClasses) throws
+  protected ContainerLauncherRouter createContainerLauncherRouter(Configuration conf,
+                                                                  String[] containerLauncherClasses,
+                                                                  boolean isLocal) throws
       UnknownHostException {
-    return  new ContainerLauncherRouter(conf, context, taskAttemptListener, workingDirectory, containerLauncherClasses);
-
+    return new ContainerLauncherRouter(conf, context, taskAttemptListener, workingDirectory,
+        containerLauncherClasses, isLocal);
   }
 
   public ApplicationId getAppID() {
@@ -2331,9 +2337,8 @@ public class DAGAppMaster extends AbstractService {
     StringBuilder sb = new StringBuilder();
     sb.append("AM Level configured ").append(component).append(": ");
     for (int i = 0; i < classIdentifiers.length; i++) {
-      sb.append("[").append(i).append(":").append(map.inverse().get(i)).append(":")
-          .append(taskSchedulers.inverse().get(i)).append(
-          "]");
+      sb.append("[").append(i).append(":").append(map.inverse().get(i))
+          .append(":").append(classIdentifiers[i]).append("]");
       if (i != classIdentifiers.length - 1) {
         sb.append(",");
       }

http://git-wip-us.apache.org/repos/asf/tez/blob/f7862e84/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 71b0d2a..1f0bb0e 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
@@ -99,13 +99,20 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
                                       TaskHeartbeatHandler thh, ContainerHeartbeatHandler chh,
                                       // TODO TEZ-2003 pre-merge. Remove reference to JobTokenSecretManager.
                                       JobTokenSecretManager jobTokenSecretManager,
-                                      String [] taskCommunicatorClassIdentifiers) {
+                                      String [] taskCommunicatorClassIdentifiers,
+                                      boolean isPureLocalMode) {
     super(TaskAttemptListenerImpTezDag.class.getName());
     this.context = context;
     this.taskHeartbeatHandler = thh;
     this.containerHeartbeatHandler = chh;
     if (taskCommunicatorClassIdentifiers == null || taskCommunicatorClassIdentifiers.length == 0) {
-      taskCommunicatorClassIdentifiers = new String[] {TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT};
+      if (isPureLocalMode) {
+        taskCommunicatorClassIdentifiers =
+            new String[]{TezConstants.TEZ_AM_SERVICE_PLUGINS_LOCAL_MODE_NAME_DEFAULT};
+      } else {
+        taskCommunicatorClassIdentifiers =
+            new String[]{TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT};
+      }
     }
     this.taskCommunicators = new TaskCommunicator[taskCommunicatorClassIdentifiers.length];
     for (int i = 0 ; i < taskCommunicatorClassIdentifiers.length ; i++) {
@@ -131,11 +138,12 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
   }
 
   private TaskCommunicator createTaskCommunicator(String taskCommClassIdentifier) {
-    if (taskCommClassIdentifier.equals(TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT) ||
-        taskCommClassIdentifier
-            .equals(TezConstants.TEZ_AM_SERVICE_PLUGINS_LOCAL_MODE_NAME_DEFAULT)) {
+    if (taskCommClassIdentifier.equals(TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT)) {
       LOG.info("Using Default Task Communicator");
       return new TezTaskCommunicatorImpl(this);
+    } else if (taskCommClassIdentifier.equals(TezConstants.TEZ_AM_SERVICE_PLUGINS_LOCAL_MODE_NAME_DEFAULT)) {
+      LOG.info("Using Default Local Task Communicator");
+      return new TezLocalTaskCommunicatorImpl(this);
     } else {
       LOG.info("Using TaskCommunicator: " + taskCommClassIdentifier);
       Class<? extends TaskCommunicator> taskCommClazz = (Class<? extends TaskCommunicator>) ReflectionUtils

http://git-wip-us.apache.org/repos/asf/tez/blob/f7862e84/tez-dag/src/main/java/org/apache/tez/dag/app/TezLocalTaskCommunicatorImpl.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/TezLocalTaskCommunicatorImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/TezLocalTaskCommunicatorImpl.java
new file mode 100644
index 0000000..3704cc4
--- /dev/null
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/TezLocalTaskCommunicatorImpl.java
@@ -0,0 +1,46 @@
+/*
+ * 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.
+ */
+
+package org.apache.tez.dag.app;
+
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.net.UnknownHostException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.tez.dag.api.TaskCommunicatorContext;
+import org.apache.tez.dag.api.TezUncheckedException;
+
+public class TezLocalTaskCommunicatorImpl extends TezTaskCommunicatorImpl {
+
+  private static final Log LOG = LogFactory.getLog(TezLocalTaskCommunicatorImpl.class);
+
+  public TezLocalTaskCommunicatorImpl(
+      TaskCommunicatorContext taskCommunicatorContext) {
+    super(taskCommunicatorContext);
+  }
+
+  @Override
+  protected void startRpcServer() {
+    try {
+      this.address = new InetSocketAddress(InetAddress.getLocalHost(), 0);
+    } catch (UnknownHostException e) {
+      throw new TezUncheckedException(e);
+    }
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Not starting TaskAttemptListener RPC in LocalMode");
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/tez/blob/f7862e84/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java
index 258c927..0bf1b5d 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java
@@ -15,10 +15,8 @@
 package org.apache.tez.dag.app;
 
 import java.io.IOException;
-import java.net.InetAddress;
 import java.net.InetSocketAddress;
 import java.net.URISyntaxException;
-import java.net.UnknownHostException;
 import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
@@ -76,7 +74,7 @@ public class TezTaskCommunicatorImpl extends TaskCommunicator {
   private final TezTaskUmbilicalProtocol taskUmbilical;
   private final String tokenIdentifier;
   private final Token<JobTokenIdentifier> sessionToken;
-  private InetSocketAddress address;
+  protected InetSocketAddress address;
   private Server server;
 
   public static final class ContainerInfo {
@@ -120,10 +118,8 @@ public class TezTaskCommunicatorImpl extends TaskCommunicator {
     this.sessionToken = TokenCache.getSessionToken(taskCommunicatorContext.getCredentials());
   }
 
-
   @Override
   public void serviceStart() {
-
     startRpcServer();
   }
 
@@ -134,43 +130,32 @@ public class TezTaskCommunicatorImpl extends TaskCommunicator {
 
   protected void startRpcServer() {
     Configuration conf = getConfig();
-    if (!conf.getBoolean(TezConfiguration.TEZ_LOCAL_MODE, TezConfiguration.TEZ_LOCAL_MODE_DEFAULT)) {
-      try {
-        JobTokenSecretManager jobTokenSecretManager =
-            new JobTokenSecretManager();
-        jobTokenSecretManager.addTokenForJob(tokenIdentifier, sessionToken);
-
-        server = new RPC.Builder(conf)
-            .setProtocol(TezTaskUmbilicalProtocol.class)
-            .setBindAddress("0.0.0.0")
-            .setPort(0)
-            .setInstance(taskUmbilical)
-            .setNumHandlers(
-                conf.getInt(TezConfiguration.TEZ_AM_TASK_LISTENER_THREAD_COUNT,
-                    TezConfiguration.TEZ_AM_TASK_LISTENER_THREAD_COUNT_DEFAULT))
-            .setSecretManager(jobTokenSecretManager).build();
-
-        // Enable service authorization?
-        if (conf.getBoolean(
-            CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHORIZATION,
-            false)) {
-          refreshServiceAcls(conf, new TezAMPolicyProvider());
-        }
-
-        server.start();
-        this.address = NetUtils.getConnectAddress(server);
-      } catch (IOException e) {
-        throw new TezUncheckedException(e);
-      }
-    } else {
-      try {
-        this.address = new InetSocketAddress(InetAddress.getLocalHost(), 0);
-      } catch (UnknownHostException e) {
-        throw new TezUncheckedException(e);
-      }
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Not starting TaskAttemptListener RPC in LocalMode");
+    try {
+      JobTokenSecretManager jobTokenSecretManager =
+          new JobTokenSecretManager();
+      jobTokenSecretManager.addTokenForJob(tokenIdentifier, sessionToken);
+
+      server = new RPC.Builder(conf)
+          .setProtocol(TezTaskUmbilicalProtocol.class)
+          .setBindAddress("0.0.0.0")
+          .setPort(0)
+          .setInstance(taskUmbilical)
+          .setNumHandlers(
+              conf.getInt(TezConfiguration.TEZ_AM_TASK_LISTENER_THREAD_COUNT,
+                  TezConfiguration.TEZ_AM_TASK_LISTENER_THREAD_COUNT_DEFAULT))
+          .setSecretManager(jobTokenSecretManager).build();
+
+      // Enable service authorization?
+      if (conf.getBoolean(
+          CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHORIZATION,
+          false)) {
+        refreshServiceAcls(conf, new TezAMPolicyProvider());
       }
+
+      server.start();
+      this.address = NetUtils.getConnectAddress(server);
+    } catch (IOException e) {
+      throw new TezUncheckedException(e);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/tez/blob/f7862e84/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherRouter.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherRouter.java b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherRouter.java
index 4f9b5bf..70b0cbc 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherRouter.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherRouter.java
@@ -25,7 +25,6 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.service.AbstractService;
 import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.tez.common.ReflectionUtils;
-import org.apache.tez.dag.api.TezConfiguration;
 import org.apache.tez.dag.api.TezConstants;
 import org.apache.tez.dag.api.TezUncheckedException;
 import org.apache.tez.dag.app.AppContext;
@@ -49,17 +48,24 @@ public class ContainerLauncherRouter extends AbstractService
   public ContainerLauncherRouter(Configuration conf, AppContext context,
                                  TaskAttemptListener taskAttemptListener,
                                  String workingDirectory,
-                                 String[] containerLauncherClassIdentifiers) throws UnknownHostException {
+                                 String[] containerLauncherClassIdentifiers,
+                                 boolean isPureLocalMode) throws UnknownHostException {
     super(ContainerLauncherRouter.class.getName());
 
     if (containerLauncherClassIdentifiers == null || containerLauncherClassIdentifiers.length == 0) {
-      containerLauncherClassIdentifiers = new String[] {TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT};
+      if (isPureLocalMode) {
+        containerLauncherClassIdentifiers =
+            new String[]{TezConstants.TEZ_AM_SERVICE_PLUGINS_LOCAL_MODE_NAME_DEFAULT};
+      } else {
+        containerLauncherClassIdentifiers =
+            new String[]{TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT};
+      }
     }
     containerLaunchers = new ContainerLauncher[containerLauncherClassIdentifiers.length];
 
     for (int i = 0; i < containerLauncherClassIdentifiers.length; i++) {
       containerLaunchers[i] = createContainerLauncher(containerLauncherClassIdentifiers[i], context,
-          taskAttemptListener, workingDirectory, conf);
+          taskAttemptListener, workingDirectory, isPureLocalMode, conf);
     }
   }
 
@@ -67,6 +73,7 @@ public class ContainerLauncherRouter extends AbstractService
                                                     AppContext context,
                                                     TaskAttemptListener taskAttemptListener,
                                                     String workingDirectory,
+                                                    boolean isPureLocalMode,
                                                     Configuration conf) throws
       UnknownHostException {
     if (containerLauncherClassIdentifier.equals(TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT)) {
@@ -76,7 +83,7 @@ public class ContainerLauncherRouter extends AbstractService
         .equals(TezConstants.TEZ_AM_SERVICE_PLUGINS_LOCAL_MODE_NAME_DEFAULT)) {
       LOG.info("Creating LocalContainerLauncher");
       return
-          new LocalContainerLauncher(context, taskAttemptListener, workingDirectory);
+          new LocalContainerLauncher(context, taskAttemptListener, workingDirectory, isPureLocalMode);
     } else {
       LOG.info("Creating container launcher : " + containerLauncherClassIdentifier);
       Class<? extends ContainerLauncher> containerLauncherClazz =

http://git-wip-us.apache.org/repos/asf/tez/blob/f7862e84/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/LocalContainerLauncher.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/LocalContainerLauncher.java b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/LocalContainerLauncher.java
index 9a38732..18b2e35 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/LocalContainerLauncher.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/LocalContainerLauncher.java
@@ -36,6 +36,7 @@ import java.util.concurrent.RejectedExecutionException;
 import java.util.concurrent.atomic.AtomicBoolean;
 
 import com.google.common.base.Preconditions;
+import com.google.common.collect.Maps;
 import com.google.common.util.concurrent.FutureCallback;
 import com.google.common.util.concurrent.Futures;
 import com.google.common.util.concurrent.ListenableFuture;
@@ -90,9 +91,10 @@ public class LocalContainerLauncher extends AbstractService implements
   private final AtomicBoolean serviceStopped = new AtomicBoolean(false);
   private final String workingDirectory;
   private final TaskAttemptListener tal;
-  private final Map<String, String> localEnv = new HashMap<String, String>();
+  private final Map<String, String> localEnv;
   private final ExecutionContext executionContext;
   private int numExecutors;
+  private final boolean isPureLocalMode;
 
   private final ConcurrentHashMap<ContainerId, ListenableFuture<TezChild.ContainerExecutionResult>>
       runningContainers =
@@ -112,16 +114,26 @@ public class LocalContainerLauncher extends AbstractService implements
 
   public LocalContainerLauncher(AppContext context,
                                 TaskAttemptListener taskAttemptListener,
-                                String workingDirectory) throws UnknownHostException {
+                                String workingDirectory,
+                                boolean isPureLocalMode) throws UnknownHostException {
     super(LocalContainerLauncher.class.getName());
     this.context = context;
     this.tal = taskAttemptListener;
 
     this.workingDirectory = workingDirectory;
-    AuxiliaryServiceHelper.setServiceDataIntoEnv(
-        ShuffleUtils.SHUFFLE_HANDLER_SERVICE_ID, ByteBuffer.allocate(4).putInt(0), localEnv);
-    executionContext = new ExecutionContextImpl(InetAddress.getLocalHost().getHostName());
-    // User cannot be set here since it isn't available till a DAG is running.
+    this.isPureLocalMode = isPureLocalMode;
+    if (isPureLocalMode) {
+      localEnv = Maps.newHashMap();
+      AuxiliaryServiceHelper.setServiceDataIntoEnv(
+          ShuffleUtils.SHUFFLE_HANDLER_SERVICE_ID, ByteBuffer.allocate(4).putInt(0), localEnv);
+    } else {
+      localEnv = System.getenv();
+    }
+
+    // Check if the hostname is set in the environment before overriding it.
+    String host = isPureLocalMode ? InetAddress.getLocalHost().getHostName() :
+        System.getenv(Environment.NM_HOST.name());
+    executionContext = new ExecutionContextImpl(host);
   }
 
   @Override
@@ -338,7 +350,9 @@ public class LocalContainerLauncher extends AbstractService implements
       InterruptedException, TezException, IOException {
     Map<String, String> containerEnv = new HashMap<String, String>();
     containerEnv.putAll(localEnv);
-    containerEnv.put(Environment.USER.name(), context.getUser());
+    // Use the user from env if it's available.
+    String user = isPureLocalMode ? System.getenv(Environment.USER.name()) : context.getUser();
+    containerEnv.put(Environment.USER.name(), user);
 
     long memAvailable;
     synchronized (this) { // needed to fix findbugs Inconsistent synchronization warning
@@ -347,8 +361,7 @@ public class LocalContainerLauncher extends AbstractService implements
     TezChild tezChild =
         TezChild.newTezChild(defaultConf, null, 0, containerId.toString(), tokenIdentifier,
             attemptNumber, localDirs, workingDirectory, containerEnv, "", executionContext, credentials,
-            memAvailable, context.getUser());
-    tezChild.setUmbilical(tezTaskUmbilicalProtocol);
+            memAvailable, context.getUser(), tezTaskUmbilicalProtocol);
     return tezChild;
   }
 

http://git-wip-us.apache.org/repos/asf/tez/blob/f7862e84/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerEventHandler.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerEventHandler.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerEventHandler.java
index 5a0ace8..5a8e9fe 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerEventHandler.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerEventHandler.java
@@ -434,6 +434,8 @@ public class TaskSchedulerEventHandler extends AbstractService
       } else {
         customAppIdIdentifier = SCHEDULER_APP_ID_BASE + (j++ * SCHEDULER_APP_ID_INCREMENT);
       }
+      LOG.info("ClusterIdentifier for TaskScheduler [" + i + ":" + taskSchedulerClasses[i] + "]=" +
+          customAppIdIdentifier);
       taskSchedulers[i] = createTaskScheduler(host, port,
           trackingUrl, appContext, taskSchedulerClasses[i], customAppIdIdentifier);
     }

http://git-wip-us.apache.org/repos/asf/tez/blob/f7862e84/tez-dag/src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java b/tez-dag/src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java
index a466bc6..73e93c5 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java
@@ -516,7 +516,8 @@ public class MockDAGAppMaster extends DAGAppMaster {
   // use mock container launcher for tests
   @Override
   protected ContainerLauncherRouter createContainerLauncherRouter(final Configuration conf,
-                                                                  String[] containerLaunchers)
+                                                                  String[] containerLaunchers,
+                                                                  boolean isLocal)
       throws UnknownHostException {
     return new ContainerLauncherRouter(containerLauncher);
   }

http://git-wip-us.apache.org/repos/asf/tez/blob/f7862e84/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag.java b/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag.java
index bffb5b9..7a365bd 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag.java
@@ -117,7 +117,7 @@ public class TestTaskAttemptListenerImplTezDag {
     doReturn(container).when(amContainer).getContainer();
 
     taskAttemptListener = new TaskAttemptListenerImplForTest(appContext,
-        mock(TaskHeartbeatHandler.class), mock(ContainerHeartbeatHandler.class), null, null);
+        mock(TaskHeartbeatHandler.class), mock(ContainerHeartbeatHandler.class), null, null, false);
 
     taskSpec = mock(TaskSpec.class);
     doReturn(taskAttemptID).when(taskSpec).getTaskAttemptID();

http://git-wip-us.apache.org/repos/asf/tez/blob/f7862e84/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/ContainerRunnerImpl.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/ContainerRunnerImpl.java b/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/ContainerRunnerImpl.java
index 4a6ce33..25d6030 100644
--- a/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/ContainerRunnerImpl.java
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/ContainerRunnerImpl.java
@@ -282,7 +282,7 @@ public class ContainerRunnerImpl extends AbstractService implements ContainerRun
               request.getContainerIdString(),
               request.getTokenIdentifier(), request.getAppAttemptNumber(), workingDir, localDirs,
               envMap, objectRegistry, pid,
-              executionContext, credentials, memoryAvailable, request.getUser());
+              executionContext, credentials, memoryAvailable, request.getUser(), null);
       ContainerExecutionResult result = tezChild.run();
       LOG.info("ExecutionTime for Container: " + request.getContainerIdString() + "=" +
           sw.stop().elapsedMillis());

http://git-wip-us.apache.org/repos/asf/tez/blob/f7862e84/tez-ext-service-tests/src/test/java/org/apache/tez/tests/TestExternalTezServices.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/tests/TestExternalTezServices.java b/tez-ext-service-tests/src/test/java/org/apache/tez/tests/TestExternalTezServices.java
index 9c149c6..01c2080 100644
--- a/tez-ext-service-tests/src/test/java/org/apache/tez/tests/TestExternalTezServices.java
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/tests/TestExternalTezServices.java
@@ -40,6 +40,7 @@ import org.apache.tez.service.MiniTezTestServiceCluster;
 import org.apache.tez.test.MiniTezCluster;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
+import org.junit.Ignore;
 import org.junit.Test;
 
 public class TestExternalTezServices {
@@ -120,26 +121,23 @@ public class TestExternalTezServices {
 
     confForJobs.setStrings(TezConfiguration.TEZ_AM_TASK_SCHEDULERS,
         TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT,
-//        TezConstants.TEZ_AM_SERVICE_PLUGINS_LOCAL_MODE_NAME_DEFAULT,
+        TezConstants.TEZ_AM_SERVICE_PLUGINS_LOCAL_MODE_NAME_DEFAULT,
         EXT_PUSH_ENTITY_NAME + ":" + TezTestServiceTaskSchedulerService.class.getName());
 
     confForJobs.setStrings(TezConfiguration.TEZ_AM_CONTAINER_LAUNCHERS,
         TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT,
-//        TezConstants.TEZ_AM_SERVICE_PLUGINS_LOCAL_MODE_NAME_DEFAULT,
+        TezConstants.TEZ_AM_SERVICE_PLUGINS_LOCAL_MODE_NAME_DEFAULT,
         EXT_PUSH_ENTITY_NAME + ":" + TezTestServiceNoOpContainerLauncher.class.getName());
 
     confForJobs.setStrings(TezConfiguration.TEZ_AM_TASK_COMMUNICATORS,
         TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT,
-//        TezConstants.TEZ_AM_SERVICE_PLUGINS_LOCAL_MODE_NAME_DEFAULT,
+        TezConstants.TEZ_AM_SERVICE_PLUGINS_LOCAL_MODE_NAME_DEFAULT,
         EXT_PUSH_ENTITY_NAME + ":" + TezTestServiceTaskCommunicatorImpl.class.getName());
 
     // Default all jobs to run via the service. Individual tests override this on a per vertex/dag level.
-    confForJobs.set(TezConfiguration.TEZ_AM_VERTEX_TASK_SCHEDULER_NAME,
-        TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT);
-    confForJobs.set(TezConfiguration.TEZ_AM_VERTEX_CONTAINER_LAUNCHER_NAME,
-        TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT);
-    confForJobs.set(TezConfiguration.TEZ_AM_VERTEX_TASK_COMMUNICATOR_NAME,
-        TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT);
+    confForJobs.set(TezConfiguration.TEZ_AM_VERTEX_TASK_SCHEDULER_NAME, EXT_PUSH_ENTITY_NAME);
+    confForJobs.set(TezConfiguration.TEZ_AM_VERTEX_CONTAINER_LAUNCHER_NAME, EXT_PUSH_ENTITY_NAME);
+    confForJobs.set(TezConfiguration.TEZ_AM_VERTEX_TASK_COMMUNICATOR_NAME, EXT_PUSH_ENTITY_NAME);
 
     // Setup various executor sets
     PROPS_REGULAR_CONTAINERS.put(TezConfiguration.TEZ_AM_VERTEX_TASK_SCHEDULER_NAME,
@@ -232,18 +230,55 @@ public class TestExternalTezServices {
 
   @Test(timeout = 60000)
   public void testMixed1() throws Exception { // M-ExtService, R-containers
-    int expectedExternalSubmissions = 4 + 0; //4 for 4 src files, 3 for num reducers.
+    int expectedExternalSubmissions = 4 + 0; //4 for 4 src files, 0 for num reducers.
     runJoinValidate("Mixed1", expectedExternalSubmissions, PROPS_EXT_SERVICE_PUSH,
         PROPS_EXT_SERVICE_PUSH, PROPS_REGULAR_CONTAINERS);
   }
 
   @Test(timeout = 60000)
   public void testMixed2() throws Exception { // M-Containers, R-ExtService
-    int expectedExternalSubmissions = 0 + 3; //4 for 4 src files, 3 for num reducers.
+    int expectedExternalSubmissions = 0 + 3; // 3 for num reducers.
     runJoinValidate("Mixed2", expectedExternalSubmissions, PROPS_REGULAR_CONTAINERS,
         PROPS_REGULAR_CONTAINERS, PROPS_EXT_SERVICE_PUSH);
   }
 
+  @Test(timeout = 60000)
+  public void testMixed3() throws Exception { // M - service, R-AM
+    int expectedExternalSubmissions = 4 + 0; //4 for 4 src files, 0 for num reducers (in-AM).
+    runJoinValidate("Mixed3", expectedExternalSubmissions, PROPS_EXT_SERVICE_PUSH,
+        PROPS_EXT_SERVICE_PUSH, PROPS_IN_AM);
+  }
+
+  @Test(timeout = 60000)
+  public void testMixed4() throws Exception { // M - containers, R-AM
+    int expectedExternalSubmissions = 0 + 0; // Nothing in external service.
+    runJoinValidate("Mixed4", expectedExternalSubmissions, PROPS_REGULAR_CONTAINERS,
+        PROPS_REGULAR_CONTAINERS, PROPS_IN_AM);
+  }
+
+  @Test(timeout = 60000)
+  public void testMixed5() throws Exception { // M1 - containers, M2-extservice, R-AM
+    int expectedExternalSubmissions = 2 + 0; // 2 for M2
+    runJoinValidate("Mixed5", expectedExternalSubmissions, PROPS_REGULAR_CONTAINERS,
+        PROPS_EXT_SERVICE_PUSH, PROPS_IN_AM);
+  }
+
+
+  @Ignore // Re-activate this after the AM registers the shuffle token with the launcher.
+  @Test(timeout = 60000)
+  public void testMixed6() throws Exception { // M - AM, R - Service
+    int expectedExternalSubmissions = 0 + 3; // 3 for R in service
+    runJoinValidate("Mixed6", expectedExternalSubmissions, PROPS_IN_AM,
+        PROPS_IN_AM, PROPS_EXT_SERVICE_PUSH);
+  }
+
+  @Test(timeout = 60000)
+  public void testMixed7() throws Exception { // M - AM, R - Containers
+    int expectedExternalSubmissions = 0; // Nothing in ext service
+    runJoinValidate("Mixed7", expectedExternalSubmissions, PROPS_IN_AM,
+        PROPS_IN_AM, PROPS_REGULAR_CONTAINERS);
+  }
+
 
   private void runJoinValidate(String name, int extExpectedCount, Map<String, String> lhsProps,
                                Map<String, String> rhsProps,

http://git-wip-us.apache.org/repos/asf/tez/blob/f7862e84/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezChild.java
----------------------------------------------------------------------
diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezChild.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezChild.java
index 3cba3ce..7615f08 100644
--- a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezChild.java
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezChild.java
@@ -64,6 +64,7 @@ import org.apache.tez.dag.utils.RelocalizationUtils;
 import org.apache.tez.runtime.api.ExecutionContext;
 import org.apache.tez.runtime.api.impl.ExecutionContextImpl;
 import org.apache.tez.runtime.api.impl.TaskSpec;
+import org.apache.tez.runtime.api.impl.TezUmbilical;
 import org.apache.tez.runtime.common.objectregistry.ObjectRegistryImpl;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -93,7 +94,6 @@ public class TezChild {
   private final int amHeartbeatInterval;
   private final long sendCounterInterval;
   private final int maxEventsToGet;
-  private final boolean isLocal;
   private final String workingDir;
 
   private final ListeningExecutorService executor;
@@ -108,9 +108,10 @@ public class TezChild {
   private final String user;
 
   private Multimap<String, String> startedInputsMap = HashMultimap.create();
+  private final boolean ownUmbilical;
 
+  private final TezTaskUmbilicalProtocol umbilical;
   private TaskReporter taskReporter;
-  private TezTaskUmbilicalProtocol umbilical;
   private int taskCount = 0;
   private TezVertexID lastVertexID;
 
@@ -119,7 +120,7 @@ public class TezChild {
       Map<String, String> serviceProviderEnvMap,
       ObjectRegistryImpl objectRegistry, String pid,
       ExecutionContext executionContext,
-      Credentials credentials, long memAvailable, String user)
+      Credentials credentials, long memAvailable, String user, TezTaskUmbilicalProtocol umbilical)
       throws IOException, InterruptedException {
     this.defaultConf = conf;
     this.containerIdString = containerIdentifier;
@@ -133,6 +134,8 @@ public class TezChild {
     this.memAvailable = memAvailable;
     this.user = user;
 
+    LOG.info("TezChild created with umbilical: " + umbilical);
+
     getTaskMaxSleepTime = defaultConf.getInt(
         TezConfiguration.TEZ_TASK_GET_TASK_SLEEP_INTERVAL_MS_MAX,
         TezConfiguration.TEZ_TASK_GET_TASK_SLEEP_INTERVAL_MS_MAX_DEFAULT);
@@ -161,25 +164,27 @@ public class TezChild {
       }
     }
 
-    this.isLocal = defaultConf.getBoolean(TezConfiguration.TEZ_LOCAL_MODE,
-        TezConfiguration.TEZ_LOCAL_MODE_DEFAULT);
     UserGroupInformation taskOwner = UserGroupInformation.createRemoteUser(tokenIdentifier);
     Token<JobTokenIdentifier> jobToken = TokenCache.getSessionToken(credentials);
 
     serviceConsumerMetadata.put(TezConstants.TEZ_SHUFFLE_HANDLER_SERVICE_ID,
         TezCommonUtils.convertJobTokenToBytes(jobToken));
 
-    if (!isLocal) {
+    if (umbilical == null) {
       final InetSocketAddress address = NetUtils.createSocketAddrForHost(host, port);
       SecurityUtil.setTokenService(jobToken, address);
       taskOwner.addToken(jobToken);
-      umbilical = taskOwner.doAs(new PrivilegedExceptionAction<TezTaskUmbilicalProtocol>() {
+      this.umbilical = taskOwner.doAs(new PrivilegedExceptionAction<TezTaskUmbilicalProtocol>() {
         @Override
         public TezTaskUmbilicalProtocol run() throws Exception {
           return RPC.getProxy(TezTaskUmbilicalProtocol.class,
               TezTaskUmbilicalProtocol.versionID, address, defaultConf);
         }
       });
+      ownUmbilical = true;
+    } else {
+      this.umbilical = umbilical;
+      ownUmbilical = false;
     }
   }
   
@@ -353,7 +358,7 @@ public class TezChild {
       if (taskReporter != null) {
         taskReporter.shutdown();
       }
-      if (!isLocal) {
+      if (ownUmbilical) {
         RPC.stopProxy(umbilical);
         // TODO Temporary change. Revert. Ideally, move this over to the main method in TezChild if possible.
 //        LogManager.shutdown();
@@ -361,12 +366,6 @@ public class TezChild {
     }
   }
 
-  public void setUmbilical(TezTaskUmbilicalProtocol tezTaskUmbilicalProtocol){
-    if(tezTaskUmbilicalProtocol != null){
-      this.umbilical = tezTaskUmbilicalProtocol;
-    }
-  }
-
   public static class ContainerExecutionResult {
     public static enum ExitStatus {
       SUCCESS(0),
@@ -412,7 +411,8 @@ public class TezChild {
   public static TezChild newTezChild(Configuration conf, String host, int port, String containerIdentifier,
       String tokenIdentifier, int attemptNumber, String[] localDirs, String workingDirectory,
       Map<String, String> serviceProviderEnvMap, @Nullable String pid,
-      ExecutionContext executionContext, Credentials credentials, long memAvailable, String user)
+      ExecutionContext executionContext, Credentials credentials, long memAvailable, String user,
+      TezTaskUmbilicalProtocol tezUmbilical)
       throws IOException, InterruptedException, TezException {
 
     // Pull in configuration specified for the session.
@@ -425,7 +425,7 @@ public class TezChild {
 
     return new TezChild(conf, host, port, containerIdentifier, tokenIdentifier,
         attemptNumber, workingDirectory, localDirs, serviceProviderEnvMap, objectRegistry, pid,
-        executionContext, credentials, memAvailable, user);
+        executionContext, credentials, memAvailable, user, tezUmbilical);
   }
 
   public static void main(String[] args) throws IOException, InterruptedException, TezException {
@@ -459,7 +459,7 @@ public class TezChild {
         tokenIdentifier, attemptNumber, localDirs, System.getenv(Environment.PWD.name()),
         System.getenv(), pid, new ExecutionContextImpl(System.getenv(Environment.NM_HOST.name())),
         credentials, Runtime.getRuntime().maxMemory(), System
-            .getenv(ApplicationConstants.Environment.USER.toString()));
+            .getenv(ApplicationConstants.Environment.USER.toString()), null);
     tezChild.run();
   }
 


[14/50] [abbrv] tez git commit: TEZ-1897 addendum to turn off by default . Create a concurrent version of AsyncDispatcher (bikas)

Posted by ss...@apache.org.
TEZ-1897 addendum to turn off by default . Create a concurrent version of AsyncDispatcher (bikas)


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

Branch: refs/heads/TEZ-2003
Commit: 5218f481dba2a26c3aa5dd8f69285ab9da419dd1
Parents: c411e4e
Author: Bikas Saha <bi...@apache.org>
Authored: Mon May 4 12:05:39 2015 -0700
Committer: Bikas Saha <bi...@apache.org>
Committed: Mon May 4 12:05:39 2015 -0700

----------------------------------------------------------------------
 tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/5218f481/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java b/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java
index a301957..f64172e 100644
--- a/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java
+++ b/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java
@@ -146,7 +146,7 @@ public class TezConfiguration extends Configuration {
   public static final String TEZ_AM_USE_CONCURRENT_DISPATCHER = TEZ_AM_PREFIX
       + "use.concurrent-dispatcher";
   @Private
-  public static boolean TEZ_AM_USE_CONCURRENT_DISPATCHER_DEFAULT = true;
+  public static boolean TEZ_AM_USE_CONCURRENT_DISPATCHER_DEFAULT = false;
   
   @Private
   @ConfigurationScope(Scope.AM)


[45/50] [abbrv] tez git commit: TEZ-2381. Fixes after rebase 04/28. (sseth)

Posted by ss...@apache.org.
TEZ-2381. Fixes after rebase 04/28. (sseth)


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

Branch: refs/heads/TEZ-2003
Commit: 48d984218ef653969cde114ea230c753ea7e4f0c
Parents: e5886f8
Author: Siddharth Seth <ss...@apache.org>
Authored: Tue Apr 28 13:41:12 2015 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Wed May 6 00:18:08 2015 -0700

----------------------------------------------------------------------
 TEZ-2003-CHANGES.txt                            |  1 +
 .../dag/app/TaskAttemptListenerImpTezDag.java   | 17 ++++----
 .../app/TestTaskAttemptListenerImplTezDag.java  | 45 ++++++++++++++------
 3 files changed, 42 insertions(+), 21 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/48d98421/TEZ-2003-CHANGES.txt
----------------------------------------------------------------------
diff --git a/TEZ-2003-CHANGES.txt b/TEZ-2003-CHANGES.txt
index f6bc8e7..d42aaf8 100644
--- a/TEZ-2003-CHANGES.txt
+++ b/TEZ-2003-CHANGES.txt
@@ -18,5 +18,6 @@ ALL CHANGES:
   TEZ-2302. Allow TaskCommunicators to subscribe for Vertex updates.
   TEZ-2347. Expose additional information in TaskCommunicatorContext.
   TEZ-2361. Propagate dag completion to TaskCommunicator.
+  TEZ-2381. Fixes after rebase 04/28.
 
 INCOMPATIBLE CHANGES:

http://git-wip-us.apache.org/repos/asf/tez/blob/48d98421/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 61bd4ca..c586787 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
@@ -17,28 +17,22 @@
 
 package org.apache.tez.dag.app;
 
-import javax.annotation.Nullable;
 import java.io.IOException;
 import java.lang.reflect.Constructor;
 import java.lang.reflect.InvocationTargetException;
-import java.net.InetSocketAddress;
-import java.net.URISyntaxException;
-import java.net.UnknownHostException;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
-import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 
+import com.google.common.annotations.VisibleForTesting;
 import org.apache.commons.collections4.ListUtils;
 import org.apache.tez.dag.app.dag.event.TaskAttemptEvent;
 import org.apache.tez.dag.app.dag.event.TaskAttemptEventStatusUpdate;
 import org.apache.tez.dag.app.dag.event.TaskAttemptEventType;
 import org.apache.tez.runtime.api.events.TaskStatusUpdateEvent;
 import org.apache.tez.runtime.api.impl.EventType;
-import com.google.common.base.Preconditions;
-import org.apache.tez.dag.api.event.VertexState;
 import org.apache.tez.dag.api.event.VertexStateUpdate;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -62,7 +56,6 @@ import org.apache.tez.dag.app.dag.Task;
 import org.apache.tez.dag.app.dag.event.TaskAttemptEventAttemptFailed;
 import org.apache.tez.dag.app.dag.event.TaskAttemptEventAttemptKilled;
 import org.apache.tez.dag.app.dag.event.TaskAttemptEventStartedRemotely;
-import org.apache.tez.dag.app.dag.event.TaskAttemptEventType;
 import org.apache.tez.dag.app.dag.event.VertexEventRouteEvent;
 import org.apache.tez.dag.app.rm.container.AMContainerTask;
 import org.apache.tez.dag.records.TezTaskAttemptID;
@@ -151,7 +144,7 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
   private TaskCommunicator createTaskCommunicator(String taskCommClassIdentifier, int taskCommIndex) {
     if (taskCommClassIdentifier.equals(TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT)) {
       LOG.info("Using Default Task Communicator");
-      return new TezTaskCommunicatorImpl(taskCommunicatorContexts[taskCommIndex]);
+      return createTezTaskCommunicator(taskCommunicatorContexts[taskCommIndex]);
     } else if (taskCommClassIdentifier.equals(TezConstants.TEZ_AM_SERVICE_PLUGINS_LOCAL_MODE_NAME_DEFAULT)) {
       LOG.info("Using Default Local Task Communicator");
       return new TezLocalTaskCommunicatorImpl(taskCommunicatorContexts[taskCommIndex]);
@@ -174,6 +167,12 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
       }
     }
   }
+
+  @VisibleForTesting
+  protected TezTaskCommunicatorImpl createTezTaskCommunicator(TaskCommunicatorContext context) {
+    return new TezTaskCommunicatorImpl(context);
+  }
+
   public TaskHeartbeatResponse heartbeat(TaskHeartbeatRequest request)
       throws IOException, TezException {
     ContainerId containerId = ConverterUtils.toContainerId(request

http://git-wip-us.apache.org/repos/asf/tez/blob/48d98421/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag.java b/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag.java
index fa62c11..55a546e 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag.java
@@ -42,11 +42,9 @@ import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.tez.common.ContainerContext;
 import org.apache.tez.common.ContainerTask;
 import org.apache.tez.common.security.JobTokenSecretManager;
+import org.apache.tez.dag.api.TaskHeartbeatRequest;
 import org.apache.tez.dag.api.TezException;
 import org.apache.hadoop.yarn.api.records.NodeId;
-import org.apache.hadoop.yarn.event.EventHandler;
-import org.apache.tez.common.ContainerContext;
-import org.apache.tez.common.ContainerTask;
 import org.apache.tez.common.TezTaskUmbilicalProtocol;
 import org.apache.tez.dag.api.TaskCommunicatorContext;
 import org.apache.tez.dag.app.dag.DAG;
@@ -68,7 +66,6 @@ import org.apache.tez.runtime.api.events.TaskStatusUpdateEvent;
 import org.apache.tez.runtime.api.impl.EventType;
 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.junit.Before;
 import org.junit.Test;
 import org.mockito.ArgumentCaptor;
@@ -77,7 +74,9 @@ import org.mockito.ArgumentCaptor;
 // TODO TEZ-2003 Rename to TestTezTaskCommunicator
 public class TestTaskAttemptListenerImplTezDag {
   private ApplicationId appId;
+  private ApplicationAttemptId appAttemptId;
   private AppContext appContext;
+  Credentials credentials;
   AMContainerMap amContainerMap;
   EventHandler eventHandler;
   DAG dag;
@@ -93,11 +92,13 @@ public class TestTaskAttemptListenerImplTezDag {
   @Before
   public void setUp() {
     appId = ApplicationId.newInstance(1000, 1);
+    appAttemptId = ApplicationAttemptId.newInstance(appId, 1);
     dag = mock(DAG.class);
     TezDAGID dagID = TezDAGID.getInstance(appId, 1);
     vertexID = TezVertexID.getInstance(dagID, 1);
     taskID = TezTaskID.getInstance(vertexID, 1);
     taskAttemptID = TezTaskAttemptID.getInstance(taskID, 1);
+    credentials = new Credentials();
 
     amContainerMap = mock(AMContainerMap.class);
     Map<ApplicationAccessType, String> appAcls = new HashMap<ApplicationAccessType, String>();
@@ -109,6 +110,8 @@ public class TestTaskAttemptListenerImplTezDag {
     doReturn(dag).when(appContext).getCurrentDAG();
     doReturn(appAcls).when(appContext).getApplicationACLs();
     doReturn(amContainerMap).when(appContext).getAllContainers();
+    doReturn(appAttemptId).when(appContext).getApplicationAttemptId();
+    doReturn(credentials).when(appContext).getAppCredentials();
     NodeId nodeId = NodeId.newInstance("localhost", 0);
 
     AMContainer amContainer = mock(AMContainer.class);
@@ -150,7 +153,7 @@ public class TestTaskAttemptListenerImplTezDag {
     assertEquals(taskSpec, containerTask.getTaskSpec());
 
     // Task unregistered. Should respond to heartbeats
-    taskAttemptListener.unregisterTaskAttempt(taskAttemptId, 0);
+    taskAttemptListener.unregisterTaskAttempt(taskAttemptID, 0);
     containerTask = tezUmbilical.getTask(containerContext2);
     assertNull(containerTask);
 
@@ -180,7 +183,7 @@ public class TestTaskAttemptListenerImplTezDag {
     TezTaskUmbilicalProtocol tezUmbilical = taskCommunicator.getUmbilical();
 
     ContainerId containerId1 = createContainerId(appId, 1);
-    doReturn(mock(AMContainer.class)).when(amContainerMap).get(containerId1);
+
     ContainerContext containerContext1 = new ContainerContext(containerId1.toString());
     taskAttemptListener.registerRunningContainer(containerId1, 0);
     containerTask = tezUmbilical.getTask(containerContext1);
@@ -241,7 +244,6 @@ public class TestTaskAttemptListenerImplTezDag {
 
   private EventHandler generateHeartbeat(List<TezEvent> events) throws IOException, TezException {
     ContainerId containerId = createContainerId(appId, 1);
-    long requestId = 0;
     Vertex vertex = mock(Vertex.class);
     Task task = mock(Task.class);
 
@@ -251,13 +253,13 @@ public class TestTaskAttemptListenerImplTezDag {
 
     doReturn(new ArrayList<TezEvent>()).when(task).getTaskAttemptTezEvents(taskAttemptID, 0, 1);
 
-    taskAttemptListener.registerRunningContainer(containerId);
-    taskAttemptListener.registerTaskAttempt(amContainerTask, containerId);
+    taskAttemptListener.registerRunningContainer(containerId, 0);
+    taskAttemptListener.registerTaskAttempt(amContainerTask, containerId, 0);
+
+    TaskHeartbeatRequest request = mock(TaskHeartbeatRequest.class);
 
-    TezHeartbeatRequest request = mock(TezHeartbeatRequest.class);
     doReturn(containerId.toString()).when(request).getContainerIdentifier();
-    doReturn(taskAttemptID).when(request).getCurrentTaskAttemptID();
-    doReturn(++requestId).when(request).getRequestId();
+    doReturn(taskAttemptID).when(request).getTaskAttemptId();
     doReturn(events).when(request).getEvents();
 
     taskAttemptListener.heartbeat(request);
@@ -271,6 +273,25 @@ public class TestTaskAttemptListenerImplTezDag {
     return ContainerId.newInstance(appAttemptId, containerIdx);
   }
 
+  private static class TaskAttemptListenerImplForTest extends TaskAttemptListenerImpTezDag {
+
+    public TaskAttemptListenerImplForTest(AppContext context,
+                                          TaskHeartbeatHandler thh,
+                                          ContainerHeartbeatHandler chh,
+                                          JobTokenSecretManager jobTokenSecretManager,
+                                          String[] taskCommunicatorClassIdentifiers,
+                                          boolean isPureLocalMode) {
+      super(context, thh, chh, jobTokenSecretManager, taskCommunicatorClassIdentifiers,
+          isPureLocalMode);
+    }
+
+    @Override
+    protected TezTaskCommunicatorImpl createTezTaskCommunicator(TaskCommunicatorContext context) {
+      return new TezTaskCommunicatorImplForTest(context);
+    }
+
+  }
+
   private static class TezTaskCommunicatorImplForTest extends TezTaskCommunicatorImpl {
 
     public TezTaskCommunicatorImplForTest(


[34/50] [abbrv] tez git commit: TEZ-2117. Add a manager for ContainerLaunchers running in the AM. (sseth)

Posted by ss...@apache.org.
TEZ-2117. Add a manager for ContainerLaunchers running in the AM.
(sseth)


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

Branch: refs/heads/TEZ-2003
Commit: fb7c0d612d21c41f6d6af232715cb3ad0a46325c
Parents: 8b40191
Author: Siddharth Seth <ss...@apache.org>
Authored: Wed Feb 18 14:45:34 2015 -0800
Committer: Siddharth Seth <ss...@apache.org>
Committed: Wed May 6 00:13:29 2015 -0700

----------------------------------------------------------------------
 TEZ-2003-CHANGES.txt                            |   1 +
 .../org/apache/tez/dag/app/DAGAppMaster.java    |  54 ++--------
 .../tez/dag/app/launcher/ContainerLauncher.java |   2 +-
 .../app/launcher/ContainerLauncherRouter.java   | 108 +++++++++++++++++++
 .../apache/tez/dag/app/MockDAGAppMaster.java    |   5 +-
 5 files changed, 124 insertions(+), 46 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/fb7c0d61/TEZ-2003-CHANGES.txt
----------------------------------------------------------------------
diff --git a/TEZ-2003-CHANGES.txt b/TEZ-2003-CHANGES.txt
index 975ce65..1cd74a4 100644
--- a/TEZ-2003-CHANGES.txt
+++ b/TEZ-2003-CHANGES.txt
@@ -2,5 +2,6 @@ ALL CHANGES:
   TEZ-2019. Temporarily allow the scheduler and launcher to be specified via configuration.
   TEZ-2006. Task communication plane needs to be pluggable.
   TEZ-2090. Add tests for jobs running in external services.
+  TEZ-2117. Add a manager for ContainerLaunchers running in the AM.
 
 INCOMPATIBLE CHANGES:

http://git-wip-us.apache.org/repos/asf/tez/blob/fb7c0d61/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 bfc2d58..0f4d812 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
@@ -25,8 +25,6 @@ import java.io.FileInputStream;
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.PrintWriter;
-import java.lang.reflect.Constructor;
-import java.lang.reflect.InvocationTargetException;
 import java.net.URI;
 import java.net.URISyntaxException;
 import java.net.URL;
@@ -139,9 +137,7 @@ import org.apache.tez.dag.app.dag.event.TaskEventType;
 import org.apache.tez.dag.app.dag.event.VertexEvent;
 import org.apache.tez.dag.app.dag.event.VertexEventType;
 import org.apache.tez.dag.app.dag.impl.DAGImpl;
-import org.apache.tez.dag.app.launcher.ContainerLauncher;
-import org.apache.tez.dag.app.launcher.ContainerLauncherImpl;
-import org.apache.tez.dag.app.launcher.LocalContainerLauncher;
+import org.apache.tez.dag.app.launcher.ContainerLauncherRouter;
 import org.apache.tez.dag.app.rm.AMSchedulerEventType;
 import org.apache.tez.dag.app.rm.NMCommunicatorEventType;
 import org.apache.tez.dag.app.rm.TaskSchedulerEventHandler;
@@ -226,7 +222,7 @@ public class DAGAppMaster extends AbstractService {
   private AppContext context;
   private Configuration amConf;
   private AsyncDispatcher dispatcher;
-  private ContainerLauncher containerLauncher;
+  private ContainerLauncherRouter containerLauncherRouter;
   private ContainerHeartbeatHandler containerHeartbeatHandler;
   private TaskHeartbeatHandler taskHeartbeatHandler;
   private TaskAttemptListener taskAttemptListener;
@@ -504,9 +500,9 @@ public class DAGAppMaster extends AbstractService {
         taskSchedulerEventHandler);
     addIfServiceDependency(taskSchedulerEventHandler, clientRpcServer);
 
-    containerLauncher = createContainerLauncher(context);
-    addIfService(containerLauncher, true);
-    dispatcher.register(NMCommunicatorEventType.class, containerLauncher);
+    this.containerLauncherRouter = createContainerLauncherRouter(conf);
+    addIfService(containerLauncherRouter, true);
+    dispatcher.register(NMCommunicatorEventType.class, containerLauncherRouter);
 
     historyEventHandler = createHistoryEventHandler(context);
     addIfService(historyEventHandler, true);
@@ -1034,38 +1030,10 @@ public class DAGAppMaster extends AbstractService {
     return chh;
   }
 
-  protected ContainerLauncher
-      createContainerLauncher(final AppContext context) throws UnknownHostException {
-    if(isLocal){
-      LOG.info("Creating LocalContainerLauncher");
-      return new LocalContainerLauncher(context, taskAttemptListener, workingDirectory);
-    } else {
-      // TODO: Temporary reflection with specific parameters until a clean interface is defined.
-      String containerLauncherClassName = getConfig().get(TezConfiguration.TEZ_AM_CONTAINER_LAUNCHER_CLASS);
-      if (containerLauncherClassName == null) {
-        LOG.info("Creating Default Container Launcher");
-        return new ContainerLauncherImpl(context);
-      } else {
-        LOG.info("Creating container launcher : " + containerLauncherClassName);
-        Class<? extends ContainerLauncher> containerLauncherClazz = (Class<? extends ContainerLauncher>) ReflectionUtils.getClazz(
-            containerLauncherClassName);
-        try {
-          Constructor<? extends ContainerLauncher> ctor = containerLauncherClazz
-              .getConstructor(AppContext.class, Configuration.class, TaskAttemptListener.class);
-          ctor.setAccessible(true);
-          ContainerLauncher instance = ctor.newInstance(context, getConfig(), taskAttemptListener);
-          return instance;
-        } catch (NoSuchMethodException e) {
-          throw new TezUncheckedException(e);
-        } catch (InvocationTargetException e) {
-          throw new TezUncheckedException(e);
-        } catch (InstantiationException e) {
-          throw new TezUncheckedException(e);
-        } catch (IllegalAccessException e) {
-          throw new TezUncheckedException(e);
-        }
-      }
-    }
+  protected ContainerLauncherRouter createContainerLauncherRouter(Configuration conf) throws
+      UnknownHostException {
+    return  new ContainerLauncherRouter(conf, isLocal, context, taskAttemptListener, workingDirectory);
+
   }
 
   public ApplicationId getAppID() {
@@ -1088,8 +1056,8 @@ public class DAGAppMaster extends AbstractService {
     return dispatcher;
   }
 
-  public ContainerLauncher getContainerLauncher() {
-    return containerLauncher;
+  public ContainerLauncherRouter getContainerLauncherRouter() {
+    return containerLauncherRouter;
   }
 
   public TaskAttemptListener getTaskAttemptListener() {

http://git-wip-us.apache.org/repos/asf/tez/blob/fb7c0d61/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncher.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncher.java b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncher.java
index 305eb50..8a8498f 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncher.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncher.java
@@ -23,7 +23,7 @@ import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.tez.dag.app.dag.DAG;
 import org.apache.tez.dag.app.rm.NMCommunicatorEvent;
 
-public interface ContainerLauncher 
+public interface ContainerLauncher
     extends EventHandler<NMCommunicatorEvent> {
 
     void dagComplete(DAG dag);

http://git-wip-us.apache.org/repos/asf/tez/blob/fb7c0d61/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherRouter.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherRouter.java b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherRouter.java
new file mode 100644
index 0000000..34001ed
--- /dev/null
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherRouter.java
@@ -0,0 +1,108 @@
+/*
+ * 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.
+ */
+
+package org.apache.tez.dag.app.launcher;
+
+import java.lang.reflect.Constructor;
+import java.lang.reflect.InvocationTargetException;
+import java.net.UnknownHostException;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.service.AbstractService;
+import org.apache.hadoop.yarn.event.EventHandler;
+import org.apache.tez.common.ReflectionUtils;
+import org.apache.tez.dag.api.TezConfiguration;
+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.app.rm.NMCommunicatorEvent;
+
+public class ContainerLauncherRouter extends AbstractService
+    implements EventHandler<NMCommunicatorEvent> {
+
+  static final Log LOG = LogFactory.getLog(ContainerLauncherImpl.class);
+
+  private final ContainerLauncher containerLauncher;
+
+  @VisibleForTesting
+  public ContainerLauncherRouter(ContainerLauncher containerLauncher) {
+    super(ContainerLauncherRouter.class.getName());
+    this.containerLauncher = containerLauncher;
+  }
+
+  // Accepting conf to setup final parameters, if required.
+  public ContainerLauncherRouter(Configuration conf, boolean isLocal, AppContext context,
+                                 TaskAttemptListener taskAttemptListener,
+                                 String workingDirectory) throws UnknownHostException {
+    super(ContainerLauncherRouter.class.getName());
+
+    if (isLocal) {
+      LOG.info("Creating LocalContainerLauncher");
+      containerLauncher =
+          new LocalContainerLauncher(context, taskAttemptListener, workingDirectory);
+    } else {
+      // TODO: Temporary reflection with specific parameters until a clean interface is defined.
+      String containerLauncherClassName =
+          conf.get(TezConfiguration.TEZ_AM_CONTAINER_LAUNCHER_CLASS);
+      if (containerLauncherClassName == null) {
+        LOG.info("Creating Default Container Launcher");
+        containerLauncher = new ContainerLauncherImpl(context);
+      } else {
+        LOG.info("Creating container launcher : " + containerLauncherClassName);
+        Class<? extends ContainerLauncher> containerLauncherClazz =
+            (Class<? extends ContainerLauncher>) ReflectionUtils.getClazz(
+                containerLauncherClassName);
+        try {
+          Constructor<? extends ContainerLauncher> ctor = containerLauncherClazz
+              .getConstructor(AppContext.class, Configuration.class, TaskAttemptListener.class);
+          ctor.setAccessible(true);
+          containerLauncher = ctor.newInstance(context, conf, taskAttemptListener);
+        } catch (NoSuchMethodException e) {
+          throw new TezUncheckedException(e);
+        } catch (InvocationTargetException e) {
+          throw new TezUncheckedException(e);
+        } catch (InstantiationException e) {
+          throw new TezUncheckedException(e);
+        } catch (IllegalAccessException e) {
+          throw new TezUncheckedException(e);
+        }
+      }
+
+    }
+  }
+
+  @Override
+  public void serviceInit(Configuration conf) {
+    ((AbstractService)containerLauncher).init(conf);
+  }
+
+  @Override
+  public void serviceStart() {
+    ((AbstractService)containerLauncher).start();
+  }
+
+  @Override
+  public void serviceStop() {
+    ((AbstractService)containerLauncher).stop();
+  }
+
+
+  @Override
+  public void handle(NMCommunicatorEvent event) {
+    containerLauncher.handle(event);
+  }
+}

http://git-wip-us.apache.org/repos/asf/tez/blob/fb7c0d61/tez-dag/src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java b/tez-dag/src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java
index 0b2ea2f..2913cee 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java
@@ -57,6 +57,7 @@ import org.apache.tez.dag.api.TezConfiguration;
 import org.apache.tez.dag.api.TaskCommunicator;
 import org.apache.tez.dag.api.TezUncheckedException;
 import org.apache.tez.dag.app.launcher.ContainerLauncher;
+import org.apache.tez.dag.app.launcher.ContainerLauncherRouter;
 import org.apache.tez.dag.app.rm.NMCommunicatorEvent;
 import org.apache.tez.dag.app.rm.NMCommunicatorLaunchRequestEvent;
 import org.apache.tez.dag.app.rm.NMCommunicatorStopRequestEvent;
@@ -514,9 +515,9 @@ public class MockDAGAppMaster extends DAGAppMaster {
   
   // use mock container launcher for tests
   @Override
-  protected ContainerLauncher createContainerLauncher(final AppContext context)
+  protected ContainerLauncherRouter createContainerLauncherRouter(final Configuration conf)
       throws UnknownHostException {
-    return containerLauncher;
+    return new ContainerLauncherRouter(containerLauncher);
   }
 
   @Override


[06/50] [abbrv] tez git commit: Fix CHANGES.txt

Posted by ss...@apache.org.
Fix CHANGES.txt


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

Branch: refs/heads/TEZ-2003
Commit: 748f58da89edc5513600a53d2be2973b60507965
Parents: fcd6bb6
Author: Gopal V <go...@apache.org>
Authored: Fri May 1 19:40:55 2015 +0530
Committer: Gopal V <go...@apache.org>
Committed: Fri May 1 19:40:55 2015 +0530

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


http://git-wip-us.apache.org/repos/asf/tez/blob/748f58da/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index c8f8a3a..3cc9764 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -12,6 +12,7 @@ ALL CHANGES:
   TEZ-2395. Tez UI: Minimum/Maximum Duration show a empty bracket next to 0 secs when you purposefully failed a job.
   TEZ-2360. per-io counters flag should generate both overall and per-edge counters
   TEZ-2389. Tez UI: Sort by attempt-no is incorrect in attempts pages.
+  TEZ-2363: Fix off-by-one error in REDUCE_INPUT_RECORDS counter
   TEZ-2383. Cleanup input/output/processor contexts in LogicalIOProcessorRuntimeTask.
   TEZ-2084. Tez UI: Stacktrace format info is lost in diagnostics
   TEZ-2374. Fix build break against hadoop-2.2 due to TEZ-2325.


[18/50] [abbrv] tez git commit: TEZ-2392. Have all readers throw an Exception on incorrect next() usage (rbalamohan)

Posted by ss...@apache.org.
TEZ-2392. Have all readers throw an Exception on incorrect next() usage (rbalamohan)


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

Branch: refs/heads/TEZ-2003
Commit: b0054628df8d6d01cfed9bf850759ebc39c1e3b7
Parents: 210619a
Author: Rajesh Balamohan <rb...@apache.org>
Authored: Tue May 5 11:02:07 2015 +0530
Committer: Rajesh Balamohan <rb...@apache.org>
Committed: Tue May 5 11:02:07 2015 +0530

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 .../org/apache/tez/mapreduce/input/MRInput.java |   3 +-
 .../tez/mapreduce/lib/MRReaderMapReduce.java    |   5 +-
 .../tez/mapreduce/lib/MRReaderMapred.java       |   3 +
 .../tez/mapreduce/input/TestMultiMRInput.java   |  13 ++
 .../tez/mapreduce/lib/TestKVReadersWithMR.java  | 178 +++++++++++++++++++
 .../tez/runtime/library/api/KeyValueReader.java |  17 ++
 .../runtime/library/api/KeyValuesReader.java    |  17 ++
 .../runtime/library/common/ValuesIterator.java  |  18 ++
 .../common/readers/UnorderedKVReader.java       |  10 +-
 .../input/ConcatenatedMergedKeyValueInput.java  |   4 +-
 .../input/ConcatenatedMergedKeyValuesInput.java |   4 +-
 .../library/input/OrderedGroupedKVInput.java    |   2 +
 .../input/OrderedGroupedMergedKVInput.java      |   3 +
 .../runtime/library/input/UnorderedKVInput.java |   2 +
 .../library/common/TestValuesIterator.java      |  21 ++-
 .../common/readers/TestUnorderedKVReader.java   | 168 +++++++++++++++++
 .../input/TestSortedGroupedMergedInput.java     | 143 ++++++++++++++-
 18 files changed, 595 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/b0054628/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 2ff7601..816c7a5 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -9,6 +9,7 @@ INCOMPATIBLE CHANGES
   TEZ-1993. Implement a pluggable InputSizeEstimator for grouping fairly
 
 ALL CHANGES:
+  TEZ-2392. Have all readers throw an Exception on incorrect next() usage.
   TEZ-2408. TestTaskAttempt fails to compile against hadoop-2.4 and hadoop-2.2.
   TEZ-2405. PipelinedSorter can throw NPE with custom compartor.
   TEZ-1897. Create a concurrent version of AsyncDispatcher

http://git-wip-us.apache.org/repos/asf/tez/blob/b0054628/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 991f6d1..270f68f 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
@@ -493,7 +493,8 @@ public class MRInput extends MRInputBase {
 
   /**
    * Returns a {@link KeyValueReader} that can be used to read 
-   * Map Reduce compatible key value data
+   * Map Reduce compatible key value data. An exception will be thrown if next()
+   * is invoked after false, either from the framework or from the underlying InputFormat
    */
   @Override
   public KeyValueReader getReader() throws IOException {

http://git-wip-us.apache.org/repos/asf/tez/blob/b0054628/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/lib/MRReaderMapReduce.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/lib/MRReaderMapReduce.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/lib/MRReaderMapReduce.java
index 39cd79c..0495751 100644
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/lib/MRReaderMapReduce.java
+++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/lib/MRReaderMapReduce.java
@@ -44,7 +44,7 @@ public class MRReaderMapReduce extends MRReader {
   @SuppressWarnings("rawtypes")
   private final InputFormat inputFormat;
   @SuppressWarnings("rawtypes")
-  private RecordReader recordReader;
+  protected RecordReader recordReader;
   private InputSplit inputSplit;
 
   private boolean setupComplete = false;
@@ -120,6 +120,9 @@ public class MRReaderMapReduce extends MRReader {
     }
     if (hasNext) {
       inputRecordCounter.increment(1);
+    } else {
+      hasCompletedProcessing();
+      completedProcessing = true;
     }
     return hasNext;
   }

http://git-wip-us.apache.org/repos/asf/tez/blob/b0054628/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/lib/MRReaderMapred.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/lib/MRReaderMapred.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/lib/MRReaderMapred.java
index c4ad7a4..366e7a7 100644
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/lib/MRReaderMapred.java
+++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/lib/MRReaderMapred.java
@@ -113,6 +113,9 @@ public class MRReaderMapred extends MRReader {
     boolean hasNext = recordReader.next(key, value);
     if (hasNext) {
       inputRecordCounter.increment(1);
+    } else {
+      hasCompletedProcessing();
+      completedProcessing = true;
     }
     return hasNext;
   }

http://git-wip-us.apache.org/repos/asf/tez/blob/b0054628/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/input/TestMultiMRInput.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/input/TestMultiMRInput.java b/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/input/TestMultiMRInput.java
index 55f6bff..4031140 100644
--- a/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/input/TestMultiMRInput.java
+++ b/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/input/TestMultiMRInput.java
@@ -131,6 +131,12 @@ public class TestMultiMRInput {
         Object val = reader.getCurrentValue();
         assertEquals(val, data1.remove(key));
       }
+      try {
+        boolean hasNext = reader.next(); //should throw exception
+        fail();
+      } catch(IOException e) {
+        assertTrue(e.getMessage().contains("For usage, please refer to"));
+      }
     }
     assertEquals(1, readerCount);
   }
@@ -198,6 +204,13 @@ public class TestMultiMRInput {
         Object val = reader.getCurrentValue();
         assertEquals(val, data.remove(key));
       }
+
+      try {
+        boolean hasNext = reader.next(); //should throw exception
+        fail();
+      } catch(IOException e) {
+        assertTrue(e.getMessage().contains("For usage, please refer to"));
+      }
     }
     assertEquals(2, readerCount);
   }

http://git-wip-us.apache.org/repos/asf/tez/blob/b0054628/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/lib/TestKVReadersWithMR.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/lib/TestKVReadersWithMR.java b/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/lib/TestKVReadersWithMR.java
new file mode 100644
index 0000000..65f5ad0
--- /dev/null
+++ b/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/lib/TestKVReadersWithMR.java
@@ -0,0 +1,178 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.lib;
+
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.RecordReader;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.tez.common.counters.TaskCounter;
+import org.apache.tez.common.counters.TezCounter;
+import org.apache.tez.common.counters.TezCounters;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.IOException;
+
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+public class TestKVReadersWithMR {
+
+  private JobConf conf;
+  private TezCounters counters;
+  private TezCounter inputRecordCounter;
+
+  @Before
+  public void setup() {
+    conf = new JobConf();
+    counters = new TezCounters();
+    inputRecordCounter = counters.findCounter(TaskCounter.INPUT_RECORDS_PROCESSED);
+  }
+
+  @Test(timeout = 10000)
+  public void testMRReaderMapred() throws IOException {
+    //empty
+    testWithSpecificNumberOfKV(0);
+
+    testWithSpecificNumberOfKV(10);
+
+    //empty
+    testWithSpecificNumberOfKV_MapReduce(0);
+
+    testWithSpecificNumberOfKV_MapReduce(10);
+  }
+
+  public void testWithSpecificNumberOfKV(int kvPairs) throws IOException {
+    MRReaderMapred reader = new MRReaderMapred(conf, counters, inputRecordCounter);
+
+    reader.recordReader = new DummyRecordReader(kvPairs);
+    int records = 0;
+    while (reader.next()) {
+      records++;
+    }
+    assertTrue(kvPairs == records);
+
+    //reading again should fail
+    try {
+      boolean hasNext = reader.next();
+      fail();
+    } catch (IOException e) {
+      assertTrue(e.getMessage().contains("For usage, please refer to"));
+    }
+
+  }
+
+  public void testWithSpecificNumberOfKV_MapReduce(int kvPairs) throws IOException {
+    MRReaderMapReduce reader = new MRReaderMapReduce(conf, counters, inputRecordCounter, -1, 1,
+        10, 20, 30);
+
+    reader.recordReader = new DummyRecordReaderMapReduce(kvPairs);
+    int records = 0;
+    while (reader.next()) {
+      records++;
+    }
+    assertTrue(kvPairs == records);
+
+    //reading again should fail
+    try {
+      boolean hasNext = reader.next();
+      fail();
+    } catch (IOException e) {
+      assertTrue(e.getMessage().contains("For usage, please refer to"));
+    }
+  }
+
+  static class DummyRecordReader implements RecordReader {
+    int records;
+
+    public DummyRecordReader(int records) {
+      this.records = records;
+    }
+
+    @Override
+    public boolean next(Object o, Object o2) throws IOException {
+      return (records-- > 0);
+    }
+
+    @Override
+    public Object createKey() {
+      return null;
+    }
+
+    @Override
+    public Object createValue() {
+      return null;
+    }
+
+    @Override
+    public long getPos() throws IOException {
+      return 0;
+    }
+
+    @Override
+    public void close() throws IOException {
+
+    }
+
+    @Override
+    public float getProgress() throws IOException {
+      return 0;
+    }
+  }
+
+  static class DummyRecordReaderMapReduce extends org.apache.hadoop.mapreduce.RecordReader {
+    int records;
+
+    public DummyRecordReaderMapReduce(int records) {
+      this.records = records;
+    }
+
+    @Override
+    public void initialize(InputSplit inputSplit, TaskAttemptContext taskAttemptContext)
+        throws IOException, InterruptedException {
+
+    }
+
+    @Override
+    public boolean nextKeyValue() throws IOException, InterruptedException {
+      return (records-- > 0);
+    }
+
+    @Override
+    public Object getCurrentKey() throws IOException, InterruptedException {
+      return null;
+    }
+
+    @Override
+    public Object getCurrentValue() throws IOException, InterruptedException {
+      return null;
+    }
+
+    @Override
+    public float getProgress() throws IOException, InterruptedException {
+      return 0;
+    }
+
+    @Override
+    public void close() throws IOException {
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/tez/blob/b0054628/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/api/KeyValueReader.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/api/KeyValueReader.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/api/KeyValueReader.java
index 67b6f85..d504d08 100644
--- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/api/KeyValueReader.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/api/KeyValueReader.java
@@ -34,11 +34,15 @@ import org.apache.tez.runtime.api.Reader;
  *   Object value = kvReader.getCurrentValue();
  * </code>
  *
+ * if next() is called after processing everything,
+ * IOException would be thrown
  */
 @Public
 @Evolving
 public abstract class KeyValueReader extends Reader {
 
+  protected boolean completedProcessing;
+
   /**
    * Moves to the next key/values(s) pair
    * 
@@ -62,4 +66,17 @@ public abstract class KeyValueReader extends Reader {
    * @throws IOException
    */
   public abstract Object getCurrentValue() throws IOException;
+
+  /**
+   * Check whether processing has been completed.
+   *
+   * @throws IOException
+   */
+  protected void hasCompletedProcessing() throws IOException {
+    if (completedProcessing) {
+      throw new IOException("Please check if you are"
+          + " invoking next() even after it returned false. For usage, please refer to "
+          + "KeyValueReader javadocs");
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/b0054628/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/api/KeyValuesReader.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/api/KeyValuesReader.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/api/KeyValuesReader.java
index 0bb2777..510f4b7 100644
--- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/api/KeyValuesReader.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/api/KeyValuesReader.java
@@ -34,11 +34,15 @@ import org.apache.tez.runtime.api.Reader;
  *   Iterable<Object> values = kvReader.getCurrentValues();
  * </code>
  *
+ * if next() is called after processing everything,
+ * IOException would be thrown
  */
 @Public
 @Evolving
 public abstract class KeyValuesReader extends Reader {
 
+  protected boolean completedProcessing;
+
   /**
    * Moves to the next key/values(s) pair
    * 
@@ -60,4 +64,17 @@ public abstract class KeyValuesReader extends Reader {
    * @return an Iterable view of the values associated with the current key
    */
   public abstract Iterable<Object> getCurrentValues() throws IOException;
+
+  /**
+   * Check whether processing has been completed.
+   *
+   * @throws IOException
+   */
+  protected void hasCompletedProcessing() throws IOException {
+    if (completedProcessing) {
+      throw new IOException("Please check if you are"
+          + " invoking next() even after it returned false. For usage, please refer to "
+          + "KeyValuesReader javadocs");
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/b0054628/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
index a1f52e7..24f9f8a 100644
--- 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
@@ -63,6 +63,8 @@ public class ValuesIterator<KEY,VALUE> {
   private int keyCtr = 0;
   private boolean hasMoreValues; // For the current key.
   private boolean isFirstRecord = true;
+
+  private boolean completedProcessing;
   
   public ValuesIterator (TezRawKeyValueIterator in, 
                          RawComparator<KEY> comparator, 
@@ -99,6 +101,10 @@ public class ValuesIterator<KEY,VALUE> {
     } else {
       nextKey();
     }
+    if (!more) {
+      hasCompletedProcessing();
+      completedProcessing = true;
+    }
     return more;
   }
 
@@ -206,4 +212,16 @@ public class ValuesIterator<KEY,VALUE> {
         nextValueBytes.getLength() - nextValueBytes.getPosition());
     value = valDeserializer.deserialize(value);
   }
+
+  /**
+   * Check whether processing has been completed.
+   *
+   * @throws IOException
+   */
+  protected void hasCompletedProcessing() throws IOException {
+    if (completedProcessing) {
+      throw new IOException("Please check if you are invoking moveToNext() even after it returned"
+          + " false.");
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/b0054628/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/readers/UnorderedKVReader.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/readers/UnorderedKVReader.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/readers/UnorderedKVReader.java
index 46af66d..b14a461 100644
--- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/readers/UnorderedKVReader.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/readers/UnorderedKVReader.java
@@ -71,8 +71,7 @@ public class UnorderedKVReader<K, V> extends KeyValueReader {
   // the counter at the moment will generate aggregate numbers. 
   private int numRecordsRead = 0;
 
-  private boolean completedProcessing;
-  
+
   public UnorderedKVReader(ShuffleManager shuffleManager, Configuration conf,
       CompressionCodec codec, boolean ifileReadAhead, int ifileReadAheadLength, int ifileBufferSize,
       TezCounter inputRecordCounter)
@@ -131,13 +130,6 @@ public class UnorderedKVReader<K, V> extends KeyValueReader {
     }
   }
 
-  private void hasCompletedProcessing() throws IOException {
-    if (completedProcessing) {
-      throw new IOException("Reader has already processed all the inputs. Please check if you are"
-          + " invoking next() even after it returned false. For usage, please refer to "
-          + "KeyValueReader javadocs");
-    }
-  }
 
   @Override
   public Object getCurrentKey() throws IOException {

http://git-wip-us.apache.org/repos/asf/tez/blob/b0054628/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/ConcatenatedMergedKeyValueInput.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/ConcatenatedMergedKeyValueInput.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/ConcatenatedMergedKeyValueInput.java
index 39e0fff..14b1e2c 100644
--- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/ConcatenatedMergedKeyValueInput.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/ConcatenatedMergedKeyValueInput.java
@@ -46,11 +46,13 @@ public class ConcatenatedMergedKeyValueInput extends MergedLogicalInput {
   public class ConcatenatedMergedKeyValueReader extends KeyValueReader {
     private int currentReaderIndex = 0;
     private KeyValueReader currentReader;
-    
+
     @Override
     public boolean next() throws IOException {
       while ((currentReader == null) || !currentReader.next()) {
         if (currentReaderIndex == getInputs().size()) {
+          hasCompletedProcessing();
+          completedProcessing = true;
           return false;
         }
         try {

http://git-wip-us.apache.org/repos/asf/tez/blob/b0054628/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/ConcatenatedMergedKeyValuesInput.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/ConcatenatedMergedKeyValuesInput.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/ConcatenatedMergedKeyValuesInput.java
index 0cc3244..2a1e4c6 100644
--- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/ConcatenatedMergedKeyValuesInput.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/ConcatenatedMergedKeyValuesInput.java
@@ -47,11 +47,13 @@ public class ConcatenatedMergedKeyValuesInput extends MergedLogicalInput {
   public class ConcatenatedMergedKeyValuesReader extends KeyValuesReader {
     private int currentReaderIndex = 0;
     private KeyValuesReader currentReader;
-    
+
     @Override
     public boolean next() throws IOException {
       while ((currentReader == null) || !currentReader.next()) {
         if (currentReaderIndex == getInputs().size()) {
+          hasCompletedProcessing();
+          completedProcessing = true;
           return false;
         }
         try {

http://git-wip-us.apache.org/repos/asf/tez/blob/b0054628/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/OrderedGroupedKVInput.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/OrderedGroupedKVInput.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/OrderedGroupedKVInput.java
index e61dbdc..d784fcd 100644
--- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/OrderedGroupedKVInput.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/OrderedGroupedKVInput.java
@@ -218,6 +218,8 @@ public class OrderedGroupedKVInput extends AbstractLogicalInput {
         return new KeyValuesReader() {
           @Override
           public boolean next() throws IOException {
+            hasCompletedProcessing();
+            completedProcessing = true;
             return false;
           }
 

http://git-wip-us.apache.org/repos/asf/tez/blob/b0054628/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/OrderedGroupedMergedKVInput.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/OrderedGroupedMergedKVInput.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/OrderedGroupedMergedKVInput.java
index 9adac54..41ca7c9 100644
--- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/OrderedGroupedMergedKVInput.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/OrderedGroupedMergedKVInput.java
@@ -126,6 +126,9 @@ public class OrderedGroupedMergedKVInput extends MergedLogicalInput {
         currentKey = nextKVReader.getCurrentKey();
         currentValues.moveToNext();
         return true;
+      } else {
+        hasCompletedProcessing();
+        completedProcessing = true;
       }
       return false;
     }

http://git-wip-us.apache.org/repos/asf/tez/blob/b0054628/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/UnorderedKVInput.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/UnorderedKVInput.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/UnorderedKVInput.java
index ce27103..62fa9a5 100644
--- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/UnorderedKVInput.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/UnorderedKVInput.java
@@ -164,6 +164,8 @@ public class UnorderedKVInput extends AbstractLogicalInput {
       return new KeyValueReader() {
         @Override
         public boolean next() throws IOException {
+          hasCompletedProcessing();
+          completedProcessing = true;
           return false;
         }
 

http://git-wip-us.apache.org/repos/asf/tez/blob/b0054628/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/TestValuesIterator.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/TestValuesIterator.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/TestValuesIterator.java
index e1718c8..edb9b15 100644
--- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/TestValuesIterator.java
+++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/TestValuesIterator.java
@@ -65,6 +65,7 @@ import java.util.TreeMap;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
 import static org.mockito.Mockito.doReturn;
 import static org.mockito.Mockito.mock;
 
@@ -208,10 +209,19 @@ public class TestValuesIterator {
   @Test(timeout = 20000)
   public void testIteratorWithIFileReaderEmptyPartitions() throws IOException {
     ValuesIterator iterator = createEmptyIterator(false);
-    assert(iterator.moveToNext() == false);
+    assertTrue(iterator.moveToNext() == false);
 
     iterator = createEmptyIterator(true);
-    assert(iterator.moveToNext() == false);
+    assertTrue(iterator.moveToNext() == false);
+  }
+
+  private void getNextFromFinishedIterator(ValuesIterator iterator) {
+    try {
+      boolean hasNext = iterator.moveToNext();
+      fail();
+    } catch(IOException e) {
+      assertTrue(e.getMessage().contains("Please check if you are invoking moveToNext()"));
+    }
   }
 
   private ValuesIterator createEmptyIterator(boolean inMemory) throws IOException {
@@ -292,7 +302,14 @@ public class TestValuesIterator {
     }
     if (expectedTestResult) {
       assertTrue(result);
+
+      assertFalse(valuesIterator.moveToNext());
+      getNextFromFinishedIterator(valuesIterator);
     } else {
+      while(valuesIterator.moveToNext()) {
+        //iterate through all keys
+      }
+      getNextFromFinishedIterator(valuesIterator);
       assertFalse(result);
     }
 

http://git-wip-us.apache.org/repos/asf/tez/blob/b0054628/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/readers/TestUnorderedKVReader.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/readers/TestUnorderedKVReader.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/readers/TestUnorderedKVReader.java
new file mode 100644
index 0000000..51ea42d
--- /dev/null
+++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/readers/TestUnorderedKVReader.java
@@ -0,0 +1,168 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.readers;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.Text;
+import org.apache.tez.common.counters.TaskCounter;
+import org.apache.tez.common.counters.TezCounter;
+import org.apache.tez.common.counters.TezCounters;
+import org.apache.tez.runtime.library.api.TezRuntimeConfiguration;
+import org.apache.tez.runtime.library.common.InputAttemptIdentifier;
+import org.apache.tez.runtime.library.common.shuffle.FetchedInput;
+import org.apache.tez.runtime.library.common.shuffle.FetchedInputCallback;
+import org.apache.tez.runtime.library.common.shuffle.LocalDiskFetchedInput;
+import org.apache.tez.runtime.library.common.shuffle.impl.ShuffleManager;
+import org.apache.tez.runtime.library.common.sort.impl.IFile;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.LinkedList;
+
+import static junit.framework.TestCase.fail;
+import static org.mockito.Mockito.doAnswer;
+import static org.mockito.Mockito.doNothing;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.spy;
+
+public class TestUnorderedKVReader {
+
+  private static final Logger LOG = LoggerFactory.getLogger(TestUnorderedKVReader.class);
+
+  private static Configuration defaultConf = new Configuration();
+  private static FileSystem localFs = null;
+  private static Path workDir = null;
+
+  private String outputFileName = "ifile.out";
+  private Path outputPath;
+  private long rawLen;
+  private long compLen;
+
+  private UnorderedKVReader<Text, Text> unorderedKVReader;
+
+  static {
+    defaultConf.set("fs.defaultFS", "file:///");
+    try {
+      localFs = FileSystem.getLocal(defaultConf);
+      workDir = new Path(
+          new Path(System.getProperty("test.build.data", "/tmp")),
+          TestUnorderedKVReader.class.getName())
+          .makeQualified(localFs.getUri(), localFs.getWorkingDirectory());
+      LOG.info("Using workDir: " + workDir);
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  @Before
+  public void setUp() throws Exception {
+    outputPath = new Path(workDir, outputFileName);
+    setupReader();
+  }
+
+  private void setupReader() throws IOException, InterruptedException {
+    defaultConf.set(TezRuntimeConfiguration.TEZ_RUNTIME_KEY_CLASS, Text.class.getName());
+    defaultConf.set(TezRuntimeConfiguration.TEZ_RUNTIME_VALUE_CLASS, Text.class.getName());
+
+    createIFile(outputPath, 1);
+
+    final LinkedList<LocalDiskFetchedInput> inputs = new LinkedList<LocalDiskFetchedInput>();
+    LocalDiskFetchedInput realFetchedInput = new LocalDiskFetchedInput(0, rawLen, compLen, new
+        InputAttemptIdentifier(0, 0), outputPath, defaultConf, new FetchedInputCallback() {
+      @Override
+      public void fetchComplete(FetchedInput fetchedInput) {
+      }
+
+      @Override
+      public void fetchFailed(FetchedInput fetchedInput) {
+      }
+
+      @Override
+      public void freeResources(FetchedInput fetchedInput) {
+      }
+    });
+    LocalDiskFetchedInput fetchedInput = spy(realFetchedInput);
+    doNothing().when(fetchedInput).free();
+
+    inputs.add(fetchedInput);
+
+    TezCounters counters = new TezCounters();
+    TezCounter inputRecords = counters.findCounter(TaskCounter.INPUT_RECORDS_PROCESSED);
+
+    ShuffleManager manager = mock(ShuffleManager.class);
+    doAnswer(new Answer() {
+      @Override public Object answer(InvocationOnMock invocationOnMock) throws Throwable {
+        return (inputs.isEmpty()) ? null : inputs.remove();
+      }
+    }).when(manager).getNextInput();
+
+    unorderedKVReader = new UnorderedKVReader<Text, Text>(manager,
+        defaultConf, null, false, -1, -1, inputRecords);
+  }
+
+  private void createIFile(Path path, int recordCount) throws IOException {
+    FSDataOutputStream out = localFs.create(path);
+    IFile.Writer writer =
+        new IFile.Writer(defaultConf, out, Text.class, Text.class, null, null, null, true);
+
+    for (int i = 0; i < recordCount; i++) {
+      writer.append(new Text("Key_" + i), new Text("Value_" + i));
+    }
+    writer.close();
+    rawLen = writer.getRawLength();
+    compLen = writer.getCompressedLength();
+    out.close();
+  }
+
+  @Before
+  @After
+  public void cleanup() throws Exception {
+    localFs.delete(workDir, true);
+  }
+
+  @Test(timeout = 5000)
+  public void testReadingMultipleTimes() throws Exception {
+    int counter = 0;
+    while (unorderedKVReader.next()) {
+      unorderedKVReader.getCurrentKey();
+      unorderedKVReader.getCurrentKey();
+      counter++;
+    }
+    Assert.assertEquals(1, counter);
+
+    //Check the reader again. This shouldn't throw EOF exception in IFile
+    try {
+      boolean next = unorderedKVReader.next();
+      fail();
+    } catch(IOException ioe) {
+      Assert.assertTrue(ioe.getMessage().contains("For usage, please refer to"));
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/tez/blob/b0054628/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/input/TestSortedGroupedMergedInput.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/input/TestSortedGroupedMergedInput.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/input/TestSortedGroupedMergedInput.java
index 570deb7..0de400e 100644
--- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/input/TestSortedGroupedMergedInput.java
+++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/input/TestSortedGroupedMergedInput.java
@@ -19,7 +19,7 @@
 package org.apache.tez.runtime.library.input;
 
 import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 import static org.mockito.Mockito.mock;
 
@@ -35,7 +35,9 @@ import org.apache.tez.runtime.api.Event;
 import org.apache.tez.runtime.api.Input;
 import org.apache.tez.runtime.api.MergedLogicalInput;
 import org.apache.tez.runtime.api.MergedInputContext;
+import org.apache.tez.runtime.api.Reader;
 import org.apache.tez.runtime.api.impl.TezMergedInputContextImpl;
+import org.apache.tez.runtime.library.api.KeyValueReader;
 import org.apache.tez.runtime.library.api.KeyValuesReader;
 import org.junit.Test;
 
@@ -82,6 +84,18 @@ public class TestSortedGroupedMergedInput {
       }
       assertEquals(6, valCount);
     }
+
+    getNextFromFinishedReader(kvsReader);
+  }
+
+  private void getNextFromFinishedReader(KeyValuesReader kvsReader) {
+    //Try reading again and it should throw IOException
+    try {
+      boolean hasNext = kvsReader.next();
+      fail();
+    } catch(IOException e) {
+      assertTrue(e.getMessage().contains("For usage, please refer to"));
+    }
   }
 
   @Test(timeout = 5000)
@@ -126,6 +140,7 @@ public class TestSortedGroupedMergedInput {
       }
       assertEquals(6, valCount);
     }
+    getNextFromFinishedReader(kvsReader);
   }
 
   @Test(timeout = 5000)
@@ -172,6 +187,7 @@ public class TestSortedGroupedMergedInput {
         assertEquals(6, valCount);
       }
     }
+    getNextFromFinishedReader(kvsReader);
   }
 
   @Test(timeout = 5000)
@@ -223,6 +239,7 @@ public class TestSortedGroupedMergedInput {
         fail("Unexpected key");
       }
     }
+    getNextFromFinishedReader(kvsReader);
   }
 
   @Test(timeout = 5000)
@@ -277,6 +294,7 @@ public class TestSortedGroupedMergedInput {
         fail("Unexpected key");
       }
     }
+    getNextFromFinishedReader(kvsReader);
   }
   
   // Reads all values for a key, but doesn't trigger the last hasNext() call.
@@ -324,6 +342,7 @@ public class TestSortedGroupedMergedInput {
       }
       assertEquals(6, valCount);
     }
+    getNextFromFinishedReader(kvsReader);
   }
 
   @Test(timeout = 5000)
@@ -350,7 +369,84 @@ public class TestSortedGroupedMergedInput {
     OrderedGroupedMergedKVInput input = new OrderedGroupedMergedKVInput(createMergedInputContext(), sInputs);
 
     KeyValuesReader kvsReader = input.getReader();
-    assertFalse(kvsReader.next());
+    assertTrue(kvsReader.next() == false);
+    getNextFromFinishedReader(kvsReader);
+  }
+
+  @Test(timeout = 5000)
+  public void testSimpleConcatenatedMergedKeyValueInput() throws Exception {
+
+    DummyInput sInput1 = new DummyInput(10);
+    DummyInput sInput2 = new DummyInput(10);
+    DummyInput sInput3 = new DummyInput(10);
+
+    List<Input> sInputs = new LinkedList<Input>();
+    sInputs.add(sInput1);
+    sInputs.add(sInput2);
+    sInputs.add(sInput3);
+    ConcatenatedMergedKeyValueInput input =
+        new ConcatenatedMergedKeyValueInput(createMergedInputContext(), sInputs);
+
+    KeyValueReader kvReader = input.getReader();
+    int keyCount = 0;
+    while (kvReader.next()) {
+      keyCount++;
+      Integer key = (Integer) kvReader.getCurrentKey();
+      Integer value = (Integer) kvReader.getCurrentValue();
+    }
+    assertTrue(keyCount == 30);
+
+    getNextFromFinishedReader(kvReader);
+  }
+
+  @Test(timeout = 5000)
+  public void testSimpleConcatenatedMergedKeyValuesInput() throws Exception {
+    SortedTestKeyValuesReader kvsReader1 = new SortedTestKeyValuesReader(new int[] { 1, 2, 3 },
+        new int[][] { { 1, 1 }, { 2, 2 }, { 3, 3 } });
+
+    SortedTestKeyValuesReader kvsReader2 = new SortedTestKeyValuesReader(new int[] { 1, 2, 3 },
+        new int[][] { { 1, 1 }, { 2, 2 }, { 3, 3 } });
+
+    SortedTestKeyValuesReader kvsReader3 = new SortedTestKeyValuesReader(new int[] { 1, 2, 3 },
+        new int[][] { { 1, 1 }, { 2, 2 }, { 3, 3 } });
+
+    SortedTestInput sInput1 = new SortedTestInput(kvsReader1);
+    SortedTestInput sInput2 = new SortedTestInput(kvsReader2);
+    SortedTestInput sInput3 = new SortedTestInput(kvsReader3);
+
+    List<Input> sInputs = new LinkedList<Input>();
+    sInputs.add(sInput1);
+    sInputs.add(sInput2);
+    sInputs.add(sInput3);
+    ConcatenatedMergedKeyValuesInput input =
+        new ConcatenatedMergedKeyValuesInput(createMergedInputContext(), sInputs);
+
+    KeyValuesReader kvsReader = input.getReader();
+    int keyCount = 0;
+    while (kvsReader.next()) {
+      keyCount++;
+      Integer key = (Integer) kvsReader.getCurrentKey();
+      Iterator<Object> valuesIter = kvsReader.getCurrentValues().iterator();
+      int valCount = 0;
+      while (valuesIter.hasNext()) {
+        valCount++;
+        Integer val = (Integer) valuesIter.next();
+      }
+      assertEquals(2, valCount);
+    }
+    assertEquals(9, keyCount);
+
+    getNextFromFinishedReader(kvsReader);
+  }
+
+  private void getNextFromFinishedReader(KeyValueReader kvReader) {
+    //Try reading again and it should throw IOException
+    try {
+      boolean hasNext = kvReader.next();
+      fail();
+    } catch(IOException e) {
+      assertTrue(e.getMessage().contains("For usage, please refer to"));
+    }
   }
 
   private static class SortedTestInput extends OrderedGroupedKVInput {
@@ -404,8 +500,10 @@ public class TestSortedGroupedMergedInput {
 
     @Override
     public boolean next() throws IOException {
+      hasCompletedProcessing();
       currentIndex++;
       if (keys == null || currentIndex >= keys.length) {
+        completedProcessing = true;
         return false;
       }
       return true;
@@ -426,6 +524,47 @@ public class TestSortedGroupedMergedInput {
     }
   }
 
+  private static class DummyInput implements Input {
+    DummyKeyValueReader reader;
+
+    public DummyInput(int records) {
+      reader = new DummyKeyValueReader(records);
+    }
+
+    @Override
+    public void start() throws Exception {
+    }
+
+    @Override
+    public Reader getReader() throws Exception {
+      return reader;
+    }
+  }
+
+  private static class DummyKeyValueReader extends KeyValueReader {
+    private int records;
+
+    public DummyKeyValueReader(int records) {
+      this.records = records;
+    }
+
+    @Override
+    public boolean next() throws IOException {
+      return (records-- > 0);
+    }
+
+    @Override
+    public Object getCurrentKey() throws IOException {
+      return records;
+    }
+
+    @Override
+    public Object getCurrentValue() throws IOException {
+      return records;
+    }
+  }
+
+
   private static class RawComparatorForTest implements RawComparator<Integer> {
 
     @Override


[07/50] [abbrv] tez git commit: TEZ-2386. Tez UI: Inconsistent usage of icon colors (pramachandran)

Posted by ss...@apache.org.
TEZ-2386. Tez UI: Inconsistent usage of icon colors (pramachandran)


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

Branch: refs/heads/TEZ-2003
Commit: 1dd5f9fdae99ebb6d1351d2ac49a16065251d689
Parents: 748f58d
Author: Prakash Ramachandran <pr...@hortonworks.com>
Authored: Fri May 1 19:40:57 2015 +0530
Committer: Prakash Ramachandran <pr...@hortonworks.com>
Committed: Fri May 1 19:41:46 2015 +0530

----------------------------------------------------------------------
 CHANGES.txt                                       |  1 +
 .../org/apache/tez/dag/app/dag/impl/TaskImpl.java |  4 ++--
 .../tez/dag/history/events/TaskFinishedEvent.java |  8 +++++++-
 .../tez/dag/app/dag/impl/TestTaskRecovery.java    |  4 ++--
 .../events/TestHistoryEventsProtoConversion.java  |  4 ++--
 .../impl/TestHistoryEventJsonConversion.java      |  2 +-
 .../ats/HistoryEventTimelineConversion.java       |  1 +
 .../ats/TestHistoryEventTimelineConversion.java   |  7 ++++---
 .../app/scripts/components/dag-view/graph-view.js |  3 ++-
 .../scripts/controllers/dag_index_controller.js   | 17 +++++++++--------
 .../webapp/app/scripts/controllers/dag_tasks.js   |  5 +++--
 .../app/scripts/controllers/dag_vertices.js       |  5 +++--
 .../app/scripts/controllers/dags_controller.js    |  5 +++--
 .../scripts/controllers/task_index_controller.js  |  7 ++++---
 .../app/scripts/controllers/tasks_controller.js   |  5 +++--
 .../controllers/tez-app-dags-controller.js        |  5 +++--
 .../controllers/vertex_index_controller.js        |  5 +++--
 .../controllers/vertex_tasks_controller.js        |  5 +++--
 .../app/scripts/models/TimelineRestAdapter.js     | 18 ++++++++++++++++++
 tez-ui/src/main/webapp/app/scripts/models/dag.js  | 16 ++++++++++++++++
 tez-ui/src/main/webapp/app/styles/main.less       | 10 ++++++----
 .../src/main/webapp/app/templates/dag/index.hbs   |  3 +++
 .../main/webapp/app/templates/vertex/index.hbs    |  6 ++++++
 23 files changed, 105 insertions(+), 41 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/1dd5f9fd/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 3cc9764..fe90418 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -9,6 +9,7 @@ INCOMPATIBLE CHANGES
   TEZ-1993. Implement a pluggable InputSizeEstimator for grouping fairly
 
 ALL CHANGES:
+  TEZ-2386. Tez UI: Inconsistent usage of icon colors
   TEZ-2395. Tez UI: Minimum/Maximum Duration show a empty bracket next to 0 secs when you purposefully failed a job.
   TEZ-2360. per-io counters flag should generate both overall and per-edge counters
   TEZ-2389. Tez UI: Sort by attempt-no is incorrect in attempts pages.

http://git-wip-us.apache.org/repos/asf/tez/blob/1dd5f9fd/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 91413a5..461339b 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
@@ -948,7 +948,7 @@ public class TaskImpl implements Task, EventHandler<TaskEvent> {
     TaskFinishedEvent finishEvt = new TaskFinishedEvent(taskId,
         getVertex().getName(), getLaunchTime(), clock.getTime(),
         successfulAttempt,
-        TaskState.SUCCEEDED, "", getCounters());
+        TaskState.SUCCEEDED, "", getCounters(), failedAttempts);
     this.appContext.getHistoryHandler().handle(
         new DAGHistoryEvent(taskId.getVertexID().getDAGId(), finishEvt));
   }
@@ -958,7 +958,7 @@ public class TaskImpl implements Task, EventHandler<TaskEvent> {
         getVertex().getName(), getLaunchTime(), clock.getTime(), null,
         finalState, 
         StringUtils.join(getDiagnostics(), LINE_SEPARATOR),
-        getCounters());
+        getCounters(), failedAttempts);
     this.appContext.getHistoryHandler().handle(
         new DAGHistoryEvent(taskId.getVertexID().getDAGId(), finishEvt));
   }

http://git-wip-us.apache.org/repos/asf/tez/blob/1dd5f9fd/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskFinishedEvent.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskFinishedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskFinishedEvent.java
index fa4f8ca..71ff6c8 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskFinishedEvent.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskFinishedEvent.java
@@ -44,11 +44,12 @@ public class TaskFinishedEvent implements HistoryEvent {
   private TezCounters tezCounters;
   private TezTaskAttemptID successfulAttemptID;
   private String diagnostics;
+  private int numFailedAttempts;
 
   public TaskFinishedEvent(TezTaskID taskID,
       String vertexName, long startTime, long finishTime,
       TezTaskAttemptID successfulAttemptID,
-      TaskState state, String diagnostics, TezCounters counters) {
+      TaskState state, String diagnostics, TezCounters counters, int failedAttempts) {
     this.vertexName = vertexName;
     this.taskID = taskID;
     this.startTime = startTime;
@@ -57,6 +58,7 @@ public class TaskFinishedEvent implements HistoryEvent {
     this.diagnostics = diagnostics;
     this.tezCounters = counters;
     this.successfulAttemptID = successfulAttemptID;
+    this.numFailedAttempts = failedAttempts;
   }
 
   public TaskFinishedEvent() {
@@ -161,4 +163,8 @@ public class TaskFinishedEvent implements HistoryEvent {
   public String getDiagnostics() {
     return diagnostics;
   }
+
+  public int getNumFailedAttempts() {
+    return numFailedAttempts;
+  }
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/1dd5f9fd/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskRecovery.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskRecovery.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskRecovery.java
index c2185d8..e182f24 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskRecovery.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskRecovery.java
@@ -240,7 +240,7 @@ public class TestTaskRecovery {
     try {
       task.restoreFromEvent(new TaskFinishedEvent(task.getTaskId(), vertexName,
           taskStartTime, taskFinishTime, null, TaskState.SUCCEEDED, "",
-          new TezCounters()));
+          new TezCounters(), 0));
       fail("Should fail due to no TaskStartEvent before TaskFinishEvent");
     } catch (Throwable e) {
       assertTrue(e.getMessage().contains(
@@ -413,7 +413,7 @@ public class TestTaskRecovery {
     recoveredState =
         task.restoreFromEvent(new TaskFinishedEvent(task.getTaskId(),
             vertexName, taskStartTime, taskFinishTime, taId,
-            TaskState.SUCCEEDED, "", new TezCounters()));
+            TaskState.SUCCEEDED, "", new TezCounters(), 0));
     assertEquals(TaskState.SUCCEEDED, recoveredState);
     assertEquals(taId, task.successfulAttempt);
 

http://git-wip-us.apache.org/repos/asf/tez/blob/1dd5f9fd/tez-dag/src/test/java/org/apache/tez/dag/history/events/TestHistoryEventsProtoConversion.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/history/events/TestHistoryEventsProtoConversion.java b/tez-dag/src/test/java/org/apache/tez/dag/history/events/TestHistoryEventsProtoConversion.java
index 302700c..1575adc 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/history/events/TestHistoryEventsProtoConversion.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/history/events/TestHistoryEventsProtoConversion.java
@@ -424,7 +424,7 @@ public class TestHistoryEventsProtoConversion {
       TaskFinishedEvent event = new TaskFinishedEvent(
           TezTaskID.getInstance(TezVertexID.getInstance(
               TezDAGID.getInstance(ApplicationId.newInstance(0, 1), 1), 111), 1),
-          "vertex1", 11000l, 1000000l, null, TaskState.FAILED, null, null);
+          "vertex1", 11000l, 1000000l, null, TaskState.FAILED, null, null, 0);
       TaskFinishedEvent deserializedEvent = (TaskFinishedEvent)
           testProtoConversion(event);
       Assert.assertEquals(event.getTaskID(), deserializedEvent.getTaskID());
@@ -444,7 +444,7 @@ public class TestHistoryEventsProtoConversion {
           "vertex1", 11000l, 1000000l,
           TezTaskAttemptID.getInstance(TezTaskID.getInstance(TezVertexID.getInstance(
               TezDAGID.getInstance(ApplicationId.newInstance(0, 1), 1), 111), 1), 1),
-          TaskState.FAILED, "task_diagnostics", new TezCounters());
+          TaskState.FAILED, "task_diagnostics", new TezCounters(), 0);
       TaskFinishedEvent deserializedEvent = (TaskFinishedEvent)
           testProtoConversion(event);
       Assert.assertEquals(event.getTaskID(), deserializedEvent.getTaskID());

http://git-wip-us.apache.org/repos/asf/tez/blob/1dd5f9fd/tez-dag/src/test/java/org/apache/tez/dag/history/logging/impl/TestHistoryEventJsonConversion.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/history/logging/impl/TestHistoryEventJsonConversion.java b/tez-dag/src/test/java/org/apache/tez/dag/history/logging/impl/TestHistoryEventJsonConversion.java
index c6749af..6469e78 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/history/logging/impl/TestHistoryEventJsonConversion.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/history/logging/impl/TestHistoryEventJsonConversion.java
@@ -155,7 +155,7 @@ public class TestHistoryEventJsonConversion {
           break;
         case TASK_FINISHED:
           event = new TaskFinishedEvent(tezTaskID, "v1", random.nextInt(), random.nextInt(),
-              tezTaskAttemptID, TaskState.FAILED, null, null);
+              tezTaskAttemptID, TaskState.FAILED, null, null, 0);
           break;
         case TASK_ATTEMPT_STARTED:
           event = new TaskAttemptStartedEvent(tezTaskAttemptID, "v1", random.nextInt(), containerId,

http://git-wip-us.apache.org/repos/asf/tez/blob/1dd5f9fd/tez-plugins/tez-yarn-timeline-history/src/main/java/org/apache/tez/dag/history/logging/ats/HistoryEventTimelineConversion.java
----------------------------------------------------------------------
diff --git a/tez-plugins/tez-yarn-timeline-history/src/main/java/org/apache/tez/dag/history/logging/ats/HistoryEventTimelineConversion.java b/tez-plugins/tez-yarn-timeline-history/src/main/java/org/apache/tez/dag/history/logging/ats/HistoryEventTimelineConversion.java
index 7c804f5..77f4dd1 100644
--- a/tez-plugins/tez-yarn-timeline-history/src/main/java/org/apache/tez/dag/history/logging/ats/HistoryEventTimelineConversion.java
+++ b/tez-plugins/tez-yarn-timeline-history/src/main/java/org/apache/tez/dag/history/logging/ats/HistoryEventTimelineConversion.java
@@ -493,6 +493,7 @@ public class HistoryEventTimelineConversion {
     atsEntity.addOtherInfo(ATSConstants.FINISH_TIME, event.getFinishTime());
     atsEntity.addOtherInfo(ATSConstants.TIME_TAKEN, (event.getFinishTime() - event.getStartTime()));
     atsEntity.addOtherInfo(ATSConstants.STATUS, event.getState().name());
+    atsEntity.addOtherInfo(ATSConstants.NUM_FAILED_TASKS_ATTEMPTS, event.getNumFailedAttempts());
     if (event.getSuccessfulAttemptID() != null) {
       atsEntity.addOtherInfo(ATSConstants.SUCCESSFUL_ATTEMPT_ID,
           event.getSuccessfulAttemptID().toString());

http://git-wip-us.apache.org/repos/asf/tez/blob/1dd5f9fd/tez-plugins/tez-yarn-timeline-history/src/test/java/org/apache/tez/dag/history/logging/ats/TestHistoryEventTimelineConversion.java
----------------------------------------------------------------------
diff --git a/tez-plugins/tez-yarn-timeline-history/src/test/java/org/apache/tez/dag/history/logging/ats/TestHistoryEventTimelineConversion.java b/tez-plugins/tez-yarn-timeline-history/src/test/java/org/apache/tez/dag/history/logging/ats/TestHistoryEventTimelineConversion.java
index 3d2b662..f5f3ae7 100644
--- a/tez-plugins/tez-yarn-timeline-history/src/test/java/org/apache/tez/dag/history/logging/ats/TestHistoryEventTimelineConversion.java
+++ b/tez-plugins/tez-yarn-timeline-history/src/test/java/org/apache/tez/dag/history/logging/ats/TestHistoryEventTimelineConversion.java
@@ -160,7 +160,7 @@ public class TestHistoryEventTimelineConversion {
           break;
         case TASK_FINISHED:
           event = new TaskFinishedEvent(tezTaskID, "v1", random.nextInt(), random.nextInt(),
-              tezTaskAttemptID, TaskState.FAILED, null, null);
+              tezTaskAttemptID, TaskState.FAILED, null, null, 0);
           break;
         case TASK_ATTEMPT_STARTED:
           event = new TaskAttemptStartedEvent(tezTaskAttemptID, "v1", random.nextInt(), containerId,
@@ -778,7 +778,7 @@ public class TestHistoryEventTimelineConversion {
     TezCounters counters = new TezCounters();
 
     TaskFinishedEvent event = new TaskFinishedEvent(tezTaskID, vertexName, startTime, finishTime,
-        tezTaskAttemptID, state, diagnostics, counters);
+        tezTaskAttemptID, state, diagnostics, counters, 3);
     TimelineEntity timelineEntity = HistoryEventTimelineConversion.convertToTimelineEntity(event);
 
     Assert.assertEquals(tezTaskID.toString(), timelineEntity.getEntityId());
@@ -800,12 +800,13 @@ public class TestHistoryEventTimelineConversion {
     Assert.assertEquals(finishTime, evt.getTimestamp());
 
     final Map<String, Object> otherInfo = timelineEntity.getOtherInfo();
-    Assert.assertEquals(6, otherInfo.size());
+    Assert.assertEquals(7, otherInfo.size());
     Assert.assertEquals(finishTime, otherInfo.get(ATSConstants.FINISH_TIME));
     Assert.assertEquals(finishTime - startTime, otherInfo.get(ATSConstants.TIME_TAKEN));
     Assert.assertEquals(state.name(), otherInfo.get(ATSConstants.STATUS));
     Assert.assertEquals(tezTaskAttemptID.toString(),
         otherInfo.get(ATSConstants.SUCCESSFUL_ATTEMPT_ID));
+    Assert.assertEquals(3, otherInfo.get(ATSConstants.NUM_FAILED_TASKS_ATTEMPTS));
     Assert.assertEquals(diagnostics, otherInfo.get(ATSConstants.DIAGNOSTICS));
     Assert.assertTrue(otherInfo.containsKey(ATSConstants.COUNTERS));
   }

http://git-wip-us.apache.org/repos/asf/tez/blob/1dd5f9fd/tez-ui/src/main/webapp/app/scripts/components/dag-view/graph-view.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/scripts/components/dag-view/graph-view.js b/tez-ui/src/main/webapp/app/scripts/components/dag-view/graph-view.js
index 45202ae..f3a87fd 100644
--- a/tez-ui/src/main/webapp/app/scripts/components/dag-view/graph-view.js
+++ b/tez-ui/src/main/webapp/app/scripts/components/dag-view/graph-view.js
@@ -223,7 +223,8 @@ App.DagViewComponent.graphView = (function (){
    */
   function _addStatusBar(node, d) {
     var group = node.append('g'),
-        statusIcon = App.Helpers.misc.getStatusClassForEntity(d.get('data.status'));
+        statusIcon = App.Helpers.misc.getStatusClassForEntity(d.get('data.status'),
+          d.get('data.hasFailedTaskAttempts'));
     group.attr('class', 'status-bar');
 
     group.append('foreignObject')

http://git-wip-us.apache.org/repos/asf/tez/blob/1dd5f9fd/tez-ui/src/main/webapp/app/scripts/controllers/dag_index_controller.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/scripts/controllers/dag_index_controller.js b/tez-ui/src/main/webapp/app/scripts/controllers/dag_index_controller.js
index 416f500..2deab2d 100644
--- a/tez-ui/src/main/webapp/app/scripts/controllers/dag_index_controller.js
+++ b/tez-ui/src/main/webapp/app/scripts/controllers/dag_index_controller.js
@@ -75,8 +75,9 @@ App.DagIndexController = Em.ObjectController.extend(App.ModelRefreshMixin, {
   },
 
   taskIconStatus: function() {
-    return App.Helpers.misc.getStatusClassForEntity(this.get('model.status'));
-  }.property('id', 'status', 'counterGroups'),
+    return App.Helpers.misc.getStatusClassForEntity(this.get('model.status'),
+      this.get('hasFailedTaskAttempts'));
+  }.property('id', 'model.status', 'hasFailedTaskAttempts'),
 
   progressStr: function() {
     var pct;
@@ -106,12 +107,12 @@ App.DagIndexController = Em.ObjectController.extend(App.ModelRefreshMixin, {
       'org.apache.tez.common.counters.DAGCounter', 'NUM_KILLED_TASKS')
   }.property('id', 'counterGroups'),
 
-  hasFailedTasks: function() {
-    return this.get('failedTasks') > 0;
-  }.property('id', 'counterGroups'),
-
   failedTasksLink: function() {
-    return '#tasks?status=FAILED&parentType=TEZ_DAG_ID&parentID=' + this.get('id');
-  }.property()
+    return '#/dag/%@/tasks?searchText=Status%3AFAILED'.fmt(this.get('id'));
+  }.property('id'),
+
+  failedTaskAttemptsLink: function() {
+    return '#/dag/%@/taskAttempts?searchText=Status%3AFAILED'.fmt(this.get('id'));
+  }.property('id'),
 
 });

http://git-wip-us.apache.org/repos/asf/tez/blob/1dd5f9fd/tez-ui/src/main/webapp/app/scripts/controllers/dag_tasks.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/scripts/controllers/dag_tasks.js b/tez-ui/src/main/webapp/app/scripts/controllers/dag_tasks.js
index 598fa3f..ec4a47f 100644
--- a/tez-ui/src/main/webapp/app/scripts/controllers/dag_tasks.js
+++ b/tez-ui/src/main/webapp/app/scripts/controllers/dag_tasks.js
@@ -163,7 +163,8 @@ App.DagTasksController = App.TablePageController.extend({
           var status = row.get('status');
           return {
             status: status,
-            statusIcon: App.Helpers.misc.getStatusClassForEntity(status)
+            statusIcon: App.Helpers.misc.getStatusClassForEntity(status,
+              row.get('hasFailedTaskAttempts'))
           };
         }
       },
@@ -189,7 +190,7 @@ App.DagTasksController = App.TablePageController.extend({
         }
       }
     ];
-  }.property(),
+  }.property('id'),
 
   columnConfigs: function() {
     return this.get('defaultColumnConfigs').concat(

http://git-wip-us.apache.org/repos/asf/tez/blob/1dd5f9fd/tez-ui/src/main/webapp/app/scripts/controllers/dag_vertices.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/scripts/controllers/dag_vertices.js b/tez-ui/src/main/webapp/app/scripts/controllers/dag_vertices.js
index 4e5531a..e531bdf 100644
--- a/tez-ui/src/main/webapp/app/scripts/controllers/dag_vertices.js
+++ b/tez-ui/src/main/webapp/app/scripts/controllers/dag_vertices.js
@@ -175,7 +175,8 @@ App.DagVerticesController = App.TablePageController.extend({
               content = Ember.Object.create({
                 vertex: row,
                 status: status,
-                statusIcon: App.Helpers.misc.getStatusClassForEntity(status)
+                statusIcon: App.Helpers.misc.getStatusClassForEntity(status,
+                  row.get('hasFailedTaskAttempts'))
               });
 
           if(status == 'RUNNING') {
@@ -203,7 +204,7 @@ App.DagVerticesController = App.TablePageController.extend({
         }
       }
     ];
-  }.property(),
+  }.property('id'),
 
   columnConfigs: function() {
     return this.get('defaultColumnConfigs').concat(

http://git-wip-us.apache.org/repos/asf/tez/blob/1dd5f9fd/tez-ui/src/main/webapp/app/scripts/controllers/dags_controller.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/scripts/controllers/dags_controller.js b/tez-ui/src/main/webapp/app/scripts/controllers/dags_controller.js
index 94c6505..c02945c 100644
--- a/tez-ui/src/main/webapp/app/scripts/controllers/dags_controller.js
+++ b/tez-ui/src/main/webapp/app/scripts/controllers/dags_controller.js
@@ -221,7 +221,8 @@ App.DagsController = Em.ObjectController.extend(App.PaginatedContentMixin, App.C
               content = Ember.Object.create({
                 dag: row,
                 status: status,
-                statusIcon: App.Helpers.misc.getStatusClassForEntity(status)
+                statusIcon: App.Helpers.misc.getStatusClassForEntity(status,
+                  row.get('hasFailedTaskAttempts'))
               });
 
           if(status == 'RUNNING') {
@@ -292,7 +293,7 @@ App.DagsController = Em.ObjectController.extend(App.PaginatedContentMixin, App.C
         }
       }
     ];
-  }.property(),
+  }.property('id'),
 
   columnConfigs: function() {
     return this.get('defaultColumnConfigs').concat(

http://git-wip-us.apache.org/repos/asf/tez/blob/1dd5f9fd/tez-ui/src/main/webapp/app/scripts/controllers/task_index_controller.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/scripts/controllers/task_index_controller.js b/tez-ui/src/main/webapp/app/scripts/controllers/task_index_controller.js
index 2c526c4..1772e9c 100644
--- a/tez-ui/src/main/webapp/app/scripts/controllers/task_index_controller.js
+++ b/tez-ui/src/main/webapp/app/scripts/controllers/task_index_controller.js
@@ -22,10 +22,11 @@ App.TaskIndexController = Em.ObjectController.extend(App.ModelRefreshMixin, {
 
   taskStatus: function() {
     return App.Helpers.misc.getFixedupDisplayStatus(this.get('model.status'));
-  }.property('id', 'status'),
+  }.property('id', 'model.status'),
 
   taskIconStatus: function() {
-    return App.Helpers.misc.getStatusClassForEntity(this.get('taskStatus'));
-  }.property('id', 'status', 'counterGroups'),
+    return App.Helpers.misc.getStatusClassForEntity(this.get('taskStatus'),
+      this.get('hasFailedTaskAttempts'));
+  }.property('id', 'taskStatus', 'hasFailedTaskAttempts'),
 
 });
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tez/blob/1dd5f9fd/tez-ui/src/main/webapp/app/scripts/controllers/tasks_controller.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/scripts/controllers/tasks_controller.js b/tez-ui/src/main/webapp/app/scripts/controllers/tasks_controller.js
index 548b516..366a86a 100644
--- a/tez-ui/src/main/webapp/app/scripts/controllers/tasks_controller.js
+++ b/tez-ui/src/main/webapp/app/scripts/controllers/tasks_controller.js
@@ -106,13 +106,14 @@ App.TasksController = Em.ObjectController.extend(App.PaginatedContentMixin, App.
           var taskStatus = row.get('status');
           return {
             status: taskStatus,
-            statusIcon: App.Helpers.misc.getStatusClassForEntity(taskStatus)
+            statusIcon: App.Helpers.misc.getStatusClassForEntity(taskStatus,
+              row.get('hasFailedTaskAttempts'))
           };
         }
       }
     ];
     
-  }.property(),
+  }.property('id'),
 
   columnConfigs: function() {
     return this.get('defaultColumnConfigs').concat(

http://git-wip-us.apache.org/repos/asf/tez/blob/1dd5f9fd/tez-ui/src/main/webapp/app/scripts/controllers/tez-app-dags-controller.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/scripts/controllers/tez-app-dags-controller.js b/tez-ui/src/main/webapp/app/scripts/controllers/tez-app-dags-controller.js
index 22aaaf9..a61f8bb 100644
--- a/tez-ui/src/main/webapp/app/scripts/controllers/tez-app-dags-controller.js
+++ b/tez-ui/src/main/webapp/app/scripts/controllers/tez-app-dags-controller.js
@@ -94,7 +94,8 @@ App.TezAppDagsController = App.TablePageController.extend({
           var status = row.get('status'),
               content = Ember.Object.create({
                 status: status,
-                statusIcon: App.Helpers.misc.getStatusClassForEntity(status)
+                statusIcon: App.Helpers.misc.getStatusClassForEntity(status,
+                  row.get('hasFailedTaskAttempts'))
               });
 
           if(status == 'RUNNING') {
@@ -149,7 +150,7 @@ App.TezAppDagsController = App.TablePageController.extend({
         },
       }
     ];
-  }.property(),
+  }.property('id'),
 
   columnConfigs: function() {
     return this.get('defaultColumnConfigs').concat(

http://git-wip-us.apache.org/repos/asf/tez/blob/1dd5f9fd/tez-ui/src/main/webapp/app/scripts/controllers/vertex_index_controller.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/scripts/controllers/vertex_index_controller.js b/tez-ui/src/main/webapp/app/scripts/controllers/vertex_index_controller.js
index ebddf53..2339e99 100644
--- a/tez-ui/src/main/webapp/app/scripts/controllers/vertex_index_controller.js
+++ b/tez-ui/src/main/webapp/app/scripts/controllers/vertex_index_controller.js
@@ -37,8 +37,9 @@ App.VertexIndexController = Em.ObjectController.extend(App.ModelRefreshMixin, {
 
   //TODO: TEZ-1705 : Create a parent class and move this function there to avoid duplication.
   iconStatus: function() {
-    return App.Helpers.misc.getStatusClassForEntity(this.get('model.status'));
-  }.property('id', 'model.status'),
+    return App.Helpers.misc.getStatusClassForEntity(this.get('model.status'),
+      this.get('model.hasFailedTaskAttempts'));
+  }.property('id', 'model.status', 'model.hasFailedTaskAttempts'),
 
   progressStr: function() {
     var pct;

http://git-wip-us.apache.org/repos/asf/tez/blob/1dd5f9fd/tez-ui/src/main/webapp/app/scripts/controllers/vertex_tasks_controller.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/scripts/controllers/vertex_tasks_controller.js b/tez-ui/src/main/webapp/app/scripts/controllers/vertex_tasks_controller.js
index 08c1a87..953ffcd 100644
--- a/tez-ui/src/main/webapp/app/scripts/controllers/vertex_tasks_controller.js
+++ b/tez-ui/src/main/webapp/app/scripts/controllers/vertex_tasks_controller.js
@@ -145,7 +145,8 @@ App.VertexTasksController = App.TablePageController.extend({
           var status = row.get('status');
           return {
             status: status,
-            statusIcon: App.Helpers.misc.getStatusClassForEntity(status)
+            statusIcon: App.Helpers.misc.getStatusClassForEntity(status,
+              row.get('hasFailedTaskAttempts'))
           };
         }
       },
@@ -171,7 +172,7 @@ App.VertexTasksController = App.TablePageController.extend({
         }
       }
     ];
-  }.property(),
+  }.property('id'),
 
   columnConfigs: function() {
     return this.get('defaultColumnConfigs').concat(

http://git-wip-us.apache.org/repos/asf/tez/blob/1dd5f9fd/tez-ui/src/main/webapp/app/scripts/models/TimelineRestAdapter.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/scripts/models/TimelineRestAdapter.js b/tez-ui/src/main/webapp/app/scripts/models/TimelineRestAdapter.js
index 9c84771..5db0990 100644
--- a/tez-ui/src/main/webapp/app/scripts/models/TimelineRestAdapter.js
+++ b/tez-ui/src/main/webapp/app/scripts/models/TimelineRestAdapter.js
@@ -95,6 +95,15 @@ var timelineJsonToDagMap = {
   user: 'primaryfilters.user.0',
   applicationId: 'otherinfo.applicationId',
   status: 'otherinfo.status',
+  hasFailedTaskAttempts: {
+    custom: function(source) {
+      // if no other info is available we say no failed tasks attempts.
+      // since otherinfo is populated only at the end.
+      var numFailedAttempts = Em.get(source, 'otherinfo.numFailedTaskAttempts');
+      return !!numFailedAttempts && numFailedAttempts > 0;
+    }
+  },
+  numFailedTasks: 'otherinfo.numFailedTasks',
   diagnostics: 'otherinfo.diagnostics',
 
   counterGroups: 'otherinfo.counters.counterGroups',
@@ -158,6 +167,7 @@ var timelineJsonToTaskMap = {
   vertexID: 'primaryfilters.TEZ_VERTEX_ID.0',
   endTime: 'otherinfo.endTime',
   status: 'otherinfo.status',
+  numFailedTaskAttempts: 'otherinfo.numFailedTaskAttempts',
   diagnostics: 'otherinfo.diagnostics',
   counterGroups: 'otherinfo.counters.counterGroups',
   successfulAttemptId: 'otherinfo.successfulAttemptId',
@@ -187,6 +197,14 @@ var timelineJsonToVertexMap = {
   endTime: 'otherinfo.endTime',
 
   status: 'otherinfo.status',
+  hasFailedTaskAttempts: {
+    custom: function(source) {
+      // if no other info is available we say no failed tasks attempts.
+      // since otherinfo is populated only at the end.
+      var numFailedAttempts = Em.get(source, 'otherinfo.numFailedTaskAttempts');
+      return !!numFailedAttempts && numFailedAttempts > 0;
+    }
+  },
   diagnostics: 'otherinfo.diagnostics',
 
   failedTasks: 'otherinfo.numFailedTasks',

http://git-wip-us.apache.org/repos/asf/tez/blob/1dd5f9fd/tez-ui/src/main/webapp/app/scripts/models/dag.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/scripts/models/dag.js b/tez-ui/src/main/webapp/app/scripts/models/dag.js
index 9053078..db6577f 100644
--- a/tez-ui/src/main/webapp/app/scripts/models/dag.js
+++ b/tez-ui/src/main/webapp/app/scripts/models/dag.js
@@ -50,6 +50,12 @@ App.Dag = App.AbstractEntity.extend({
 
 	// status
 	status: DS.attr('string'),
+  hasFailedTaskAttempts: DS.attr('boolean'),
+  hasFailedTasks: function() {
+    var f = this.get('numFailedTasks');
+    return !!f && f > 0;
+  }.property('numFailedTasks'),
+  numFailedTasks: DS.attr('number'),
 
 	// diagnostics info if any.
 	diagnostics: DS.attr('string'),
@@ -118,6 +124,11 @@ App.Vertex = App.AbstractEntity.extend({
    * App.VertexState.
    */
   status: DS.attr('string'),
+  hasFailedTaskAttempts: DS.attr('boolean'),
+  hasFailedTasks: function() {
+    var f = this.get('failedTasks');
+    return !!f && f > 0;
+  }.property('failedTasks'),
 
   /**
    * Vertex type has to be one of the types defined in 'App.VertexType'
@@ -349,6 +360,11 @@ App.Task = App.AbstractEntity.extend({
   pivotAttempt: DS.belongsTo('taskAttempt'),
 
   counterGroups: DS.attr('array'), // Serialize when required
+  numFailedTaskAttempts: DS.attr('number'),
+  hasFailedTaskAttempts: function() {
+    var numAttempts = this.get('numFailedTaskAttempts') || 0;
+    return numAttempts > 1;
+  }.property('numFailedTaskAttempts')
 });
 App.DagTask = App.Task.extend({});
 App.VertexTask = App.Task.extend({});

http://git-wip-us.apache.org/repos/asf/tez/blob/1dd5f9fd/tez-ui/src/main/webapp/app/styles/main.less
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/styles/main.less b/tez-ui/src/main/webapp/app/styles/main.less
index ba8f375..5da7e06 100644
--- a/tez-ui/src/main/webapp/app/styles/main.less
+++ b/tez-ui/src/main/webapp/app/styles/main.less
@@ -382,9 +382,6 @@ body, html {
 .task-status {
   .fa;
   .fa-lg;
-  .absolute;
-
-  top: 7px;
 
   &.success {
     .fa-icon(check-circle);
@@ -408,7 +405,7 @@ body, html {
   }
 
   &.warning {
-    .fa-icon(exclamation-triangle);
+    .fa-icon(check-circle);
     color: @warning-color;
   }
 
@@ -637,6 +634,11 @@ body, html {
 .table-container {
   .use-gpu;
 
+  .task-status {
+    .absolute;
+    top: 7px;
+  }
+
   margin: 10px 0px;
 
   .table-header {

http://git-wip-us.apache.org/repos/asf/tez/blob/1dd5f9fd/tez-ui/src/main/webapp/app/templates/dag/index.hbs
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/templates/dag/index.hbs b/tez-ui/src/main/webapp/app/templates/dag/index.hbs
index e7c6264..1384de2 100644
--- a/tez-ui/src/main/webapp/app/templates/dag/index.hbs
+++ b/tez-ui/src/main/webapp/app/templates/dag/index.hbs
@@ -63,6 +63,9 @@
             {{#if hasFailedTasks}}
               [ <a href='{{unbound failedTasksLink}}'>Failed Tasks</a> ]
             {{/if}}
+            {{#if hasFailedTaskAttempts}}
+              [ <a href='{{unbound failedTaskAttemptsLink}}'>Failed TaskAttempts</a> ]
+            {{/if}}
           </td>
         </tr>
         <tr>

http://git-wip-us.apache.org/repos/asf/tez/blob/1dd5f9fd/tez-ui/src/main/webapp/app/templates/vertex/index.hbs
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/templates/vertex/index.hbs b/tez-ui/src/main/webapp/app/templates/vertex/index.hbs
index acdce99..56f4eb9 100644
--- a/tez-ui/src/main/webapp/app/templates/vertex/index.hbs
+++ b/tez-ui/src/main/webapp/app/templates/vertex/index.hbs
@@ -65,6 +65,12 @@
             <i {{bind-attr class=':task-status iconStatus'}}></i>
             {{status}}
             {{#if progressStr}} {{bs-badge content=progressStr}}{{/if}}
+            {{#if hasFailedTasks}}
+              [ <a href='{{unbound failedTasksLink}}'>Failed Tasks</a> ]
+            {{/if}}
+            {{#if hasFailedTaskAttempts}}
+              [ <a href='{{unbound failedTaskAttemptsLink}}'>Failed TaskAttempts</a> ]
+            {{/if}}
           </td>
         </tr>
         <tr>


[30/50] [abbrv] tez git commit: TEZ-2090. Add tests for jobs running in external services. (sseth)

Posted by ss...@apache.org.
TEZ-2090. Add tests for jobs running in external services. (sseth)


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

Branch: refs/heads/TEZ-2003
Commit: 8b40191d9d009b716c258dc9c88e9f13a1c3a6f6
Parents: 5698650
Author: Siddharth Seth <ss...@apache.org>
Authored: Fri Feb 13 17:24:05 2015 -0800
Committer: Siddharth Seth <ss...@apache.org>
Committed: Wed May 6 00:13:29 2015 -0700

----------------------------------------------------------------------
 TEZ-2003-CHANGES.txt                            |   1 +
 pom.xml                                         |   6 +
 .../apache/tez/dag/api/TezConfiguration.java    |   2 +
 .../apache/tez/dag/api/TaskCommunicator.java    |   1 +
 .../tez/dag/api/TaskCommunicatorContext.java    |   3 +
 .../tez/dag/app/TezTaskCommunicatorImpl.java    |  42 +-
 .../dag/app/rm/TaskSchedulerEventHandler.java   |   2 +-
 tez-ext-service-tests/pom.xml                   | 161 ++++
 .../tez/dag/app/TezTestServiceCommunicator.java | 152 ++++
 .../TezTestServiceContainerLauncher.java        | 144 ++++
 .../TezTestServiceNoOpContainerLauncher.java    |  66 ++
 .../rm/TezTestServiceTaskSchedulerService.java  | 347 ++++++++
 .../TezTestServiceTaskCommunicatorImpl.java     | 182 ++++
 .../org/apache/tez/service/ContainerRunner.java |  27 +
 .../tez/service/MiniTezTestServiceCluster.java  | 163 ++++
 .../service/TezTestServiceConfConstants.java    |  41 +
 .../TezTestServiceProtocolBlockingPB.java       |  22 +
 .../tez/service/impl/ContainerRunnerImpl.java   | 512 +++++++++++
 .../apache/tez/service/impl/TezTestService.java | 126 +++
 .../impl/TezTestServiceProtocolClientImpl.java  |  82 ++
 .../impl/TezTestServiceProtocolServerImpl.java  | 133 +++
 .../tez/shufflehandler/FadvisedChunkedFile.java |  78 ++
 .../tez/shufflehandler/FadvisedFileRegion.java  | 160 ++++
 .../apache/tez/shufflehandler/IndexCache.java   | 199 +++++
 .../tez/shufflehandler/ShuffleHandler.java      | 840 +++++++++++++++++++
 .../tez/tests/TestExternalTezServices.java      | 183 ++++
 .../org/apache/tez/util/ProtoConverters.java    | 172 ++++
 .../src/test/proto/TezDaemonProtocol.proto      |  84 ++
 .../src/test/resources/log4j.properties         |  19 +
 .../org/apache/tez/runtime/task/TezChild.java   |   2 +-
 .../apache/tez/runtime/task/TezTaskRunner.java  |   2 +-
 31 files changed, 3943 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/8b40191d/TEZ-2003-CHANGES.txt
----------------------------------------------------------------------
diff --git a/TEZ-2003-CHANGES.txt b/TEZ-2003-CHANGES.txt
index d7e4be5..975ce65 100644
--- a/TEZ-2003-CHANGES.txt
+++ b/TEZ-2003-CHANGES.txt
@@ -1,5 +1,6 @@
 ALL CHANGES:
   TEZ-2019. Temporarily allow the scheduler and launcher to be specified via configuration.
   TEZ-2006. Task communication plane needs to be pluggable.
+  TEZ-2090. Add tests for jobs running in external services.
 
 INCOMPATIBLE CHANGES:

http://git-wip-us.apache.org/repos/asf/tez/blob/8b40191d/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 419138b..d1149fd 100644
--- a/pom.xml
+++ b/pom.xml
@@ -169,6 +169,11 @@
         <type>test-jar</type>
       </dependency>
       <dependency>
+        <groupId>org.apache.tez</groupId>
+        <artifactId>tez-ext-service-tests</artifactId>
+        <version>${project.version}</version>
+      </dependency>
+      <dependency>
         <groupId>org.slf4j</groupId>
         <artifactId>slf4j-api</artifactId>
         <version>1.7.5</version>
@@ -631,6 +636,7 @@
     <module>tez-dag</module>
     <module>tez-ui</module>
     <module>tez-plugins</module>
+    <module>tez-ext-service-tests</module>
     <module>tez-dist</module>
     <module>docs</module>
   </modules>

http://git-wip-us.apache.org/repos/asf/tez/blob/8b40191d/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java b/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java
index 87ccd19..533febb 100644
--- a/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java
+++ b/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java
@@ -1172,6 +1172,8 @@ public class TezConfiguration extends Configuration {
   public static final String TEZ_AM_CONTAINER_LAUNCHER_CLASS = TEZ_AM_PREFIX + "container-launcher.class";
   @ConfigurationScope(Scope.VERTEX)
   public static final String TEZ_AM_TASK_SCHEDULER_CLASS = TEZ_AM_PREFIX + "task-scheduler.class";
+  @ConfigurationScope(Scope.VERTEX)
+  public static final String TEZ_AM_TASK_COMMUNICATOR_CLASS = TEZ_AM_PREFIX + "task-communicator.class";
 
 
   // TODO only validate property here, value can also be validated if necessary

http://git-wip-us.apache.org/repos/asf/tez/blob/8b40191d/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicator.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicator.java b/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicator.java
index 97f9c16..c9f85e0 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicator.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicator.java
@@ -14,6 +14,7 @@
 
 package org.apache.tez.dag.api;
 
+import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.util.Map;
 

http://git-wip-us.apache.org/repos/asf/tez/blob/8b40191d/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicatorContext.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicatorContext.java b/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicatorContext.java
index 9b2d889..41675fe 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicatorContext.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicatorContext.java
@@ -44,5 +44,8 @@ public interface TaskCommunicatorContext {
   // TODO TEZ-2003 Move to vertex, taskIndex, version
   void taskStartedRemotely(TezTaskAttemptID taskAttemptID, ContainerId containerId);
 
+  // TODO TEZ-2003 Add an API to register task failure - for example, a communication failure.
+  // This will have to take into consideration the TA_FAILED event
+
   // TODO Eventually Add methods to report availability stats to the scheduler.
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/8b40191d/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java
index 5652937..258c927 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java
@@ -74,16 +74,22 @@ public class TezTaskCommunicatorImpl extends TaskCommunicator {
       new ConcurrentHashMap<TaskAttempt, ContainerId>();
 
   private final TezTaskUmbilicalProtocol taskUmbilical;
+  private final String tokenIdentifier;
+  private final Token<JobTokenIdentifier> sessionToken;
   private InetSocketAddress address;
   private Server server;
 
-  private static final class ContainerInfo {
+  public static final class ContainerInfo {
 
-    ContainerInfo(ContainerId containerId) {
+    ContainerInfo(ContainerId containerId, String host, int port) {
       this.containerId = containerId;
+      this.host = host;
+      this.port = port;
     }
 
-    ContainerId containerId;
+    final ContainerId containerId;
+    public final String host;
+    public final int port;
     TezHeartbeatResponse lastResponse = null;
     TaskSpec taskSpec = null;
     long lastRequestId = 0;
@@ -110,6 +116,8 @@ public class TezTaskCommunicatorImpl extends TaskCommunicator {
     super(TezTaskCommunicatorImpl.class.getName());
     this.taskCommunicatorContext = taskCommunicatorContext;
     this.taskUmbilical = new TezTaskUmbilicalProtocolImpl();
+    this.tokenIdentifier = this.taskCommunicatorContext.getApplicationAttemptId().getApplicationId().toString();
+    this.sessionToken = TokenCache.getSessionToken(taskCommunicatorContext.getCredentials());
   }
 
 
@@ -130,9 +138,7 @@ public class TezTaskCommunicatorImpl extends TaskCommunicator {
       try {
         JobTokenSecretManager jobTokenSecretManager =
             new JobTokenSecretManager();
-        Token<JobTokenIdentifier> sessionToken = TokenCache.getSessionToken(taskCommunicatorContext.getCredentials());
-        jobTokenSecretManager.addTokenForJob(
-            taskCommunicatorContext.getApplicationAttemptId().getApplicationId().toString(), sessionToken);
+        jobTokenSecretManager.addTokenForJob(tokenIdentifier, sessionToken);
 
         server = new RPC.Builder(conf)
             .setProtocol(TezTaskUmbilicalProtocol.class)
@@ -182,7 +188,7 @@ public class TezTaskCommunicatorImpl extends TaskCommunicator {
 
   @Override
   public void registerRunningContainer(ContainerId containerId, String host, int port) {
-    ContainerInfo oldInfo = registeredContainers.putIfAbsent(containerId, new ContainerInfo(containerId));
+    ContainerInfo oldInfo = registeredContainers.putIfAbsent(containerId, new ContainerInfo(containerId, host, port));
     if (oldInfo != null) {
       throw new TezUncheckedException("Multiple registrations for containerId: " + containerId);
     }
@@ -230,9 +236,9 @@ public class TezTaskCommunicatorImpl extends TaskCommunicator {
                 ". Already registered to containerId: " + oldId);
       }
     }
-
   }
 
+
   @Override
   public void unregisterRunningTaskAttempt(TezTaskAttemptID taskAttemptID) {
     TaskAttempt taskAttempt = new TaskAttempt(taskAttemptID);
@@ -258,6 +264,18 @@ public class TezTaskCommunicatorImpl extends TaskCommunicator {
     return address;
   }
 
+  protected String getTokenIdentifier() {
+    return tokenIdentifier;
+  }
+
+  protected Token<JobTokenIdentifier> getSessionToken() {
+    return sessionToken;
+  }
+
+  protected TaskCommunicatorContext getTaskCommunicatorContext() {
+    return taskCommunicatorContext;
+  }
+
   public TezTaskUmbilicalProtocol getUmbilical() {
     return this.taskUmbilical;
   }
@@ -471,4 +489,12 @@ public class TezTaskCommunicatorImpl extends TaskCommunicator {
       return "TaskAttempt{" + "taskAttemptId=" + taskAttemptId + '}';
     }
   }
+
+  protected ContainerInfo getContainerInfo(ContainerId containerId) {
+    return registeredContainers.get(containerId);
+  }
+
+  protected ContainerId getContainerForAttempt(TezTaskAttemptID taskAttemptId) {
+    return attemptToContainerMap.get(new TaskAttempt(taskAttemptId));
+  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tez/blob/8b40191d/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerEventHandler.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerEventHandler.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerEventHandler.java
index 62f82db..8c3ed87 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerEventHandler.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerEventHandler.java
@@ -350,7 +350,7 @@ public class TaskSchedulerEventHandler extends AbstractService
         try {
           Constructor<? extends TaskSchedulerService> ctor = taskSchedulerClazz
               .getConstructor(TaskSchedulerAppCallback.class, AppContext.class, String.class,
-                  Integer.class, String.class, Configuration.class);
+                  int.class, String.class, Configuration.class);
           ctor.setAccessible(true);
           TaskSchedulerService taskSchedulerService =
               ctor.newInstance(this, appContext, host, port, trackingUrl, getConfig());

http://git-wip-us.apache.org/repos/asf/tez/blob/8b40191d/tez-ext-service-tests/pom.xml
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/pom.xml b/tez-ext-service-tests/pom.xml
new file mode 100644
index 0000000..37f68b1
--- /dev/null
+++ b/tez-ext-service-tests/pom.xml
@@ -0,0 +1,161 @@
+<?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.
+  -->
+
+<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>
+    <artifactId>tez</artifactId>
+    <groupId>org.apache.tez</groupId>
+    <version>0.7.0-SNAPSHOT</version>
+  </parent>
+
+  <!-- TODO TEZ-2003 Merge this into the tez-tests module -->
+  <artifactId>tez-ext-service-tests</artifactId>
+
+  <dependencies>
+    <dependency>
+      <groupId>log4j</groupId>
+      <artifactId>log4j</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>com.google.guava</groupId>
+      <artifactId>guava</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-common</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>junit</groupId>
+      <artifactId>junit</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.mockito</groupId>
+      <artifactId>mockito-all</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.tez</groupId>
+      <artifactId>tez-runtime-internals</artifactId>
+    </dependency>
+    <dependency>
+      <!-- Required for the ShuffleHandler -->
+      <groupId>org.apache.tez</groupId>
+      <artifactId>tez-runtime-library</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.tez</groupId>
+      <artifactId>tez-dag</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.tez</groupId>
+      <artifactId>tez-tests</artifactId>
+      <scope>test</scope>
+      <type>test-jar</type>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-hdfs</artifactId>
+      <scope>test</scope>
+      <type>test-jar</type>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-yarn-server-tests</artifactId>
+      <type>test-jar</type>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-common</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-common</artifactId>
+      <type>test-jar</type>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-mapreduce-client-jobclient</artifactId>
+      <type>test-jar</type>
+      <scope>test</scope>
+    </dependency>
+
+  </dependencies>
+
+  <build>
+    <!--
+     Include all files in src/main/resources.  By default, do not apply property
+     substitution (filtering=false), but do apply property substitution to
+     version-info.properties (filtering=true).  This will substitute the
+     version information correctly, but prevent Maven from altering other files.
+     -->
+    <resources>
+      <resource>
+        <directory>${basedir}/src/main/resources</directory>
+        <excludes>
+          <exclude>tez-api-version-info.properties</exclude>
+        </excludes>
+        <filtering>false</filtering>
+      </resource>
+      <resource>
+        <directory>${basedir}/src/main/resources</directory>
+        <includes>
+          <include>tez-api-version-info.properties</include>
+        </includes>
+        <filtering>true</filtering>
+      </resource>
+    </resources>
+    <plugins>
+      <plugin>
+        <groupId>org.apache.rat</groupId>
+        <artifactId>apache-rat-plugin</artifactId>
+      </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/test/proto</param>
+                <param>${basedir}/../tez-api/src/main/proto</param>
+              </imports>
+              <source>
+                <directory>${basedir}/src/test/proto</directory>
+                <includes>
+                  <include>TezDaemonProtocol.proto</include>
+                </includes>
+              </source>
+              <output>${project.build.directory}/generated-test-sources/java</output>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
+    </plugins>
+  </build>
+
+</project>

http://git-wip-us.apache.org/repos/asf/tez/blob/8b40191d/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/TezTestServiceCommunicator.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/TezTestServiceCommunicator.java b/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/TezTestServiceCommunicator.java
new file mode 100644
index 0000000..ac50878
--- /dev/null
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/TezTestServiceCommunicator.java
@@ -0,0 +1,152 @@
+/*
+ * 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.
+ */
+
+package org.apache.tez.dag.app;
+
+
+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 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;
+import com.google.protobuf.Message;
+import org.apache.hadoop.service.AbstractService;
+import org.apache.tez.service.TezTestServiceProtocolBlockingPB;
+import org.apache.tez.service.impl.TezTestServiceProtocolClientImpl;
+import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos.RunContainerRequestProto;
+import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos.RunContainerResponseProto;
+import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos.SubmitWorkRequestProto;
+import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos.SubmitWorkResponseProto;
+
+public class TezTestServiceCommunicator extends AbstractService {
+
+  private final ConcurrentMap<String, TezTestServiceProtocolBlockingPB> hostProxies;
+  private final ListeningExecutorService executor;
+
+  // TODO Convert this into a singleton
+  public TezTestServiceCommunicator(int numThreads) {
+    super(TezTestServiceCommunicator.class.getSimpleName());
+    ExecutorService localExecutor = Executors.newFixedThreadPool(numThreads,
+        new ThreadFactoryBuilder().setNameFormat("TezTestServiceCommunicator #%2d").build());
+    this.hostProxies = new ConcurrentHashMap<String, TezTestServiceProtocolBlockingPB>();
+    executor = MoreExecutors.listeningDecorator(localExecutor);
+  }
+
+  @Override
+  public void serviceStop() {
+    executor.shutdownNow();
+  }
+
+
+  public void runContainer(RunContainerRequestProto request, String host, int port,
+                           final ExecuteRequestCallback<RunContainerResponseProto> callback) {
+    ListenableFuture<RunContainerResponseProto> future = executor.submit(new RunContainerCallable(request, host, port));
+    Futures.addCallback(future, new FutureCallback<RunContainerResponseProto>() {
+      @Override
+      public void onSuccess(RunContainerResponseProto result) {
+        callback.setResponse(result);
+      }
+
+      @Override
+      public void onFailure(Throwable t) {
+        callback.indicateError(t);
+      }
+    });
+
+  }
+
+  public void submitWork(SubmitWorkRequestProto request, String host, int port,
+                         final ExecuteRequestCallback<SubmitWorkResponseProto> callback) {
+    ListenableFuture<SubmitWorkResponseProto> future = executor.submit(new SubmitWorkCallable(request, host, port));
+    Futures.addCallback(future, new FutureCallback<SubmitWorkResponseProto>() {
+      @Override
+      public void onSuccess(SubmitWorkResponseProto result) {
+        callback.setResponse(result);
+      }
+
+      @Override
+      public void onFailure(Throwable t) {
+        callback.indicateError(t);
+      }
+    });
+
+  }
+
+
+  private class RunContainerCallable implements Callable<RunContainerResponseProto> {
+
+    final String hostname;
+    final int port;
+    final RunContainerRequestProto request;
+
+    private RunContainerCallable(RunContainerRequestProto request, String hostname, int port) {
+      this.hostname = hostname;
+          this.port = port;
+      this.request = request;
+    }
+
+    @Override
+    public RunContainerResponseProto call() throws Exception {
+      return getProxy(hostname, port).runContainer(null, request);
+    }
+  }
+
+  private class SubmitWorkCallable implements Callable<SubmitWorkResponseProto> {
+    final String hostname;
+    final int port;
+    final SubmitWorkRequestProto request;
+
+    private SubmitWorkCallable(SubmitWorkRequestProto request, String hostname, int port) {
+      this.hostname = hostname;
+      this.port = port;
+      this.request = request;
+    }
+
+    @Override
+    public SubmitWorkResponseProto call() throws Exception {
+      return getProxy(hostname, port).submitWork(null, request);
+    }
+  }
+
+  public interface ExecuteRequestCallback<T extends Message> {
+    void setResponse(T response);
+    void indicateError(Throwable t);
+  }
+
+  private TezTestServiceProtocolBlockingPB getProxy(String hostname, int port) {
+    String hostId = getHostIdentifier(hostname, port);
+
+    TezTestServiceProtocolBlockingPB proxy = hostProxies.get(hostId);
+    if (proxy == null) {
+      proxy = new TezTestServiceProtocolClientImpl(getConfig(), hostname, port);
+      TezTestServiceProtocolBlockingPB proxyOld = hostProxies.putIfAbsent(hostId, proxy);
+      if (proxyOld != null) {
+        // TODO Shutdown the new proxy.
+        proxy = proxyOld;
+      }
+    }
+    return proxy;
+  }
+
+  private String getHostIdentifier(String hostname, int port) {
+    return hostname + ":" + port;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tez/blob/8b40191d/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/launcher/TezTestServiceContainerLauncher.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/launcher/TezTestServiceContainerLauncher.java b/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/launcher/TezTestServiceContainerLauncher.java
new file mode 100644
index 0000000..e83165b
--- /dev/null
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/launcher/TezTestServiceContainerLauncher.java
@@ -0,0 +1,144 @@
+/*
+ * 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.
+ */
+
+package org.apache.tez.dag.app.launcher;
+
+import com.google.common.base.Preconditions;
+import com.google.protobuf.ByteString;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.service.AbstractService;
+import org.apache.hadoop.yarn.api.ApplicationConstants;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.util.Clock;
+import org.apache.tez.dag.app.AppContext;
+import org.apache.tez.dag.app.TaskAttemptListener;
+import org.apache.tez.dag.app.TezTestServiceCommunicator;
+import org.apache.tez.dag.app.rm.NMCommunicatorEvent;
+import org.apache.tez.dag.app.rm.NMCommunicatorLaunchRequestEvent;
+import org.apache.tez.dag.app.rm.container.AMContainerEvent;
+import org.apache.tez.dag.app.rm.container.AMContainerEventLaunchFailed;
+import org.apache.tez.dag.app.rm.container.AMContainerEventLaunched;
+import org.apache.tez.dag.app.rm.container.AMContainerEventType;
+import org.apache.tez.dag.history.DAGHistoryEvent;
+import org.apache.tez.dag.history.events.ContainerLaunchedEvent;
+import org.apache.tez.service.TezTestServiceConfConstants;
+import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos;
+import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos.RunContainerRequestProto;
+
+public class TezTestServiceContainerLauncher extends AbstractService implements ContainerLauncher {
+
+  // TODO Support interruptability of tasks which haven't yet been launched.
+
+  // TODO May need multiple connections per target machine, depending upon how synchronization is handled in the RPC layer
+
+  static final Log LOG = LogFactory.getLog(TezTestServiceContainerLauncher.class);
+
+  private final AppContext context;
+  private final String tokenIdentifier;
+  private final TaskAttemptListener tal;
+  private final int servicePort;
+  private final TezTestServiceCommunicator communicator;
+  private final Clock clock;
+
+
+  // Configuration passed in here to set up final parameters
+  public TezTestServiceContainerLauncher(AppContext appContext, Configuration conf,
+                                         TaskAttemptListener tal) {
+    super(TezTestServiceContainerLauncher.class.getName());
+    this.clock = appContext.getClock();
+    int numThreads = conf.getInt(TezTestServiceConfConstants.TEZ_TEST_SERVICE_AM_COMMUNICATOR_NUM_THREADS,
+        TezTestServiceConfConstants.TEZ_TEST_SERVICE_AM_COMMUNICATOR_NUM_THREADS_DEFAULT);
+
+    this.servicePort = conf.getInt(TezTestServiceConfConstants.TEZ_TEST_SERVICE_RPC_PORT, -1);
+    Preconditions.checkArgument(servicePort > 0,
+        TezTestServiceConfConstants.TEZ_TEST_SERVICE_RPC_PORT + " must be set");
+    this.communicator = new TezTestServiceCommunicator(numThreads);
+    this.context = appContext;
+    this.tokenIdentifier = context.getApplicationID().toString();
+    this.tal = tal;
+  }
+
+  @Override
+  public void serviceInit(Configuration conf) {
+    communicator.init(conf);
+  }
+
+  @Override
+  public void serviceStart() {
+    communicator.start();
+  }
+
+  @Override
+  public void serviceStop() {
+    communicator.stop();
+  }
+
+  @Override
+  public void handle(NMCommunicatorEvent event) {
+    switch (event.getType()) {
+      case CONTAINER_LAUNCH_REQUEST:
+        final NMCommunicatorLaunchRequestEvent launchEvent = (NMCommunicatorLaunchRequestEvent) event;
+        RunContainerRequestProto runRequest = constructRunContainerRequest(launchEvent);
+        communicator.runContainer(runRequest, launchEvent.getNodeId().getHost(),
+            launchEvent.getNodeId().getPort(),
+            new TezTestServiceCommunicator.ExecuteRequestCallback<TezTestServiceProtocolProtos.RunContainerResponseProto>() {
+              @Override
+              public void setResponse(TezTestServiceProtocolProtos.RunContainerResponseProto response) {
+                LOG.info("Container: " + launchEvent.getContainerId() + " launch succeeded on host: " + launchEvent.getNodeId());
+                context.getEventHandler().handle(new AMContainerEventLaunched(launchEvent.getContainerId()));
+                ContainerLaunchedEvent lEvt = new ContainerLaunchedEvent(
+                    launchEvent.getContainerId(), clock.getTime(), context.getApplicationAttemptId());
+                context.getHistoryHandler().handle(new DAGHistoryEvent(
+                    null, lEvt));
+              }
+
+              @Override
+              public void indicateError(Throwable t) {
+                LOG.error("Failed to launch container: " + launchEvent.getContainer() + " on host: " + launchEvent.getNodeId(), t);
+                sendContainerLaunchFailedMsg(launchEvent.getContainerId(), t);
+              }
+            });
+        break;
+      case CONTAINER_STOP_REQUEST:
+        LOG.info("DEBUG: Ignoring STOP_REQUEST for event: " + event);
+        // that the container is actually done (normally received from RM)
+        // TODO Sending this out for an un-launched container is invalid
+        context.getEventHandler().handle(new AMContainerEvent(event.getContainerId(),
+            AMContainerEventType.C_NM_STOP_SENT));
+        break;
+    }
+  }
+
+  private RunContainerRequestProto constructRunContainerRequest(NMCommunicatorLaunchRequestEvent event) {
+    RunContainerRequestProto.Builder builder = RunContainerRequestProto.newBuilder();
+    builder.setAmHost(tal.getAddress().getHostName()).setAmPort(tal.getAddress().getPort());
+    builder.setAppAttemptNumber(event.getContainer().getId().getApplicationAttemptId().getAttemptId());
+    builder.setApplicationIdString(
+        event.getContainer().getId().getApplicationAttemptId().getApplicationId().toString());
+    builder.setTokenIdentifier(tokenIdentifier);
+    builder.setContainerIdString(event.getContainer().getId().toString());
+    builder.setCredentialsBinary(
+        ByteString.copyFrom(event.getContainerLaunchContext().getTokens()));
+    // TODO Avoid reading this from the environment
+    builder.setUser(System.getenv(ApplicationConstants.Environment.USER.name()));
+    return builder.build();
+  }
+
+  @SuppressWarnings("unchecked")
+  void sendContainerLaunchFailedMsg(ContainerId containerId, Throwable t) {
+    context.getEventHandler().handle(new AMContainerEventLaunchFailed(containerId, t == null ? "" : t.getMessage()));
+  }
+}

http://git-wip-us.apache.org/repos/asf/tez/blob/8b40191d/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/launcher/TezTestServiceNoOpContainerLauncher.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/launcher/TezTestServiceNoOpContainerLauncher.java b/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/launcher/TezTestServiceNoOpContainerLauncher.java
new file mode 100644
index 0000000..8c8e486
--- /dev/null
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/launcher/TezTestServiceNoOpContainerLauncher.java
@@ -0,0 +1,66 @@
+/*
+ * 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.
+ */
+
+package org.apache.tez.dag.app.launcher;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.service.AbstractService;
+import org.apache.hadoop.yarn.util.Clock;
+import org.apache.tez.dag.app.AppContext;
+import org.apache.tez.dag.app.TaskAttemptListener;
+import org.apache.tez.dag.app.rm.NMCommunicatorEvent;
+import org.apache.tez.dag.app.rm.NMCommunicatorLaunchRequestEvent;
+import org.apache.tez.dag.app.rm.container.AMContainerEvent;
+import org.apache.tez.dag.app.rm.container.AMContainerEventLaunched;
+import org.apache.tez.dag.app.rm.container.AMContainerEventType;
+import org.apache.tez.dag.history.DAGHistoryEvent;
+import org.apache.tez.dag.history.events.ContainerLaunchedEvent;
+
+public class TezTestServiceNoOpContainerLauncher extends AbstractService implements ContainerLauncher {
+
+  static final Log LOG = LogFactory.getLog(TezTestServiceNoOpContainerLauncher.class);
+
+  private final AppContext context;
+  private final Clock clock;
+
+  public TezTestServiceNoOpContainerLauncher(AppContext appContext, Configuration conf,
+                                         TaskAttemptListener tal) {
+    super(TezTestServiceNoOpContainerLauncher.class.getName());
+    this.context = appContext;
+    this.clock = appContext.getClock();
+  }
+
+  @Override
+  public void handle(NMCommunicatorEvent event) {
+    switch(event.getType()) {
+      case CONTAINER_LAUNCH_REQUEST:
+        final NMCommunicatorLaunchRequestEvent launchEvent = (NMCommunicatorLaunchRequestEvent) event;
+        LOG.info("No-op launch for container: " + launchEvent.getContainerId() + " succeeded on host: " + launchEvent.getNodeId());
+        context.getEventHandler().handle(new AMContainerEventLaunched(launchEvent.getContainerId()));
+        ContainerLaunchedEvent lEvt = new ContainerLaunchedEvent(
+            launchEvent.getContainerId(), clock.getTime(), context.getApplicationAttemptId());
+        context.getHistoryHandler().handle(new DAGHistoryEvent(
+            null, lEvt));
+        break;
+      case CONTAINER_STOP_REQUEST:
+        LOG.info("DEBUG: Ignoring STOP_REQUEST for event: " + event);
+        context.getEventHandler().handle(new AMContainerEvent(event.getContainerId(),
+            AMContainerEventType.C_NM_STOP_SENT));
+        break;
+    }
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/tez/blob/8b40191d/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/rm/TezTestServiceTaskSchedulerService.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/rm/TezTestServiceTaskSchedulerService.java b/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/rm/TezTestServiceTaskSchedulerService.java
new file mode 100644
index 0000000..e3c18bf
--- /dev/null
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/rm/TezTestServiceTaskSchedulerService.java
@@ -0,0 +1,347 @@
+/*
+ * 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.
+ */
+
+package org.apache.tez.dag.app.rm;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Random;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import com.google.common.base.Preconditions;
+import com.google.common.primitives.Ints;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.ContainerStatus;
+import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.hadoop.yarn.api.records.NodeReport;
+import org.apache.hadoop.yarn.api.records.Priority;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.client.api.AMRMClient;
+import org.apache.hadoop.yarn.client.api.async.AMRMClientAsync;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.tez.dag.api.TezUncheckedException;
+import org.apache.tez.dag.app.AppContext;
+import org.apache.tez.service.TezTestServiceConfConstants;
+
+
+// TODO Registration with RM - so that the AM is considered dead and restarted in the expiry interval - 10 minutes.
+
+public class TezTestServiceTaskSchedulerService extends TaskSchedulerService {
+
+  private static final Log LOG = LogFactory.getLog(TezTestServiceTaskSchedulerService.class);
+
+  private final ExecutorService appCallbackExecutor;
+  private final TaskSchedulerAppCallback appClientDelegate;
+  private final AppContext appContext;
+  private final List<String> serviceHosts;
+  private final ContainerFactory containerFactory;
+  private final Random random = new Random();
+  // Currently all services must be running on the same port.
+  private final int containerPort;
+
+  private final String clientHostname;
+  private final int clientPort;
+  private final String trackingUrl;
+  private final AtomicBoolean isStopped = new AtomicBoolean(false);
+  private final ConcurrentMap<Object, ContainerId> runningTasks =
+      new ConcurrentHashMap<Object, ContainerId>();
+
+  private final AMRMClientAsync<AMRMClient.ContainerRequest> amRmClient;
+
+  // Per instance
+  private final int memoryPerInstance;
+  private final int coresPerInstance;
+  private final int executorsPerInstance;
+
+  // Per Executor Thread
+  private final Resource resourcePerContainer;
+
+
+  public TezTestServiceTaskSchedulerService(TaskSchedulerAppCallback appClient,
+                                            AppContext appContext,
+                                            String clientHostname, int clientPort,
+                                            String trackingUrl,
+                                            Configuration conf) {
+    // Accepting configuration here to allow setting up fields as final
+    super(TezTestServiceTaskSchedulerService.class.getName());
+    this.appCallbackExecutor = createAppCallbackExecutorService();
+    this.appClientDelegate = createAppCallbackDelegate(appClient);
+    this.appContext = appContext;
+    this.serviceHosts = new LinkedList<String>();
+    this.containerFactory = new ContainerFactory(appContext);
+
+    this.memoryPerInstance = conf
+        .getInt(TezTestServiceConfConstants.TEZ_TEST_SERVICE_MEMORY_PER_INSTANCE_MB, -1);
+    Preconditions.checkArgument(memoryPerInstance > 0,
+        TezTestServiceConfConstants.TEZ_TEST_SERVICE_MEMORY_PER_INSTANCE_MB +
+            " must be configured");
+
+    this.executorsPerInstance = conf.getInt(
+        TezTestServiceConfConstants.TEZ_TEST_SERVICE_NUM_EXECUTORS_PER_INSTANCE,
+        -1);
+    Preconditions.checkArgument(executorsPerInstance > 0,
+        TezTestServiceConfConstants.TEZ_TEST_SERVICE_NUM_EXECUTORS_PER_INSTANCE +
+            " must be configured");
+
+    this.coresPerInstance = conf
+        .getInt(TezTestServiceConfConstants.TEZ_TEST_SERVICE_VCPUS_PER_INSTANCE,
+            executorsPerInstance);
+
+    this.containerPort = conf.getInt(TezTestServiceConfConstants.TEZ_TEST_SERVICE_RPC_PORT, -1);
+    Preconditions.checkArgument(executorsPerInstance > 0,
+        TezTestServiceConfConstants.TEZ_TEST_SERVICE_RPC_PORT + " must be configured");
+
+    this.clientHostname = clientHostname;
+    this.clientPort = clientPort;
+    this.trackingUrl = trackingUrl;
+
+    int memoryPerContainer = (int) (memoryPerInstance / (float) executorsPerInstance);
+    int coresPerContainer = (int) (coresPerInstance / (float) executorsPerInstance);
+    this.resourcePerContainer = Resource.newInstance(memoryPerContainer, coresPerContainer);
+    this.amRmClient = TezAMRMClientAsync.createAMRMClientAsync(5000, new FakeAmRmCallbackHandler());
+
+    String[] hosts = conf.getTrimmedStrings(TezTestServiceConfConstants.TEZ_TEST_SERVICE_HOSTS);
+    if (hosts == null || hosts.length == 0) {
+      hosts = new String[]{"localhost"};
+    }
+    for (String host : hosts) {
+      serviceHosts.add(host);
+    }
+
+    LOG.info("Running with configuration: " +
+        "memoryPerInstance=" + memoryPerInstance +
+        ", vcoresPerInstance=" + coresPerInstance +
+        ", executorsPerInstance=" + executorsPerInstance +
+        ", resourcePerContainerInferred=" + resourcePerContainer +
+        ", hosts=" + serviceHosts.toString());
+
+  }
+
+  @Override
+  public void serviceInit(Configuration conf) {
+    amRmClient.init(conf);
+  }
+
+  @Override
+  public void serviceStart() {
+    amRmClient.start();
+    RegisterApplicationMasterResponse response;
+    try {
+      amRmClient.registerApplicationMaster(clientHostname, clientPort, trackingUrl);
+    } catch (YarnException e) {
+      throw new TezUncheckedException(e);
+    } catch (IOException e) {
+      throw new TezUncheckedException(e);
+    }
+  }
+
+  @Override
+  public void serviceStop() {
+    if (!this.isStopped.getAndSet(true)) {
+
+      try {
+        TaskSchedulerAppCallback.AppFinalStatus status = appClientDelegate.getFinalAppStatus();
+        amRmClient.unregisterApplicationMaster(status.exitStatus, status.exitMessage,
+            status.postCompletionTrackingUrl);
+      } catch (YarnException e) {
+        throw new TezUncheckedException(e);
+      } catch (IOException e) {
+        throw new TezUncheckedException(e);
+      }
+      appCallbackExecutor.shutdownNow();
+    }
+  }
+
+  @Override
+  public Resource getAvailableResources() {
+    // TODO This needs information about all running executors, and the amount of memory etc available across the cluster.
+    return Resource
+        .newInstance(Ints.checkedCast(serviceHosts.size() * memoryPerInstance),
+            serviceHosts.size() * coresPerInstance);
+  }
+
+  @Override
+  public int getClusterNodeCount() {
+    return serviceHosts.size();
+  }
+
+  @Override
+  public void resetMatchLocalityForAllHeldContainers() {
+  }
+
+  @Override
+  public Resource getTotalResources() {
+    return Resource
+        .newInstance(Ints.checkedCast(serviceHosts.size() * memoryPerInstance),
+            serviceHosts.size() * coresPerInstance);
+  }
+
+  @Override
+  public void blacklistNode(NodeId nodeId) {
+    LOG.info("DEBUG: BlacklistNode not supported");
+  }
+
+  @Override
+  public void unblacklistNode(NodeId nodeId) {
+    LOG.info("DEBUG: unBlacklistNode not supported");
+  }
+
+  @Override
+  public void allocateTask(Object task, Resource capability, String[] hosts, String[] racks,
+                           Priority priority, Object containerSignature, Object clientCookie) {
+    String host = selectHost(hosts);
+    Container container =
+        containerFactory.createContainer(resourcePerContainer, priority, host, containerPort);
+    runningTasks.put(task, container.getId());
+    appClientDelegate.taskAllocated(task, clientCookie, container);
+  }
+
+
+  @Override
+  public void allocateTask(Object task, Resource capability, ContainerId containerId,
+                           Priority priority, Object containerSignature, Object clientCookie) {
+    String host = selectHost(null);
+    Container container =
+        containerFactory.createContainer(resourcePerContainer, priority, host, containerPort);
+    runningTasks.put(task, container.getId());
+    appClientDelegate.taskAllocated(task, clientCookie, container);
+  }
+
+  @Override
+  public boolean deallocateTask(Object task, boolean taskSucceeded) {
+    ContainerId containerId = runningTasks.remove(task);
+    if (containerId == null) {
+      LOG.error("Could not determine ContainerId for task: " + task +
+          " . Could have hit a race condition. Ignoring." +
+          " The query may hang since this \"unknown\" container is now taking up a slot permanently");
+      return false;
+    }
+    appClientDelegate.containerBeingReleased(containerId);
+    return true;
+  }
+
+  @Override
+  public Object deallocateContainer(ContainerId containerId) {
+    LOG.info("DEBUG: Ignoring deallocateContainer for containerId: " + containerId);
+    return null;
+  }
+
+  @Override
+  public void setShouldUnregister() {
+
+  }
+
+  @Override
+  public boolean hasUnregistered() {
+    // Nothing to do. No registration involved.
+    return true;
+  }
+
+  private ExecutorService createAppCallbackExecutorService() {
+    return Executors.newSingleThreadExecutor(new ThreadFactoryBuilder()
+        .setNameFormat("TaskSchedulerAppCaller #%d").setDaemon(true).build());
+  }
+
+  private TaskSchedulerAppCallback createAppCallbackDelegate(
+      TaskSchedulerAppCallback realAppClient) {
+    return new TaskSchedulerAppCallbackWrapper(realAppClient,
+        appCallbackExecutor);
+  }
+
+  private String selectHost(String[] requestedHosts) {
+    String host = null;
+    if (requestedHosts != null && requestedHosts.length > 0) {
+      Arrays.sort(requestedHosts);
+      host = requestedHosts[0];
+      LOG.info("Selected host: " + host + " from requested hosts: " + Arrays.toString(requestedHosts));
+    } else {
+      host = serviceHosts.get(random.nextInt(serviceHosts.size()));
+      LOG.info("Selected random host: " + host + " since the request contained no host information");
+    }
+    return host;
+  }
+
+  static class ContainerFactory {
+    final AppContext appContext;
+    AtomicInteger nextId;
+
+    public ContainerFactory(AppContext appContext) {
+      this.appContext = appContext;
+      this.nextId = new AtomicInteger(2);
+    }
+
+    public Container createContainer(Resource capability, Priority priority, String hostname, int port) {
+      ApplicationAttemptId appAttemptId = appContext.getApplicationAttemptId();
+      ContainerId containerId = ContainerId.newInstance(appAttemptId, nextId.getAndIncrement());
+      NodeId nodeId = NodeId.newInstance(hostname, port);
+      String nodeHttpAddress = "hostname:0";
+
+      Container container = Container.newInstance(containerId,
+          nodeId,
+          nodeHttpAddress,
+          capability,
+          priority,
+          null);
+
+      return container;
+    }
+  }
+
+  private static class FakeAmRmCallbackHandler implements AMRMClientAsync.CallbackHandler {
+
+    @Override
+    public void onContainersCompleted(List<ContainerStatus> statuses) {
+
+    }
+
+    @Override
+    public void onContainersAllocated(List<Container> containers) {
+
+    }
+
+    @Override
+    public void onShutdownRequest() {
+
+    }
+
+    @Override
+    public void onNodesUpdated(List<NodeReport> updatedNodes) {
+
+    }
+
+    @Override
+    public float getProgress() {
+      return 0;
+    }
+
+    @Override
+    public void onError(Throwable e) {
+
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/tez/blob/8b40191d/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/taskcomm/TezTestServiceTaskCommunicatorImpl.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/taskcomm/TezTestServiceTaskCommunicatorImpl.java b/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/taskcomm/TezTestServiceTaskCommunicatorImpl.java
new file mode 100644
index 0000000..78cdcde
--- /dev/null
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/taskcomm/TezTestServiceTaskCommunicatorImpl.java
@@ -0,0 +1,182 @@
+/*
+ * 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.
+ */
+
+package org.apache.tez.dag.app.taskcomm;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+import com.google.protobuf.ByteString;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.DataOutputBuffer;
+import org.apache.hadoop.security.Credentials;
+import org.apache.hadoop.yarn.api.ApplicationConstants;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.LocalResource;
+import org.apache.tez.dag.api.TaskCommunicatorContext;
+import org.apache.tez.dag.app.TezTaskCommunicatorImpl;
+import org.apache.tez.dag.app.TezTestServiceCommunicator;
+import org.apache.tez.dag.records.TezTaskAttemptID;
+import org.apache.tez.runtime.api.impl.TaskSpec;
+import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos.SubmitWorkRequestProto;
+import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos.SubmitWorkResponseProto;
+import org.apache.tez.util.ProtoConverters;
+
+
+public class TezTestServiceTaskCommunicatorImpl extends TezTaskCommunicatorImpl {
+
+  private static final Log LOG = LogFactory.getLog(TezTestServiceTaskCommunicatorImpl.class);
+
+  private final TezTestServiceCommunicator communicator;
+  private final SubmitWorkRequestProto BASE_SUBMIT_WORK_REQUEST;
+  private final ConcurrentMap<String, ByteBuffer> credentialMap;
+
+  public TezTestServiceTaskCommunicatorImpl(
+      TaskCommunicatorContext taskCommunicatorContext) {
+    super(taskCommunicatorContext);
+    // TODO Maybe make this configurable
+    this.communicator = new TezTestServiceCommunicator(3);
+
+    SubmitWorkRequestProto.Builder baseBuilder = SubmitWorkRequestProto.newBuilder();
+
+    // TODO Avoid reading this from the environment
+    baseBuilder.setUser(System.getenv(ApplicationConstants.Environment.USER.name()));
+    baseBuilder.setApplicationIdString(
+        taskCommunicatorContext.getApplicationAttemptId().getApplicationId().toString());
+    baseBuilder
+        .setAppAttemptNumber(taskCommunicatorContext.getApplicationAttemptId().getAttemptId());
+    baseBuilder.setTokenIdentifier(getTokenIdentifier());
+
+    BASE_SUBMIT_WORK_REQUEST = baseBuilder.build();
+
+    credentialMap = new ConcurrentHashMap<String, ByteBuffer>();
+  }
+
+  @Override
+  public void serviceInit(Configuration conf) throws Exception {
+    super.serviceInit(conf);
+    this.communicator.init(conf);
+  }
+
+  @Override
+  public void serviceStart() {
+    super.serviceStart();
+    this.communicator.start();
+  }
+
+  @Override
+  public void serviceStop() {
+    super.serviceStop();
+  }
+
+
+  @Override
+  public void registerRunningContainer(ContainerId containerId, String hostname, int port) {
+    super.registerRunningContainer(containerId, hostname, port);
+  }
+
+  @Override
+  public void registerContainerEnd(ContainerId containerId) {
+    super.registerContainerEnd(containerId);
+  }
+
+  @Override
+  public void registerRunningTaskAttempt(final ContainerId containerId, final TaskSpec taskSpec,
+                                         Map<String, LocalResource> additionalResources,
+                                         Credentials credentials,
+                                         boolean credentialsChanged)  {
+    super.registerRunningTaskAttempt(containerId, taskSpec, additionalResources, credentials,
+        credentialsChanged);
+    SubmitWorkRequestProto requestProto = null;
+    try {
+      requestProto = constructSubmitWorkRequest(containerId, taskSpec);
+    } catch (IOException e) {
+      throw new RuntimeException("Failed to construct request", e);
+    }
+    ContainerInfo containerInfo = getContainerInfo(containerId);
+    String host;
+    int port;
+    if (containerInfo != null) {
+      synchronized (containerInfo) {
+        host = containerInfo.host;
+        port = containerInfo.port;
+      }
+    } else {
+      // TODO Handle this properly
+      throw new RuntimeException("ContainerInfo not found for container: " + containerId +
+          ", while trying to launch task: " + taskSpec.getTaskAttemptID());
+    }
+    communicator.submitWork(requestProto, host, port,
+        new TezTestServiceCommunicator.ExecuteRequestCallback<SubmitWorkResponseProto>() {
+          @Override
+          public void setResponse(SubmitWorkResponseProto response) {
+            LOG.info("Successfully launched task: " + taskSpec.getTaskAttemptID());
+            getTaskCommunicatorContext()
+                .taskStartedRemotely(taskSpec.getTaskAttemptID(), containerId);
+          }
+
+          @Override
+          public void indicateError(Throwable t) {
+            // TODO Handle this error. This is where an API on the context to indicate failure / rejection comes in.
+            LOG.info("Failed to run task: " + taskSpec.getTaskAttemptID() + " on containerId: " +
+                containerId, t);
+          }
+        });
+  }
+
+  @Override
+  public void unregisterRunningTaskAttempt(TezTaskAttemptID taskAttemptID) {
+    super.unregisterRunningTaskAttempt(taskAttemptID);
+    // Nothing else to do for now. The push API in the test does not support termination of a running task
+  }
+
+  private SubmitWorkRequestProto constructSubmitWorkRequest(ContainerId containerId,
+                                                            TaskSpec taskSpec) throws
+      IOException {
+    SubmitWorkRequestProto.Builder builder =
+        SubmitWorkRequestProto.newBuilder(BASE_SUBMIT_WORK_REQUEST);
+    builder.setContainerIdString(containerId.toString());
+    builder.setAmHost(getAddress().getHostName());
+    builder.setAmPort(getAddress().getPort());
+    Credentials taskCredentials = new Credentials();
+    // Credentials can change across DAGs. Ideally construct only once per DAG.
+    taskCredentials.addAll(getTaskCommunicatorContext().getCredentials());
+
+    ByteBuffer credentialsBinary = credentialMap.get(taskSpec.getDAGName());
+    if (credentialsBinary == null) {
+      credentialsBinary = serializeCredentials(getTaskCommunicatorContext().getCredentials());
+      credentialMap.putIfAbsent(taskSpec.getDAGName(), credentialsBinary.duplicate());
+    } else {
+      credentialsBinary = credentialsBinary.duplicate();
+    }
+    builder.setCredentialsBinary(ByteString.copyFrom(credentialsBinary));
+    builder.setTaskSpec(ProtoConverters.convertTaskSpecToProto(taskSpec));
+    return builder.build();
+  }
+
+  private ByteBuffer serializeCredentials(Credentials credentials) throws IOException {
+    Credentials containerCredentials = new Credentials();
+    containerCredentials.addAll(credentials);
+    DataOutputBuffer containerTokens_dob = new DataOutputBuffer();
+    containerCredentials.writeTokenStorageToStream(containerTokens_dob);
+    ByteBuffer containerCredentialsBuffer = ByteBuffer.wrap(containerTokens_dob.getData(), 0,
+        containerTokens_dob.getLength());
+    return containerCredentialsBuffer;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tez/blob/8b40191d/tez-ext-service-tests/src/test/java/org/apache/tez/service/ContainerRunner.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/service/ContainerRunner.java b/tez-ext-service-tests/src/test/java/org/apache/tez/service/ContainerRunner.java
new file mode 100644
index 0000000..2bca4ed
--- /dev/null
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/service/ContainerRunner.java
@@ -0,0 +1,27 @@
+/*
+ * 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.
+ */
+
+package org.apache.tez.service;
+
+import java.io.IOException;
+
+import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos;
+import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos.RunContainerRequestProto;
+import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos.SubmitWorkRequestProto;
+
+public interface ContainerRunner {
+
+  void queueContainer(RunContainerRequestProto request) throws IOException;
+  void submitWork(SubmitWorkRequestProto request) throws IOException;
+}

http://git-wip-us.apache.org/repos/asf/tez/blob/8b40191d/tez-ext-service-tests/src/test/java/org/apache/tez/service/MiniTezTestServiceCluster.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/service/MiniTezTestServiceCluster.java b/tez-ext-service-tests/src/test/java/org/apache/tez/service/MiniTezTestServiceCluster.java
new file mode 100644
index 0000000..f47bd67
--- /dev/null
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/service/MiniTezTestServiceCluster.java
@@ -0,0 +1,163 @@
+/*
+ * 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.
+ */
+
+package org.apache.tez.service;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.InetSocketAddress;
+
+import com.google.common.base.Preconditions;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileContext;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.service.AbstractService;
+import org.apache.hadoop.util.Shell;
+import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
+import org.apache.tez.service.impl.TezTestService;
+
+public class MiniTezTestServiceCluster extends AbstractService {
+
+  private static final Log LOG = LogFactory.getLog(MiniTezTestServiceCluster.class);
+
+  private final File testWorkDir;
+  private final long availableMemory;
+  private final int numExecutorsPerService;
+  private final String[] localDirs;
+  private final Configuration clusterSpecificConfiguration = new Configuration(false);
+
+  private TezTestService tezTestService;
+
+  public static MiniTezTestServiceCluster create(String clusterName, int numExecutorsPerService, long availableMemory, int numLocalDirs) {
+    return new MiniTezTestServiceCluster(clusterName, numExecutorsPerService, availableMemory, numLocalDirs);
+  }
+
+  // TODO Add support for multiple instances
+  private MiniTezTestServiceCluster(String clusterName, int numExecutorsPerService, long availableMemory, int numLocalDirs) {
+    super(clusterName + "_TezTestServerCluster");
+    Preconditions.checkArgument(numExecutorsPerService > 0);
+    Preconditions.checkArgument(availableMemory > 0);
+    Preconditions.checkArgument(numLocalDirs > 0);
+    String clusterNameTrimmed = clusterName.replace("$", "") + "_TezTestServerCluster";
+    File targetWorkDir = new File("target", clusterNameTrimmed);
+    try {
+      FileContext.getLocalFSFileContext().delete(
+          new Path(targetWorkDir.getAbsolutePath()), true);
+    } catch (Exception e) {
+      LOG.warn("Could not cleanup test workDir: " + targetWorkDir, e);
+      throw new RuntimeException("Could not cleanup test workDir: " + targetWorkDir, e);
+    }
+
+    if (Shell.WINDOWS) {
+      // The test working directory can exceed the maximum path length supported
+      // by some Windows APIs and cmd.exe (260 characters).  To work around this,
+      // create a symlink in temporary storage with a much shorter path,
+      // targeting the full path to the test working directory.  Then, use the
+      // symlink as the test working directory.
+      String targetPath = targetWorkDir.getAbsolutePath();
+      File link = new File(System.getProperty("java.io.tmpdir"),
+          String.valueOf(System.currentTimeMillis()));
+      String linkPath = link.getAbsolutePath();
+
+      try {
+        FileContext.getLocalFSFileContext().delete(new Path(linkPath), true);
+      } catch (IOException e) {
+        throw new YarnRuntimeException("could not cleanup symlink: " + linkPath, e);
+      }
+
+      // Guarantee target exists before creating symlink.
+      targetWorkDir.mkdirs();
+
+      Shell.ShellCommandExecutor shexec = new Shell.ShellCommandExecutor(
+          Shell.getSymlinkCommand(targetPath, linkPath));
+      try {
+        shexec.execute();
+      } catch (IOException e) {
+        throw new YarnRuntimeException(String.format(
+            "failed to create symlink from %s to %s, shell output: %s", linkPath,
+            targetPath, shexec.getOutput()), e);
+      }
+
+      this.testWorkDir = link;
+    } else {
+      this.testWorkDir = targetWorkDir;
+    }
+    this.numExecutorsPerService = numExecutorsPerService;
+    this.availableMemory = availableMemory;
+
+    // Setup Local Dirs
+    localDirs = new String[numLocalDirs];
+    for (int i = 0 ; i < numLocalDirs ; i++) {
+      File f = new File(testWorkDir, "localDir");
+      f.mkdirs();
+      LOG.info("Created localDir: " + f.getAbsolutePath());
+      localDirs[i] = f.getAbsolutePath();
+    }
+  }
+
+  @Override
+  public void serviceInit(Configuration conf) {
+    tezTestService = new TezTestService(conf, numExecutorsPerService, availableMemory, localDirs);
+    tezTestService.init(conf);
+
+  }
+
+  @Override
+  public void serviceStart() {
+    tezTestService.start();
+
+    clusterSpecificConfiguration.set(TezTestServiceConfConstants.TEZ_TEST_SERVICE_HOSTS,
+        getServiceAddress().getHostName());
+    clusterSpecificConfiguration.setInt(TezTestServiceConfConstants.TEZ_TEST_SERVICE_RPC_PORT,
+        getServiceAddress().getPort());
+
+    clusterSpecificConfiguration.setInt(
+        TezTestServiceConfConstants.TEZ_TEST_SERVICE_NUM_EXECUTORS_PER_INSTANCE,
+        numExecutorsPerService);
+    clusterSpecificConfiguration.setLong(
+        TezTestServiceConfConstants.TEZ_TEST_SERVICE_MEMORY_PER_INSTANCE_MB, availableMemory);
+  }
+
+  @Override
+  public void serviceStop() {
+    tezTestService.stop();
+  }
+
+  /**
+   * return the address at which the service is listening
+   * @return host:port
+   */
+  public InetSocketAddress getServiceAddress() {
+    Preconditions.checkState(getServiceState() == STATE.STARTED);
+    return tezTestService.getListenerAddress();
+  }
+
+  public int getShufflePort() {
+    Preconditions.checkState(getServiceState() == STATE.STARTED);
+    return tezTestService.getShufflePort();
+  }
+
+  public Configuration getClusterSpecificConfiguration() {
+    Preconditions.checkState(getServiceState() == STATE.STARTED);
+    return clusterSpecificConfiguration;
+  }
+
+  // Mainly for verification
+  public int getNumSubmissions() {
+    return tezTestService.getNumSubmissions();
+  }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tez/blob/8b40191d/tez-ext-service-tests/src/test/java/org/apache/tez/service/TezTestServiceConfConstants.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/service/TezTestServiceConfConstants.java b/tez-ext-service-tests/src/test/java/org/apache/tez/service/TezTestServiceConfConstants.java
new file mode 100644
index 0000000..bf4a5bd
--- /dev/null
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/service/TezTestServiceConfConstants.java
@@ -0,0 +1,41 @@
+/*
+ * 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.
+ */
+
+package org.apache.tez.service;
+
+public class TezTestServiceConfConstants {
+
+  private static final String TEZ_TEST_SERVICE_PREFIX = "tez.test.service.";
+
+  /** Number of executors per instance - used by the scheduler */
+  public static final String TEZ_TEST_SERVICE_NUM_EXECUTORS_PER_INSTANCE = TEZ_TEST_SERVICE_PREFIX + "num.executors.per-instance";
+
+  /** Memory available per instance - used by the scheduler */
+  public static final String TEZ_TEST_SERVICE_MEMORY_PER_INSTANCE_MB = TEZ_TEST_SERVICE_PREFIX + "memory.per.instance.mb";
+
+  /** CPUs available per instance - used by the scheduler */
+  public static final String TEZ_TEST_SERVICE_VCPUS_PER_INSTANCE = TEZ_TEST_SERVICE_PREFIX + "vcpus.per.instance";
+
+
+  /** Hosts on which the service is running. Currently assuming a single port for all instances */
+  public static final String TEZ_TEST_SERVICE_HOSTS = TEZ_TEST_SERVICE_PREFIX + "hosts";
+
+  /** Port on which the Service(s) listen. Current a single port for all instances */
+  public static final String TEZ_TEST_SERVICE_RPC_PORT = TEZ_TEST_SERVICE_PREFIX + "rpc.port";
+
+  /** Number of threads to use in the AM to communicate with the external service */
+  public static final String TEZ_TEST_SERVICE_AM_COMMUNICATOR_NUM_THREADS = TEZ_TEST_SERVICE_PREFIX + "communicator.num.threads";
+  public static final int TEZ_TEST_SERVICE_AM_COMMUNICATOR_NUM_THREADS_DEFAULT = 2;
+
+}

http://git-wip-us.apache.org/repos/asf/tez/blob/8b40191d/tez-ext-service-tests/src/test/java/org/apache/tez/service/TezTestServiceProtocolBlockingPB.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/service/TezTestServiceProtocolBlockingPB.java b/tez-ext-service-tests/src/test/java/org/apache/tez/service/TezTestServiceProtocolBlockingPB.java
new file mode 100644
index 0000000..1108f72
--- /dev/null
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/service/TezTestServiceProtocolBlockingPB.java
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+
+package org.apache.tez.service;
+
+import org.apache.hadoop.ipc.ProtocolInfo;
+import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos;
+
+@ProtocolInfo(protocolName = "org.apache.tez.service.TezTestServiceProtocolBlockingPB", protocolVersion = 1)
+public interface TezTestServiceProtocolBlockingPB extends TezTestServiceProtocolProtos.TezTestServiceProtocol.BlockingInterface {
+}
\ No newline at end of file