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/08/14 22:58:19 UTC

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

Repository: tez
Updated Branches:
  refs/heads/TEZ-2003 85cbf1695 -> ded95e59e (forced update)


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/899a310b
Tree: http://git-wip-us.apache.org/repos/asf/tez/tree/899a310b
Diff: http://git-wip-us.apache.org/repos/asf/tez/diff/899a310b

Branch: refs/heads/TEZ-2003
Commit: 899a310b61123c7fa6757935eaf63733d04d7699
Parents: 699634f
Author: Siddharth Seth <ss...@apache.org>
Authored: Fri Feb 20 16:12:52 2015 -0800
Committer: Siddharth Seth <ss...@apache.org>
Committed: Fri Aug 14 13:46:42 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    | 71 ++++++++------------
 .../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, 206 insertions(+), 103 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/899a310b/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/899a310b/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 a304b37..43f8794 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
@@ -469,7 +469,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();
@@ -535,7 +535,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);
 
@@ -1043,9 +1043,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;
   }
 
@@ -1066,10 +1070,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() {
@@ -2389,9 +2395,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/899a310b/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 05c4623..c48601c 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
@@ -98,13 +98,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++) {
@@ -130,11 +137,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/899a310b/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/899a310b/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 77d2e39..ac8da40 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,45 +130,34 @@ 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))
-            .setPortRangeConfig(TezConfiguration.TEZ_AM_TASK_AM_PORT_RANGE)
-            .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);
-        LOG.info("Instantiated TezTaskCommunicator RPC at " + this.address);
-      } 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))
+          .setPortRangeConfig(TezConfiguration.TEZ_AM_TASK_AM_PORT_RANGE)
+          .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);
+      LOG.info("Instantiated TezTaskCommunicator RPC at " + this.address);
+    } catch (IOException e) {
+      throw new TezUncheckedException(e);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/tez/blob/899a310b/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/899a310b/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 3c27678..7dbf937 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, RunningTaskCallback>
       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
@@ -349,7 +361,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
@@ -358,8 +372,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/899a310b/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 ae83730..ba46a67 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/899a310b/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 b70b9ea..7f45ee6 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/899a310b/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 200e737..1572c8b 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
@@ -125,7 +125,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);
     TezTaskCommunicatorImpl taskCommunicator = (TezTaskCommunicatorImpl)taskAttemptListener.getTaskCommunicator();
     TezTaskUmbilicalProtocol tezUmbilical = taskCommunicator.getUmbilical();
 

http://git-wip-us.apache.org/repos/asf/tez/blob/899a310b/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/899a310b/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/899a310b/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 e6ef5e2..32da8fb 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
@@ -67,6 +67,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;
@@ -96,7 +97,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;
@@ -111,9 +111,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;
 
@@ -122,7 +123,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;
@@ -136,6 +137,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);
@@ -164,25 +167,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;
     }
   }
   
@@ -368,7 +373,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();
@@ -376,12 +381,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),
@@ -436,7 +435,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.
@@ -449,7 +449,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 {
@@ -483,7 +483,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();
   }
 


[23/50] [abbrv] tez git commit: TEZ-2495. Inform TaskCommunicaor about Task and Container termination reasons. (sseth)

Posted by ss...@apache.org.
TEZ-2495. Inform TaskCommunicaor about Task and Container termination reasons. (sseth)


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

Branch: refs/heads/TEZ-2003
Commit: 2c19f3c3e8caaffad35a8b04b9da1fb4b58e329a
Parents: 34633d9
Author: Siddharth Seth <ss...@apache.org>
Authored: Thu May 28 02:01:04 2015 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Fri Aug 14 13:46:44 2015 -0700

----------------------------------------------------------------------
 TEZ-2003-CHANGES.txt                            |  1 +
 .../org/apache/tez/common/TezUtilsInternal.java | 31 ++++++++-----
 .../apache/tez/dag/api/ContainerEndReason.java  | 27 +++++++++++
 .../tez/dag/api/TaskAttemptEndReason.java       | 13 +++---
 .../apache/tez/dag/api/TaskCommunicator.java    | 11 +++--
 .../apache/tez/dag/app/TaskAttemptListener.java |  6 ++-
 .../dag/app/TaskAttemptListenerImpTezDag.java   |  9 ++--
 .../tez/dag/app/TezTaskCommunicatorImpl.java    |  6 ++-
 .../rm/container/AMContainerEventCompleted.java | 41 +++++++++++++++++
 .../dag/app/rm/container/AMContainerImpl.java   | 35 ++++++++-------
 .../app/TestTaskAttemptListenerImplTezDag.java  |  8 ++--
 .../dag/app/rm/container/TestAMContainer.java   | 47 +++++++++++---------
 .../TezTestServiceTaskCommunicatorImpl.java     |  9 ++--
 .../apache/tez/runtime/task/TezTaskRunner2.java | 20 +++++++--
 14 files changed, 186 insertions(+), 78 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/2c19f3c3/TEZ-2003-CHANGES.txt
----------------------------------------------------------------------
diff --git a/TEZ-2003-CHANGES.txt b/TEZ-2003-CHANGES.txt
index d651960..e333832 100644
--- a/TEZ-2003-CHANGES.txt
+++ b/TEZ-2003-CHANGES.txt
@@ -28,5 +28,6 @@ ALL CHANGES:
   TEZ-2443. TaskRunner2 should call abort, NPEs while cleaning up tasks.
   TEZ-2465. Retrun the status of a kill request in TaskRunner2.
   TEZ-2471. NPE in LogicalIOProcessorRuntimeTask while printing thread info.
+  TEZ-2495. Inform TaskCommunicaor about Task and Container termination reasons.
 
 INCOMPATIBLE CHANGES:

http://git-wip-us.apache.org/repos/asf/tez/blob/2c19f3c3/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 347a4f6..0bdeb79 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
@@ -246,10 +246,16 @@ public class TezUtilsInternal {
         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 INTERNAL_PREEMPTION:
+        return TaskAttemptTerminationCause.INTERNAL_PREEMPTION;
+      case EXTERNAL_PREEMPTION:
+        return TaskAttemptTerminationCause.EXTERNAL_PREEMPTION;
+      case APPLICATION_ERROR:
+        return TaskAttemptTerminationCause.APPLICATION_ERROR;
+      case FRAMEWORK_ERROR:
+        return TaskAttemptTerminationCause.FRAMEWORK_ERROR;
+      case NODE_FAILED:
+        return TaskAttemptTerminationCause.NODE_FAILED;
       case OTHER:
         return TaskAttemptTerminationCause.UNKNOWN_ERROR;
       default:
@@ -267,20 +273,24 @@ public class TezUtilsInternal {
         return TaskAttemptEndReason.COMMUNICATION_ERROR;
       case SERVICE_BUSY:
         return TaskAttemptEndReason.SERVICE_BUSY;
+      case INTERNAL_PREEMPTION:
+        return TaskAttemptEndReason.INTERNAL_PREEMPTION;
+      case EXTERNAL_PREEMPTION:
+        return TaskAttemptEndReason.EXTERNAL_PREEMPTION;
+      case APPLICATION_ERROR:
+        return TaskAttemptEndReason.APPLICATION_ERROR;
+      case FRAMEWORK_ERROR:
+        return TaskAttemptEndReason.FRAMEWORK_ERROR;
+      case NODE_FAILED:
+        return TaskAttemptEndReason.NODE_FAILED;
       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:
@@ -288,7 +298,6 @@ public class TezUtilsInternal {
       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/2c19f3c3/tez-common/src/main/java/org/apache/tez/dag/api/ContainerEndReason.java
----------------------------------------------------------------------
diff --git a/tez-common/src/main/java/org/apache/tez/dag/api/ContainerEndReason.java b/tez-common/src/main/java/org/apache/tez/dag/api/ContainerEndReason.java
new file mode 100644
index 0000000..e13e886
--- /dev/null
+++ b/tez-common/src/main/java/org/apache/tez/dag/api/ContainerEndReason.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.dag.api;
+
+// TODO TEZ-2003 Expose as a public API
+public enum ContainerEndReason {
+  NODE_FAILED, // Completed because the node running the container was marked as dead
+  INTERNAL_PREEMPTION, // Preempted by the AM, due to an internal decision
+  EXTERNAL_PREEMPTION, // Preempted due to cluster contention
+  APPLICATION_ERROR, // An error in the AM caused by user code
+  FRAMEWORK_ERROR, // An error in the AM - likely a bug.
+  LAUNCH_FAILED, // Failure to launch the container
+  COMPLETED, // Completed via normal flow
+  OTHER
+}

http://git-wip-us.apache.org/repos/asf/tez/blob/2c19f3c3/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
index 96a4768..de78d21 100644
--- 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
@@ -16,9 +16,12 @@ 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
+  NODE_FAILED, // Completed because the node running the container was marked as dead
+  COMMUNICATION_ERROR, // Communication error with the task
+  SERVICE_BUSY, // External service busy
+  INTERNAL_PREEMPTION, // Preempted by the AM, due to an internal decision
+  EXTERNAL_PREEMPTION, // Preempted due to cluster contention
+  APPLICATION_ERROR, // An error in the AM caused by user code
+  FRAMEWORK_ERROR, // An error in the AM - likely a bug.
+  OTHER // Unknown reason
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/2c19f3c3/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 2651013..d0a006b 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
@@ -36,7 +36,10 @@ public abstract class TaskCommunicator extends AbstractService {
   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);
+  public abstract void registerContainerEnd(ContainerId containerId, ContainerEndReason endReason);
+
+  // TODO TEZ-2003 Provide additional inforamtion like reason for container end / Task end.
+  // Was it caused by preemption - or as a result of a general task completion / container completion
 
   // TODO TEZ-2003 TaskSpec breakup into a clean interface
   // TODO TEZ-2003 Add support for priority
@@ -48,11 +51,7 @@ public abstract class TaskCommunicator extends AbstractService {
   // TODO TEZ-2003. Are additional APIs required to mark a container as completed ? - for completeness.
 
   // TODO TEZ-2003 Remove reference to TaskAttemptID
-  // TODO TEZ-2003 This needs some information about why the attempt is being unregistered.
-  // e.g. preempted in which case the task may need to be informed. Alternately as a result of
-  // a failed task.
-  // In case of preemption - a killTask API is likely a better bet than trying to overload this method.
-  public abstract void unregisterRunningTaskAttempt(TezTaskAttemptID taskAttemptID);
+  public abstract void unregisterRunningTaskAttempt(TezTaskAttemptID taskAttemptID, TaskAttemptEndReason endReason);
 
   // TODO TEZ-2003 This doesn't necessarily belong here. A server may not start within the AM.
   public abstract InetSocketAddress getAddress();

http://git-wip-us.apache.org/repos/asf/tez/blob/2c19f3c3/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 e4dad27..92e38ae 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
@@ -21,6 +21,8 @@ package org.apache.tez.dag.app;
 import java.net.InetSocketAddress;
 
 import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.tez.dag.api.ContainerEndReason;
+import org.apache.tez.dag.api.TaskAttemptEndReason;
 import org.apache.tez.dag.app.dag.DAG;
 import org.apache.tez.dag.api.TaskCommunicator;
 import org.apache.tez.dag.app.rm.container.AMContainerTask;
@@ -34,9 +36,9 @@ public interface TaskAttemptListener {
 
   void registerTaskAttempt(AMContainerTask amContainerTask, ContainerId containerId, int taskCommId);
   
-  void unregisterRunningContainer(ContainerId containerId, int taskCommId);
+  void unregisterRunningContainer(ContainerId containerId, int taskCommId, ContainerEndReason endReason);
   
-  void unregisterTaskAttempt(TezTaskAttemptID attemptID, int taskCommId);
+  void unregisterTaskAttempt(TezTaskAttemptID attemptID, int taskCommId, TaskAttemptEndReason endReason);
 
   void dagComplete(DAG dag);
 

http://git-wip-us.apache.org/repos/asf/tez/blob/2c19f3c3/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 db78fa9..1c61a0d 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
@@ -28,6 +28,7 @@ import java.util.concurrent.ConcurrentMap;
 
 import com.google.common.annotations.VisibleForTesting;
 import org.apache.commons.collections4.ListUtils;
+import org.apache.tez.dag.api.ContainerEndReason;
 import org.apache.tez.dag.app.dag.event.TaskAttemptEvent;
 import org.apache.tez.dag.app.dag.event.TaskAttemptEventStatusUpdate;
 import org.apache.tez.runtime.api.events.TaskStatusUpdateEvent;
@@ -355,7 +356,7 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
   }
 
   @Override
-  public void unregisterRunningContainer(ContainerId containerId, int taskCommId) {
+  public void unregisterRunningContainer(ContainerId containerId, int taskCommId, ContainerEndReason endReason) {
     if (LOG.isDebugEnabled()) {
       LOG.debug("Unregistering Container from TaskAttemptListener: " + containerId);
     }
@@ -363,7 +364,7 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
     if (containerInfo.taskAttemptId != null) {
       registeredAttempts.remove(containerInfo.taskAttemptId);
     }
-    taskCommunicators[taskCommId].registerContainerEnd(containerId);
+    taskCommunicators[taskCommId].registerContainerEnd(containerId, endReason);
   }
 
   @Override
@@ -404,7 +405,7 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
   }
 
   @Override
-  public void unregisterTaskAttempt(TezTaskAttemptID attemptId, int taskCommId) {
+  public void unregisterTaskAttempt(TezTaskAttemptID attemptId, int taskCommId, TaskAttemptEndReason endReason) {
     ContainerId containerId = registeredAttempts.remove(attemptId);
     if (containerId == null) {
       LOG.warn("Unregister task attempt: " + attemptId + " from unknown container");
@@ -418,7 +419,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);
-    taskCommunicators[taskCommId].unregisterRunningTaskAttempt(attemptId);
+    taskCommunicators[taskCommId].unregisterRunningTaskAttempt(attemptId, endReason);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/tez/blob/2c19f3c3/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 accde2c..3774eb4 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
@@ -41,6 +41,8 @@ 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.ContainerEndReason;
+import org.apache.tez.dag.api.TaskAttemptEndReason;
 import org.apache.tez.dag.api.TaskCommunicator;
 import org.apache.tez.dag.api.TaskCommunicatorContext;
 import org.apache.tez.dag.api.TaskHeartbeatRequest;
@@ -185,7 +187,7 @@ public class TezTaskCommunicatorImpl extends TaskCommunicator {
   }
 
   @Override
-  public void registerContainerEnd(ContainerId containerId) {
+  public void registerContainerEnd(ContainerId containerId, ContainerEndReason endReason) {
     ContainerInfo containerInfo = registeredContainers.remove(containerId);
     if (containerInfo != null) {
       synchronized(containerInfo) {
@@ -231,7 +233,7 @@ public class TezTaskCommunicatorImpl extends TaskCommunicator {
 
 
   @Override
-  public void unregisterRunningTaskAttempt(TezTaskAttemptID taskAttemptID) {
+  public void unregisterRunningTaskAttempt(TezTaskAttemptID taskAttemptID, TaskAttemptEndReason endReason) {
     TaskAttempt taskAttempt = new TaskAttempt(taskAttemptID);
     ContainerId containerId = attemptToContainerMap.remove(taskAttempt);
     if(containerId == null) {

http://git-wip-us.apache.org/repos/asf/tez/blob/2c19f3c3/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerEventCompleted.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerEventCompleted.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerEventCompleted.java
index 9bb6d7f..8ef2a83 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerEventCompleted.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerEventCompleted.java
@@ -20,6 +20,7 @@ package org.apache.tez.dag.app.rm.container;
 
 import org.apache.hadoop.yarn.api.records.ContainerExitStatus;
 import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.tez.dag.api.ContainerEndReason;
 import org.apache.tez.dag.records.TaskAttemptTerminationCause;
 
 public class AMContainerEventCompleted extends AMContainerEvent {
@@ -61,4 +62,44 @@ public class AMContainerEventCompleted extends AMContainerEvent {
     return errCause;
   }
 
+  public ContainerEndReason getContainerEndReason() {
+    if (errCause != null) {
+      switch (errCause) {
+        case INTERNAL_PREEMPTION:
+          return ContainerEndReason.INTERNAL_PREEMPTION;
+        case EXTERNAL_PREEMPTION:
+          return ContainerEndReason.EXTERNAL_PREEMPTION;
+        case FRAMEWORK_ERROR:
+          return ContainerEndReason.FRAMEWORK_ERROR;
+        case APPLICATION_ERROR:
+          return ContainerEndReason.APPLICATION_ERROR;
+        case CONTAINER_LAUNCH_FAILED:
+          return ContainerEndReason.LAUNCH_FAILED;
+        case NODE_FAILED:
+          return ContainerEndReason.NODE_FAILED;
+        case CONTAINER_EXITED:
+          return ContainerEndReason.COMPLETED;
+        case UNKNOWN_ERROR:
+        case TERMINATED_BY_CLIENT:
+        case TERMINATED_AT_SHUTDOWN:
+        case TERMINATED_INEFFECTIVE_SPECULATION:
+        case TERMINATED_EFFECTIVE_SPECULATION:
+        case TERMINATED_ORPHANED:
+        case INPUT_READ_ERROR:
+        case OUTPUT_WRITE_ERROR:
+        case OUTPUT_LOST:
+        case TASK_HEARTBEAT_ERROR:
+        case CONTAINER_STOPPED:
+        case NODE_DISK_ERROR:
+        case COMMUNICATION_ERROR:
+        case SERVICE_BUSY:
+        case INTERRUPTED_BY_SYSTEM:
+        case INTERRUPTED_BY_USER:
+        default:
+          return ContainerEndReason.OTHER;
+      }
+    } else {
+      return ContainerEndReason.OTHER;
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/2c19f3c3/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 8685556..7446734 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
@@ -27,6 +27,9 @@ import java.util.concurrent.locks.ReentrantReadWriteLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock.ReadLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock.WriteLock;
 
+import org.apache.tez.common.TezUtilsInternal;
+import org.apache.tez.dag.api.ContainerEndReason;
+import org.apache.tez.dag.api.TaskAttemptEndReason;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.security.Credentials;
@@ -635,7 +638,7 @@ public class AMContainerImpl implements AMContainer {
         container.sendTerminatingToTaskAttempt(container.currentAttempt,
             event.getMessage(), TaskAttemptTerminationCause.CONTAINER_LAUNCH_FAILED);
       }
-      container.unregisterFromTAListener();
+      container.unregisterFromTAListener(ContainerEndReason.LAUNCH_FAILED);
       container.deAllocate();
     }
   }
@@ -665,7 +668,7 @@ public class AMContainerImpl implements AMContainer {
       }
       container.containerLocalResources = null;
       container.additionalLocalResources = null;
-      container.unregisterFromTAListener();
+      container.unregisterFromTAListener(event.getContainerEndReason());
       String diag = event.getDiagnostics();
       if (!(diag == null || diag.equals(""))) {
         LOG.info("Container " + container.getContainerId()
@@ -691,7 +694,7 @@ public class AMContainerImpl implements AMContainer {
         container.sendTerminatingToTaskAttempt(container.currentAttempt,
             getMessage(container, cEvent), TaskAttemptTerminationCause.CONTAINER_STOPPED);
       }
-      container.unregisterFromTAListener();
+      container.unregisterFromTAListener(ContainerEndReason.OTHER);
       container.logStopped(container.currentAttempt == null ?
           ContainerExitStatus.SUCCESS 
           : ContainerExitStatus.INVALID);
@@ -743,7 +746,7 @@ public class AMContainerImpl implements AMContainer {
     @Override
     public void transition(AMContainerImpl container, AMContainerEvent cEvent) {
       super.transition(container, cEvent);
-      container.unregisterFromTAListener();
+      container.unregisterFromTAListener(ContainerEndReason.NODE_FAILED);
       container.deAllocate();
     }
   }
@@ -760,7 +763,7 @@ public class AMContainerImpl implements AMContainer {
                 container.getState(), TaskAttemptTerminationCause.FRAMEWORK_ERROR);
       }
       container.logStopped(ContainerExitStatus.ABORTED);
-      container.unregisterFromTAListener();
+      container.unregisterFromTAListener(ContainerEndReason.FRAMEWORK_ERROR);
       container.sendStopRequestToNM();
     }
   }
@@ -832,7 +835,7 @@ public class AMContainerImpl implements AMContainer {
     public void transition(AMContainerImpl container, AMContainerEvent cEvent) {
 
       AMContainerEventAssignTA event = (AMContainerEventAssignTA) cEvent;
-      container.unregisterAttemptFromListener(container.currentAttempt);
+      container.unregisterAttemptFromListener(container.currentAttempt, TaskAttemptEndReason.FRAMEWORK_ERROR);
       container.handleExtraTAAssign(event, container.currentAttempt);
     }
   }
@@ -843,7 +846,7 @@ public class AMContainerImpl implements AMContainer {
     public void transition(AMContainerImpl container, AMContainerEvent cEvent) {
       container.lastTaskFinishTime = System.currentTimeMillis();
       container.completedAttempts.add(container.currentAttempt);
-      container.unregisterAttemptFromListener(container.currentAttempt);
+      container.unregisterAttemptFromListener(container.currentAttempt, TaskAttemptEndReason.OTHER);
       container.currentAttempt = null;
     }
   }
@@ -860,7 +863,7 @@ public class AMContainerImpl implements AMContainer {
         container.sendTerminatedToTaskAttempt(container.currentAttempt,
             getMessage(container, event), event.getTerminationCause());
       }
-      container.unregisterAttemptFromListener(container.currentAttempt);
+      container.unregisterAttemptFromListener(container.currentAttempt, TezUtilsInternal.toTaskAttemptEndReason(event.getTerminationCause()));
       container.registerFailedAttempt(container.currentAttempt);
       container.currentAttempt= null;
       super.transition(container, cEvent);
@@ -870,7 +873,7 @@ public class AMContainerImpl implements AMContainer {
   protected static class StopRequestAtRunningTransition
       extends StopRequestAtIdleTransition {
     public void transition(AMContainerImpl container, AMContainerEvent cEvent) {
-      container.unregisterAttemptFromListener(container.currentAttempt);
+      container.unregisterAttemptFromListener(container.currentAttempt, TaskAttemptEndReason.OTHER);
       super.transition(container, cEvent);
     }
   }
@@ -891,7 +894,7 @@ public class AMContainerImpl implements AMContainer {
     @Override
     public void transition(AMContainerImpl container, AMContainerEvent cEvent) {
       super.transition(container, cEvent);
-      container.unregisterAttemptFromListener(container.currentAttempt);
+      container.unregisterAttemptFromListener(container.currentAttempt, TaskAttemptEndReason.NODE_FAILED);
     }
   }
 
@@ -900,7 +903,7 @@ public class AMContainerImpl implements AMContainer {
     @Override
     public void transition(AMContainerImpl container, AMContainerEvent cEvent) {
       super.transition(container, cEvent);
-      container.unregisterAttemptFromListener(container.currentAttempt);
+      container.unregisterAttemptFromListener(container.currentAttempt, TaskAttemptEndReason.FRAMEWORK_ERROR);
       container.sendTerminatingToTaskAttempt(container.currentAttempt,
           "Container " + container.getContainerId() +
               " hit an invalid transition - " + cEvent.getType() + " at " +
@@ -1026,7 +1029,7 @@ public class AMContainerImpl implements AMContainer {
     LOG.warn(errorMessage);
     this.logStopped(ContainerExitStatus.INVALID);
     this.sendStopRequestToNM();
-    this.unregisterFromTAListener();
+    this.unregisterFromTAListener(ContainerEndReason.FRAMEWORK_ERROR);
     this.unregisterFromContainerListener();
   }
 
@@ -1084,8 +1087,8 @@ public class AMContainerImpl implements AMContainer {
         container.getNodeId(), container.getContainerToken(), launcherId));
   }
 
-  protected void unregisterAttemptFromListener(TezTaskAttemptID attemptId) {
-    taskAttemptListener.unregisterTaskAttempt(attemptId, taskCommId);
+  protected void unregisterAttemptFromListener(TezTaskAttemptID attemptId, TaskAttemptEndReason endReason) {
+    taskAttemptListener.unregisterTaskAttempt(attemptId, taskCommId, endReason);
   }
 
   protected void registerAttemptWithListener(AMContainerTask amContainerTask) {
@@ -1096,8 +1099,8 @@ public class AMContainerImpl implements AMContainer {
     taskAttemptListener.registerRunningContainer(containerId, taskCommId);
   }
 
-  protected void unregisterFromTAListener() {
-    this.taskAttemptListener.unregisterRunningContainer(containerId, taskCommId);
+  protected void unregisterFromTAListener(ContainerEndReason endReason) {
+    this.taskAttemptListener.unregisterRunningContainer(containerId, taskCommId, endReason);
   }
 
   protected void registerWithContainerListener() {

http://git-wip-us.apache.org/repos/asf/tez/blob/2c19f3c3/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 34b9792..68d3baf 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
@@ -47,6 +47,8 @@ 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.TezConfiguration;
+import org.apache.tez.dag.api.ContainerEndReason;
+import org.apache.tez.dag.api.TaskAttemptEndReason;
 import org.apache.tez.dag.api.TaskHeartbeatRequest;
 import org.apache.tez.dag.api.TaskHeartbeatResponse;
 import org.apache.tez.dag.api.TezException;
@@ -163,12 +165,12 @@ public class TestTaskAttemptListenerImplTezDag {
     assertEquals(taskSpec, containerTask.getTaskSpec());
 
     // Task unregistered. Should respond to heartbeats
-    taskAttemptListener.unregisterTaskAttempt(taskAttemptID, 0);
+    taskAttemptListener.unregisterTaskAttempt(taskAttemptID, 0, TaskAttemptEndReason.OTHER);
     containerTask = tezUmbilical.getTask(containerContext2);
     assertNull(containerTask);
 
     // Container unregistered. Should send a shouldDie = true
-    taskAttemptListener.unregisterRunningContainer(containerId2, 0);
+    taskAttemptListener.unregisterRunningContainer(containerId2, 0, ContainerEndReason.OTHER);
     containerTask = tezUmbilical.getTask(containerContext2);
     assertTrue(containerTask.shouldDie());
 
@@ -182,7 +184,7 @@ public class TestTaskAttemptListenerImplTezDag {
     doReturn(taskAttemptId2).when(taskSpec2).getTaskAttemptID();
     AMContainerTask amContainerTask2 = new AMContainerTask(taskSpec, null, null, false, 0);
     taskAttemptListener.registerTaskAttempt(amContainerTask2, containerId3, 0);
-    taskAttemptListener.unregisterRunningContainer(containerId3, 0);
+    taskAttemptListener.unregisterRunningContainer(containerId3, 0, ContainerEndReason.OTHER);
     containerTask = tezUmbilical.getTask(containerContext3);
     assertTrue(containerTask.shouldDie());
   }

http://git-wip-us.apache.org/repos/asf/tez/blob/2c19f3c3/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 44dcd1f..322eabc 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,8 @@ 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.ContainerEndReason;
+import org.apache.tez.dag.api.TaskAttemptEndReason;
 import org.apache.tez.dag.api.TaskCommunicator;
 import org.apache.tez.dag.app.AppContext;
 import org.apache.tez.dag.app.ContainerHeartbeatHandler;
@@ -135,14 +137,14 @@ public class TestAMContainer {
     wc.verifyState(AMContainerState.IDLE);
     wc.verifyNoOutgoingEvents();
     assertNull(wc.amContainer.getCurrentTaskAttempt());
-    verify(wc.tal).unregisterTaskAttempt(wc.taskAttemptID, 0);
+    verify(wc.tal).unregisterTaskAttempt(wc.taskAttemptID, 0, TaskAttemptEndReason.OTHER);
 
     // Container completed
     wc.containerCompleted();
     wc.verifyHistoryStopEvent();
     wc.verifyState(AMContainerState.COMPLETED);
     wc.verifyNoOutgoingEvents();
-    verify(wc.tal).unregisterRunningContainer(wc.containerID, 0);
+    verify(wc.tal).unregisterRunningContainer(wc.containerID, 0, ContainerEndReason.COMPLETED);
     verify(wc.chh).unregister(wc.containerID);
 
     assertEquals(1, wc.amContainer.getAllTaskAttempts().size());
@@ -184,13 +186,13 @@ public class TestAMContainer {
     wc.verifyState(AMContainerState.IDLE);
     wc.verifyNoOutgoingEvents();
     assertNull(wc.amContainer.getCurrentTaskAttempt());
-    verify(wc.tal).unregisterTaskAttempt(wc.taskAttemptID, 0);
+    verify(wc.tal).unregisterTaskAttempt(wc.taskAttemptID, 0, TaskAttemptEndReason.OTHER);
 
     wc.containerCompleted();
     wc.verifyHistoryStopEvent();
     wc.verifyState(AMContainerState.COMPLETED);
     wc.verifyNoOutgoingEvents();
-    verify(wc.tal).unregisterRunningContainer(wc.containerID, 0);
+    verify(wc.tal).unregisterRunningContainer(wc.containerID, 0, ContainerEndReason.COMPLETED);
     verify(wc.chh).unregister(wc.containerID);
 
     assertEquals(1, wc.amContainer.getAllTaskAttempts().size());
@@ -235,7 +237,7 @@ public class TestAMContainer {
     wc.verifyState(AMContainerState.IDLE);
     wc.verifyNoOutgoingEvents();
     assertNull(wc.amContainer.getCurrentTaskAttempt());
-    verify(wc.tal).unregisterTaskAttempt(wc.taskAttemptID, 0);
+    verify(wc.tal).unregisterTaskAttempt(wc.taskAttemptID, 0, TaskAttemptEndReason.OTHER);
 
     TezTaskAttemptID taId2 = TezTaskAttemptID.getInstance(wc.taskID, 2);
     wc.assignTaskAttempt(taId2);
@@ -250,14 +252,14 @@ public class TestAMContainer {
     wc.verifyState(AMContainerState.IDLE);
     wc.verifyNoOutgoingEvents();
     assertNull(wc.amContainer.getCurrentTaskAttempt());
-    verify(wc.tal).unregisterTaskAttempt(taId2, 0);
+    verify(wc.tal).unregisterTaskAttempt(taId2, 0, TaskAttemptEndReason.OTHER);
 
     // Container completed
     wc.containerCompleted();
     wc.verifyHistoryStopEvent();
     wc.verifyState(AMContainerState.COMPLETED);
     wc.verifyNoOutgoingEvents();
-    verify(wc.tal).unregisterRunningContainer(wc.containerID, 0);
+    verify(wc.tal).unregisterRunningContainer(wc.containerID, 0, ContainerEndReason.COMPLETED);
     verify(wc.chh).unregister(wc.containerID);
 
     assertEquals(2, wc.amContainer.getAllTaskAttempts().size());
@@ -290,7 +292,7 @@ public class TestAMContainer {
     wc.verifyHistoryStopEvent();
     wc.verifyState(AMContainerState.COMPLETED);
     wc.verifyNoOutgoingEvents();
-    verify(wc.tal).unregisterRunningContainer(wc.containerID, 0);
+    verify(wc.tal).unregisterRunningContainer(wc.containerID, 0, ContainerEndReason.OTHER);
     verify(wc.chh).unregister(wc.containerID);
 
     assertNull(wc.amContainer.getCurrentTaskAttempt());
@@ -327,7 +329,7 @@ public class TestAMContainer {
     wc.verifyHistoryStopEvent();
     wc.verifyState(AMContainerState.COMPLETED);
     wc.verifyNoOutgoingEvents();
-    verify(wc.tal).unregisterRunningContainer(wc.containerID, 0);
+    verify(wc.tal).unregisterRunningContainer(wc.containerID, 0, ContainerEndReason.OTHER);
     verify(wc.chh).unregister(wc.containerID);
 
     assertNull(wc.amContainer.getCurrentTaskAttempt());
@@ -350,7 +352,7 @@ public class TestAMContainer {
     wc.assignTaskAttempt(taID2);
 
     wc.verifyState(AMContainerState.STOP_REQUESTED);
-    verify(wc.tal).unregisterRunningContainer(wc.containerID, 0);
+    verify(wc.tal).unregisterRunningContainer(wc.containerID, 0, ContainerEndReason.FRAMEWORK_ERROR);
     verify(wc.chh).unregister(wc.containerID);
     // 1 for NM stop request. 2 TERMINATING to TaskAttempt.
     outgoingEvents = wc.verifyCountAndGetOutgoingEvents(3);
@@ -388,7 +390,7 @@ public class TestAMContainer {
     wc.assignTaskAttempt(taID2);
 
     wc.verifyState(AMContainerState.STOP_REQUESTED);
-    verify(wc.tal).unregisterRunningContainer(wc.containerID, 0);
+    verify(wc.tal).unregisterRunningContainer(wc.containerID, 0, ContainerEndReason.FRAMEWORK_ERROR);
     verify(wc.chh).unregister(wc.containerID);
     // 1 for NM stop request. 2 TERMINATING to TaskAttempt.
     outgoingEvents = wc.verifyCountAndGetOutgoingEvents(3);
@@ -424,7 +426,7 @@ public class TestAMContainer {
 
     wc.containerTimedOut();
     wc.verifyState(AMContainerState.STOP_REQUESTED);
-    verify(wc.tal).unregisterRunningContainer(wc.containerID, 0);
+    verify(wc.tal).unregisterRunningContainer(wc.containerID, 0, ContainerEndReason.OTHER);
     verify(wc.chh).unregister(wc.containerID);
     // 1 to TA, 1 for RM de-allocate.
     outgoingEvents = wc.verifyCountAndGetOutgoingEvents(2);
@@ -458,7 +460,7 @@ public class TestAMContainer {
 
     wc.stopRequest();
     wc.verifyState(AMContainerState.STOP_REQUESTED);
-    verify(wc.tal).unregisterRunningContainer(wc.containerID, 0);
+    verify(wc.tal).unregisterRunningContainer(wc.containerID, 0, ContainerEndReason.OTHER);
     verify(wc.chh).unregister(wc.containerID);
     // 1 to TA, 1 for RM de-allocate.
     outgoingEvents = wc.verifyCountAndGetOutgoingEvents(2);
@@ -492,7 +494,7 @@ public class TestAMContainer {
     wc.launchFailed();
     wc.verifyState(AMContainerState.STOPPING);
     verify(wc.tal).registerRunningContainer(wc.containerID, 0);
-    verify(wc.tal).unregisterRunningContainer(wc.containerID, 0);
+    verify(wc.tal).unregisterRunningContainer(wc.containerID, 0, ContainerEndReason.LAUNCH_FAILED);
 
     outgoingEvents = wc.verifyCountAndGetOutgoingEvents(2);
     verifyUnOrderedOutgoingEventTypes(outgoingEvents,
@@ -542,7 +544,7 @@ public class TestAMContainer {
     wc.containerCompleted();
     wc.verifyState(AMContainerState.COMPLETED);
     verify(wc.tal).registerRunningContainer(wc.containerID, 0);
-    verify(wc.tal).unregisterRunningContainer(wc.containerID, 0);
+    verify(wc.tal).unregisterRunningContainer(wc.containerID, 0, ContainerEndReason.COMPLETED);
 
     outgoingEvents = wc.verifyCountAndGetOutgoingEvents(1);
     verifyUnOrderedOutgoingEventTypes(outgoingEvents,
@@ -572,7 +574,7 @@ public class TestAMContainer {
     wc.containerCompleted(ContainerExitStatus.DISKS_FAILED, TaskAttemptTerminationCause.NODE_DISK_ERROR);
     wc.verifyState(AMContainerState.COMPLETED);
     verify(wc.tal).registerRunningContainer(wc.containerID, 0);
-    verify(wc.tal).unregisterRunningContainer(wc.containerID, 0);
+    verify(wc.tal).unregisterRunningContainer(wc.containerID, 0, ContainerEndReason.OTHER);
 
     outgoingEvents = wc.verifyCountAndGetOutgoingEvents(1);
     verifyUnOrderedOutgoingEventTypes(outgoingEvents,
@@ -603,7 +605,7 @@ public class TestAMContainer {
     wc.containerCompleted(ContainerExitStatus.ABORTED, TaskAttemptTerminationCause.NODE_FAILED);
     wc.verifyState(AMContainerState.COMPLETED);
     verify(wc.tal).registerRunningContainer(wc.containerID, 0);
-    verify(wc.tal).unregisterRunningContainer(wc.containerID, 0);
+    verify(wc.tal).unregisterRunningContainer(wc.containerID, 0, ContainerEndReason.NODE_FAILED);
 
     outgoingEvents = wc.verifyCountAndGetOutgoingEvents(1);
     verifyUnOrderedOutgoingEventTypes(outgoingEvents,
@@ -634,7 +636,7 @@ public class TestAMContainer {
     wc.containerCompleted();
     wc.verifyState(AMContainerState.COMPLETED);
     verify(wc.tal).registerRunningContainer(wc.containerID, 0);
-    verify(wc.tal).unregisterRunningContainer(wc.containerID, 0);
+    verify(wc.tal).unregisterRunningContainer(wc.containerID, 0, ContainerEndReason.COMPLETED);
     verify(wc.chh).register(wc.containerID);
     verify(wc.chh).unregister(wc.containerID);
 
@@ -663,7 +665,7 @@ public class TestAMContainer {
     wc.containerCompleted();
     wc.verifyState(AMContainerState.COMPLETED);
     verify(wc.tal).registerRunningContainer(wc.containerID, 0);
-    verify(wc.tal).unregisterRunningContainer(wc.containerID, 0);
+    verify(wc.tal).unregisterRunningContainer(wc.containerID, 0, ContainerEndReason.COMPLETED);
     verify(wc.chh).register(wc.containerID);
     verify(wc.chh).unregister(wc.containerID);
 
@@ -698,7 +700,7 @@ public class TestAMContainer {
     wc.containerCompleted(ContainerExitStatus.PREEMPTED, TaskAttemptTerminationCause.EXTERNAL_PREEMPTION);
     wc.verifyState(AMContainerState.COMPLETED);
     verify(wc.tal).registerRunningContainer(wc.containerID, 0);
-    verify(wc.tal).unregisterRunningContainer(wc.containerID, 0);
+    verify(wc.tal).unregisterRunningContainer(wc.containerID, 0, ContainerEndReason.EXTERNAL_PREEMPTION);
     verify(wc.chh).register(wc.containerID);
     verify(wc.chh).unregister(wc.containerID);
 
@@ -735,7 +737,8 @@ public class TestAMContainer {
     wc.containerCompleted(ContainerExitStatus.INVALID, TaskAttemptTerminationCause.INTERNAL_PREEMPTION);
     wc.verifyState(AMContainerState.COMPLETED);
     verify(wc.tal).registerRunningContainer(wc.containerID, 0);
-    verify(wc.tal).unregisterRunningContainer(wc.containerID, 0);
+    verify(wc.tal).unregisterRunningContainer(wc.containerID, 0,
+        ContainerEndReason.INTERNAL_PREEMPTION);
     verify(wc.chh).register(wc.containerID);
     verify(wc.chh).unregister(wc.containerID);
 
@@ -772,7 +775,7 @@ public class TestAMContainer {
     wc.containerCompleted(ContainerExitStatus.DISKS_FAILED, TaskAttemptTerminationCause.NODE_DISK_ERROR);
     wc.verifyState(AMContainerState.COMPLETED);
     verify(wc.tal).registerRunningContainer(wc.containerID, 0);
-    verify(wc.tal).unregisterRunningContainer(wc.containerID, 0);
+    verify(wc.tal).unregisterRunningContainer(wc.containerID, 0, ContainerEndReason.OTHER);
     verify(wc.chh).register(wc.containerID);
     verify(wc.chh).unregister(wc.containerID);
 

http://git-wip-us.apache.org/repos/asf/tez/blob/2c19f3c3/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 cf28b11..98673a6 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
@@ -30,6 +30,7 @@ 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.ContainerEndReason;
 import org.apache.tez.dag.api.TaskAttemptEndReason;
 import org.apache.tez.dag.api.TaskCommunicatorContext;
 import org.apache.tez.dag.app.TezTaskCommunicatorImpl;
@@ -98,8 +99,8 @@ public class TezTestServiceTaskCommunicatorImpl extends TezTaskCommunicatorImpl
   }
 
   @Override
-  public void registerContainerEnd(ContainerId containerId) {
-    super.registerContainerEnd(containerId);
+  public void registerContainerEnd(ContainerId containerId, ContainerEndReason endReason) {
+    super.registerContainerEnd(containerId, endReason);
   }
 
   @Override
@@ -175,8 +176,8 @@ public class TezTestServiceTaskCommunicatorImpl extends TezTaskCommunicatorImpl
   }
 
   @Override
-  public void unregisterRunningTaskAttempt(TezTaskAttemptID taskAttemptID) {
-    super.unregisterRunningTaskAttempt(taskAttemptID);
+  public void unregisterRunningTaskAttempt(TezTaskAttemptID taskAttemptID, TaskAttemptEndReason endReason) {
+    super.unregisterRunningTaskAttempt(taskAttemptID, endReason);
     // Nothing else to do for now. The push API in the test does not support termination of a running task
   }
 

http://git-wip-us.apache.org/repos/asf/tez/blob/2c19f3c3/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezTaskRunner2.java
----------------------------------------------------------------------
diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezTaskRunner2.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezTaskRunner2.java
index 3bf9f84..15629fd 100644
--- a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezTaskRunner2.java
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezTaskRunner2.java
@@ -261,7 +261,13 @@ public class TezTaskRunner2 {
             taskRunnerCallable.interruptTask();
           }
           return true;
+        } else {
+          LOG.info("Ignoring killTask request for {} since end reason is already set to {}",
+              task.getTaskAttemptID(), firstEndReason);
         }
+      } else {
+        LOG.info("Ignoring killTask request for {} since it is not in a running state",
+            task.getTaskAttemptID());
       }
     }
     return false;
@@ -389,10 +395,18 @@ public class TezTaskRunner2 {
         isFirstTerminate = trySettingEndReason(EndReason.CONTAINER_STOP_REQUESTED);
         // Respect stopContainerRequested since it can come in at any point, despite a previous failure.
         stopContainerRequested.set(true);
-      }
 
-      if (isFirstTerminate) {
-        killTask();
+        if (isFirstTerminate) {
+          LOG.info("Attempting to abort {} since a shutdown request was received",
+              task.getTaskAttemptID());
+          if (taskRunnerCallable != null) {
+            taskKillStartTime = System.currentTimeMillis();
+            taskRunnerCallable.interruptTask();
+          }
+        } else {
+          LOG.info("Not acting on shutdown request for {} since the task is not in running state",
+              task.getTaskAttemptID());
+        }
       }
     }
   }


[43/50] [abbrv] tez git commit: TEZ-2126. Add unit tests for verifying multiple schedulers, launchers, communicators. (sseth)

Posted by ss...@apache.org.
http://git-wip-us.apache.org/repos/asf/tez/blob/267fe737/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexImpl2.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexImpl2.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexImpl2.java
index b4064a0..352ad87 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexImpl2.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexImpl2.java
@@ -28,17 +28,23 @@ import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
 
+import com.google.common.collect.BiMap;
+import com.google.common.collect.HashBiMap;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.yarn.api.ApplicationConstants;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.util.Clock;
+import org.apache.tez.dag.api.DagTypeConverters;
 import org.apache.tez.dag.api.TaskLocationHint;
 import org.apache.tez.dag.api.TezConfiguration;
 import org.apache.tez.dag.api.TezConstants;
+import org.apache.tez.dag.api.Vertex;
+import org.apache.tez.dag.api.Vertex.VertexExecutionContext;
 import org.apache.tez.dag.api.VertexLocationHint;
 import org.apache.tez.dag.api.records.DAGProtos;
+import org.apache.tez.dag.api.records.DAGProtos.VertexPlan;
 import org.apache.tez.dag.app.AppContext;
 import org.apache.tez.dag.app.ContainerContext;
 import org.apache.tez.dag.app.TaskAttemptListener;
@@ -47,6 +53,7 @@ import org.apache.tez.dag.app.dag.DAG;
 import org.apache.tez.dag.app.dag.StateChangeNotifier;
 import org.apache.tez.dag.records.TezVertexID;
 import org.apache.tez.dag.utils.TaskSpecificLaunchCmdOption;
+import org.apache.tez.runtime.api.ExecutionContext;
 import org.junit.Test;
 
 /**
@@ -60,7 +67,8 @@ public class TestVertexImpl2 {
     Configuration conf = new TezConfiguration();
     conf.set(TezConfiguration.TEZ_TASK_LOG_LEVEL, "DEBUG;org.apache.hadoop.ipc=INFO;org.apache.hadoop.server=INFO");
 
-    LogTestInfoHolder testInfo = new LogTestInfoHolder(conf);
+    LogTestInfoHolder testInfo = new LogTestInfoHolder();
+    VertexWrapper vertexWrapper = createVertexWrapperForLogTests(testInfo, conf);
 
     List<String> expectedCommands = new LinkedList<String>();
     expectedCommands.add("-Dlog4j.configuratorClass=org.apache.tez.common.TezLog4jConfigurator");
@@ -71,7 +79,8 @@ public class TestVertexImpl2 {
         TezConstants.TEZ_CONTAINER_LOGGER_NAME);
 
     for (int i = 0 ; i < testInfo.numTasks ; i++) {
-      ContainerContext containerContext = testInfo.vertex.getContainerContext(i);
+      ContainerContext containerContext = vertexWrapper
+          .vertex.getContainerContext(i);
       String javaOpts = containerContext.getJavaOpts();
       assertTrue(javaOpts.contains(testInfo.initialJavaOpts));
       for (String expectedCmd : expectedCommands) {
@@ -92,7 +101,8 @@ public class TestVertexImpl2 {
     Configuration conf = new TezConfiguration();
     conf.set(TezConfiguration.TEZ_TASK_LOG_LEVEL, "DEBUG");
 
-    LogTestInfoHolder testInfo = new LogTestInfoHolder(conf);
+    LogTestInfoHolder testInfo = new LogTestInfoHolder();
+    VertexWrapper vertexWrapper = createVertexWrapperForLogTests(testInfo, conf);
 
     List<String> expectedCommands = new LinkedList<String>();
     expectedCommands.add("-Dlog4j.configuratorClass=org.apache.tez.common.TezLog4jConfigurator");
@@ -103,7 +113,7 @@ public class TestVertexImpl2 {
         TezConstants.TEZ_CONTAINER_LOGGER_NAME);
 
     for (int i = 0 ; i < testInfo.numTasks ; i++) {
-      ContainerContext containerContext = testInfo.vertex.getContainerContext(i);
+      ContainerContext containerContext = vertexWrapper.vertex.getContainerContext(i);
       String javaOpts = containerContext.getJavaOpts();
       assertTrue(javaOpts.contains(testInfo.initialJavaOpts));
       for (String expectedCmd : expectedCommands) {
@@ -130,7 +140,8 @@ public class TestVertexImpl2 {
     conf.set(TezConfiguration.TEZ_TASK_SPECIFIC_LOG_LEVEL, "DEBUG;org.apache.tez=INFO");
     conf.set(TezConfiguration.TEZ_TASK_SPECIFIC_LAUNCH_CMD_OPTS, customJavaOpts);
 
-    LogTestInfoHolder testInfo = new LogTestInfoHolder(conf);
+    LogTestInfoHolder testInfo = new LogTestInfoHolder();
+    VertexWrapper vertexWrapper = createVertexWrapperForLogTests(testInfo, conf);
 
     // Expected command opts for regular tasks
     List<String> expectedCommands = new LinkedList<String>();
@@ -142,7 +153,7 @@ public class TestVertexImpl2 {
         TezConstants.TEZ_CONTAINER_LOGGER_NAME);
 
     for (int i = 3 ; i < testInfo.numTasks ; i++) {
-      ContainerContext containerContext = testInfo.vertex.getContainerContext(i);
+      ContainerContext containerContext = vertexWrapper.vertex.getContainerContext(i);
       String javaOpts = containerContext.getJavaOpts();
 
       assertTrue(javaOpts.contains(testInfo.initialJavaOpts));
@@ -167,7 +178,7 @@ public class TestVertexImpl2 {
         TezConstants.TEZ_CONTAINER_LOGGER_NAME);
 
     for (int i = 0 ; i < 3 ; i++) {
-      ContainerContext containerContext = testInfo.vertex.getContainerContext(i);
+      ContainerContext containerContext = vertexWrapper.vertex.getContainerContext(i);
       String javaOpts = containerContext.getJavaOpts();
 
       assertTrue(javaOpts.contains(testInfo.initialJavaOpts));
@@ -195,7 +206,8 @@ public class TestVertexImpl2 {
     conf.set(TezConfiguration.TEZ_TASK_SPECIFIC_LOG_LEVEL, "DEBUG");
     conf.set(TezConfiguration.TEZ_TASK_SPECIFIC_LAUNCH_CMD_OPTS, customJavaOpts);
 
-    LogTestInfoHolder testInfo = new LogTestInfoHolder(conf);
+    LogTestInfoHolder testInfo = new LogTestInfoHolder();
+    VertexWrapper vertexWrapper = createVertexWrapperForLogTests(testInfo, conf);
 
     // Expected command opts for regular tasks
     List<String> expectedCommands = new LinkedList<String>();
@@ -207,7 +219,7 @@ public class TestVertexImpl2 {
         TezConstants.TEZ_CONTAINER_LOGGER_NAME);
 
     for (int i = 3 ; i < testInfo.numTasks ; i++) {
-      ContainerContext containerContext = testInfo.vertex.getContainerContext(i);
+      ContainerContext containerContext = vertexWrapper.vertex.getContainerContext(i);
       String javaOpts = containerContext.getJavaOpts();
 
       assertTrue(javaOpts.contains(testInfo.initialJavaOpts));
@@ -232,7 +244,7 @@ public class TestVertexImpl2 {
         TezConstants.TEZ_CONTAINER_LOGGER_NAME);
 
     for (int i = 0 ; i < 3 ; i++) {
-      ContainerContext containerContext = testInfo.vertex.getContainerContext(i);
+      ContainerContext containerContext = vertexWrapper.vertex.getContainerContext(i);
       String javaOpts = containerContext.getJavaOpts();
 
       assertTrue(javaOpts.contains(testInfo.initialJavaOpts));
@@ -248,43 +260,224 @@ public class TestVertexImpl2 {
     }
   }
 
+  @Test(timeout = 5000)
+  public void testNullExecutionContexts() {
 
-  private static class LogTestInfoHolder {
+    ExecutionContextTestInfoHolder info = new ExecutionContextTestInfoHolder(null, null);
+    VertexWrapper vertexWrapper = createVertexWrapperForExecutionContextTest(info);
 
-    final AppContext mockAppContext;
-    final DAG mockDag;
-    final VertexImpl vertex;
-    final DAGProtos.VertexPlan vertexPlan;
+    assertEquals(0, vertexWrapper.vertex.taskSchedulerIdentifier);
+    assertEquals(0, vertexWrapper.vertex.containerLauncherIdentifier);
+    assertEquals(0, vertexWrapper.vertex.taskCommunicatorIdentifier);
+  }
+
+  @Test(timeout = 5000)
+  public void testDefaultExecContextViaDag() {
+    VertexExecutionContext defaultExecContext = VertexExecutionContext.create(
+        ExecutionContextTestInfoHolder
+            .append(ExecutionContextTestInfoHolder.TASK_SCHEDULER_NAME_BASE, 0),
+        ExecutionContextTestInfoHolder
+            .append(ExecutionContextTestInfoHolder.CONTAINER_LAUNCHER_NAME_BASE, 2),
+        ExecutionContextTestInfoHolder
+            .append(ExecutionContextTestInfoHolder.TASK_COMM_NAME_BASE, 2));
+    ExecutionContextTestInfoHolder info =
+        new ExecutionContextTestInfoHolder(null, defaultExecContext, 3);
+    VertexWrapper vertexWrapper = createVertexWrapperForExecutionContextTest(info);
+
+    assertEquals(0, vertexWrapper.vertex.taskSchedulerIdentifier);
+    assertEquals(2, vertexWrapper.vertex.containerLauncherIdentifier);
+    assertEquals(2, vertexWrapper.vertex.taskCommunicatorIdentifier);
+  }
+
+  @Test(timeout = 5000)
+  public void testVertexExecutionContextOnly() {
+    VertexExecutionContext vertexExecutionContext = VertexExecutionContext.create(
+        ExecutionContextTestInfoHolder
+            .append(ExecutionContextTestInfoHolder.TASK_SCHEDULER_NAME_BASE, 1),
+        ExecutionContextTestInfoHolder
+            .append(ExecutionContextTestInfoHolder.CONTAINER_LAUNCHER_NAME_BASE, 1),
+        ExecutionContextTestInfoHolder
+            .append(ExecutionContextTestInfoHolder.TASK_COMM_NAME_BASE, 1));
+    ExecutionContextTestInfoHolder info =
+        new ExecutionContextTestInfoHolder(vertexExecutionContext, null, 3);
+    VertexWrapper vertexWrapper = createVertexWrapperForExecutionContextTest(info);
+
+    assertEquals(1, vertexWrapper.vertex.taskSchedulerIdentifier);
+    assertEquals(1, vertexWrapper.vertex.containerLauncherIdentifier);
+    assertEquals(1, vertexWrapper.vertex.taskCommunicatorIdentifier);
+  }
+
+  @Test(timeout = 5000)
+  public void testVertexExecutionContextOverride() {
+    VertexExecutionContext defaultExecContext = VertexExecutionContext.create(
+        ExecutionContextTestInfoHolder
+            .append(ExecutionContextTestInfoHolder.TASK_SCHEDULER_NAME_BASE, 0),
+        ExecutionContextTestInfoHolder
+            .append(ExecutionContextTestInfoHolder.CONTAINER_LAUNCHER_NAME_BASE, 2),
+        ExecutionContextTestInfoHolder
+            .append(ExecutionContextTestInfoHolder.TASK_COMM_NAME_BASE, 2));
+
+    VertexExecutionContext vertexExecutionContext = VertexExecutionContext.create(
+        ExecutionContextTestInfoHolder
+            .append(ExecutionContextTestInfoHolder.TASK_SCHEDULER_NAME_BASE, 1),
+        ExecutionContextTestInfoHolder
+            .append(ExecutionContextTestInfoHolder.CONTAINER_LAUNCHER_NAME_BASE, 1),
+        ExecutionContextTestInfoHolder
+            .append(ExecutionContextTestInfoHolder.TASK_COMM_NAME_BASE, 1));
+    ExecutionContextTestInfoHolder info =
+        new ExecutionContextTestInfoHolder(vertexExecutionContext, defaultExecContext, 3);
+    VertexWrapper vertexWrapper = createVertexWrapperForExecutionContextTest(info);
+
+    assertEquals(1, vertexWrapper.vertex.taskSchedulerIdentifier);
+    assertEquals(1, vertexWrapper.vertex.containerLauncherIdentifier);
+    assertEquals(1, vertexWrapper.vertex.taskCommunicatorIdentifier);
+  }
+
+
+  private static class ExecutionContextTestInfoHolder {
+
+    static final String TASK_SCHEDULER_NAME_BASE = "TASK_SCHEDULER";
+    static final String CONTAINER_LAUNCHER_NAME_BASE = "CONTAINER_LAUNCHER";
+    static final String TASK_COMM_NAME_BASE = "TASK_COMMUNICATOR";
+
+    static String append(String base, int index) {
+      return base + index;
+    }
+
+    final String vertexName;
+    final VertexExecutionContext defaultExecutionContext;
+    final VertexExecutionContext vertexExecutionContext;
+    final BiMap<String, Integer> taskSchedulers = HashBiMap.create();
+    final BiMap<String, Integer> containerLaunchers = HashBiMap.create();
+    final BiMap<String, Integer> taskComms = HashBiMap.create();
+    final AppContext appContext;
+
+    public ExecutionContextTestInfoHolder(VertexExecutionContext vertexExecutionContext,
+                                          VertexExecutionContext defaultDagExecutionContext) {
+      this(vertexExecutionContext, defaultDagExecutionContext, 0);
+    }
+
+    public ExecutionContextTestInfoHolder(VertexExecutionContext vertexExecutionContext,
+                                          VertexExecutionContext defaultDagExecitionContext,
+                                          int numPlugins) {
+      this.vertexName = "testvertex";
+      this.vertexExecutionContext = vertexExecutionContext;
+      this.defaultExecutionContext = defaultDagExecitionContext;
+      if (numPlugins == 0) {
+        this.taskSchedulers.put(TezConstants.getTezYarnServicePluginName(), 0);
+        this.containerLaunchers.put(TezConstants.getTezYarnServicePluginName(), 0);
+        this.taskSchedulers.put(TezConstants.getTezYarnServicePluginName(), 0);
+      } else {
+        for (int i = 0; i < numPlugins; i++) {
+          this.taskSchedulers.put(append(TASK_SCHEDULER_NAME_BASE, i), i);
+          this.containerLaunchers.put(append(CONTAINER_LAUNCHER_NAME_BASE, i), i);
+          this.taskComms.put(append(TASK_COMM_NAME_BASE, i), i);
+        }
+      }
+
+      this.appContext = createDefaultMockAppContext();
+      DAG dag = appContext.getCurrentDAG();
+      doReturn(defaultDagExecitionContext).when(dag).getDefaultExecutionContext();
+      for (Map.Entry<String, Integer> entry : taskSchedulers.entrySet()) {
+        doReturn(entry.getKey()).when(appContext).getTaskSchedulerName(entry.getValue());
+        doReturn(entry.getValue()).when(appContext).getTaskScheduerIdentifier(entry.getKey());
+      }
+      for (Map.Entry<String, Integer> entry : containerLaunchers.entrySet()) {
+        doReturn(entry.getKey()).when(appContext).getContainerLauncherName(entry.getValue());
+        doReturn(entry.getValue()).when(appContext).getContainerLauncherIdentifier(entry.getKey());
+      }
+      for (Map.Entry<String, Integer> entry : taskComms.entrySet()) {
+        doReturn(entry.getKey()).when(appContext).getTaskCommunicatorName(entry.getValue());
+        doReturn(entry.getValue()).when(appContext).getTaskCommunicatorIdentifier(entry.getKey());
+      }
+    }
+  }
 
+  private VertexWrapper createVertexWrapperForExecutionContextTest(
+      ExecutionContextTestInfoHolder vertexInfo) {
+    VertexPlan vertexPlan = createVertexPlanForExeuctionContextTests(vertexInfo);
+    VertexWrapper vertexWrapper =
+        new VertexWrapper(vertexInfo.appContext, vertexPlan, new Configuration(false));
+    return vertexWrapper;
+  }
+
+  private VertexPlan createVertexPlanForExeuctionContextTests(ExecutionContextTestInfoHolder info) {
+    VertexPlan.Builder vertexPlanBuilder = VertexPlan.newBuilder()
+        .setName(info.vertexName)
+        .setTaskConfig(DAGProtos.PlanTaskConfiguration.newBuilder()
+            .setNumTasks(10)
+            .setJavaOpts("dontcare")
+            .setMemoryMb(1024)
+            .setVirtualCores(1)
+            .setTaskModule("taskmodule")
+            .build())
+        .setType(DAGProtos.PlanVertexType.NORMAL);
+    if (info.vertexExecutionContext != null) {
+      vertexPlanBuilder
+          .setExecutionContext(DagTypeConverters.convertToProto(info.vertexExecutionContext));
+    }
+    return vertexPlanBuilder.build();
+  }
+
+  private static class LogTestInfoHolder {
     final int numTasks = 10;
     final String initialJavaOpts = "initialJavaOpts";
     final String envKey = "key1";
     final String envVal = "val1";
+    final String vertexName;
+
+    public LogTestInfoHolder() {
+      this("testvertex");
+    }
 
-    LogTestInfoHolder(Configuration conf) {
-      this(conf, "testvertex");
+    public LogTestInfoHolder(String vertexName) {
+      this.vertexName = vertexName;
     }
+  }
+
+  private VertexWrapper createVertexWrapperForLogTests(LogTestInfoHolder logTestInfoHolder,
+                                                       Configuration conf) {
+    VertexPlan vertexPlan = createVertexPlanForLogTests(logTestInfoHolder);
+    VertexWrapper vertexWrapper = new VertexWrapper(vertexPlan, conf);
+    return vertexWrapper;
+  }
+
+  private VertexPlan createVertexPlanForLogTests(LogTestInfoHolder logTestInfoHolder) {
+    VertexPlan vertexPlan = VertexPlan.newBuilder()
+        .setName(logTestInfoHolder.vertexName)
+        .setTaskConfig(DAGProtos.PlanTaskConfiguration.newBuilder()
+            .setJavaOpts(logTestInfoHolder.initialJavaOpts)
+            .setNumTasks(logTestInfoHolder.numTasks)
+            .setMemoryMb(1024)
+            .setVirtualCores(1)
+            .setTaskModule("taskmodule")
+            .addEnvironmentSetting(DAGProtos.PlanKeyValuePair.newBuilder()
+                .setKey(logTestInfoHolder.envKey)
+                .setValue(logTestInfoHolder.envVal)
+                .build())
+            .build())
+        .setType(DAGProtos.PlanVertexType.NORMAL).build();
+    return vertexPlan;
+  }
+
+  private static class VertexWrapper {
 
-    LogTestInfoHolder(Configuration conf, String vertexName) {
-      mockAppContext = mock(AppContext.class);
-      mockDag = mock(DAG.class);
-      doReturn(new Credentials()).when(mockDag).getCredentials();
-      doReturn(mockDag).when(mockAppContext).getCurrentDAG();
-
-      vertexPlan = DAGProtos.VertexPlan.newBuilder()
-          .setName(vertexName)
-          .setTaskConfig(DAGProtos.PlanTaskConfiguration.newBuilder()
-              .setJavaOpts(initialJavaOpts)
-              .setNumTasks(numTasks)
-              .setMemoryMb(1024)
-              .setVirtualCores(1)
-              .setTaskModule("taskmodule")
-              .addEnvironmentSetting(DAGProtos.PlanKeyValuePair.newBuilder()
-                  .setKey(envKey)
-                  .setValue(envVal)
-                  .build())
-              .build())
-          .setType(DAGProtos.PlanVertexType.NORMAL).build();
+    final AppContext mockAppContext;
+    final VertexImpl vertex;
+    final VertexPlan vertexPlan;
+
+    VertexWrapper(AppContext appContext, VertexPlan vertexPlan, Configuration conf) {
+      if (appContext == null) {
+        mockAppContext = createDefaultMockAppContext();
+        DAG mockDag = mock(DAG.class);
+        doReturn(new Credentials()).when(mockDag).getCredentials();
+        doReturn(mockDag).when(mockAppContext).getCurrentDAG();
+      } else {
+        mockAppContext = appContext;
+      }
+
+
+      this.vertexPlan = vertexPlan;
 
       vertex =
           new VertexImpl(TezVertexID.fromString("vertex_1418197758681_0001_1_00"), vertexPlan,
@@ -293,5 +486,17 @@ public class TestVertexImpl2 {
               VertexLocationHint.create(new LinkedList<TaskLocationHint>()), null,
               new TaskSpecificLaunchCmdOption(conf), mock(StateChangeNotifier.class));
     }
+
+    VertexWrapper(VertexPlan vertexPlan, Configuration conf) {
+      this(null, vertexPlan, conf);
+    }
+  }
+
+  private static AppContext createDefaultMockAppContext() {
+    AppContext appContext = mock(AppContext.class);
+    DAG mockDag = mock(DAG.class);
+    doReturn(new Credentials()).when(mockDag).getCredentials();
+    doReturn(mockDag).when(appContext).getCurrentDAG();
+    return appContext;
   }
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/267fe737/tez-dag/src/test/java/org/apache/tez/dag/app/launcher/TestContainerLauncherRouter.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/launcher/TestContainerLauncherRouter.java b/tez-dag/src/test/java/org/apache/tez/dag/app/launcher/TestContainerLauncherRouter.java
new file mode 100644
index 0000000..62a5f19
--- /dev/null
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/launcher/TestContainerLauncherRouter.java
@@ -0,0 +1,361 @@
+/*
+ * 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 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;
+import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.verify;
+
+import java.io.IOException;
+import java.net.UnknownHostException;
+import java.nio.ByteBuffer;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
+import org.apache.tez.common.TezUtils;
+import org.apache.tez.dag.api.NamedEntityDescriptor;
+import org.apache.tez.dag.api.TezConstants;
+import org.apache.tez.dag.api.UserPayload;
+import org.apache.tez.dag.app.AppContext;
+import org.apache.tez.dag.app.TaskAttemptListener;
+import org.apache.tez.dag.app.rm.NMCommunicatorLaunchRequestEvent;
+import org.apache.tez.serviceplugins.api.ContainerLaunchRequest;
+import org.apache.tez.serviceplugins.api.ContainerLauncher;
+import org.apache.tez.serviceplugins.api.ContainerLauncherContext;
+import org.apache.tez.serviceplugins.api.ContainerStopRequest;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.ArgumentCaptor;
+
+public class TestContainerLauncherRouter {
+
+  @Before
+  @After
+  public void reset() {
+    ContainerLaucherRouterForMultipleLauncherTest.reset();
+  }
+
+  @Test(timeout = 5000)
+  public void testNoLaunchersSpecified() throws IOException {
+
+    AppContext appContext = mock(AppContext.class);
+    TaskAttemptListener tal = mock(TaskAttemptListener.class);
+
+    try {
+
+      new ContainerLaucherRouterForMultipleLauncherTest(appContext, tal, null, null,
+          false);
+      fail("Expecting a failure without any launchers being specified");
+    } catch (IllegalArgumentException e) {
+
+    }
+  }
+
+  @Test(timeout = 5000)
+  public void testCustomLauncherSpecified() throws IOException {
+    Configuration conf = new Configuration(false);
+
+    AppContext appContext = mock(AppContext.class);
+    TaskAttemptListener tal = mock(TaskAttemptListener.class);
+
+    String customLauncherName = "customLauncher";
+    List<NamedEntityDescriptor> launcherDescriptors = new LinkedList<>();
+    ByteBuffer bb = ByteBuffer.allocate(4);
+    bb.putInt(0, 3);
+    UserPayload customPayload = UserPayload.create(bb);
+    launcherDescriptors.add(
+        new NamedEntityDescriptor(customLauncherName, FakeContainerLauncher.class.getName())
+            .setUserPayload(customPayload));
+
+    ContainerLaucherRouterForMultipleLauncherTest clr =
+        new ContainerLaucherRouterForMultipleLauncherTest(appContext, tal, null,
+            launcherDescriptors,
+            true);
+    try {
+      clr.init(conf);
+      clr.start();
+
+      assertEquals(1, clr.getNumContainerLaunchers());
+      assertFalse(clr.getYarnContainerLauncherCreated());
+      assertFalse(clr.getUberContainerLauncherCreated());
+      assertEquals(customLauncherName, clr.getContainerLauncherName(0));
+      assertEquals(bb, clr.getContainerLauncherContext(0).getInitialUserPayload().getPayload());
+    } finally {
+      clr.stop();
+    }
+  }
+
+  @Test(timeout = 5000)
+  public void testMultipleContainerLaunchers() throws IOException {
+    Configuration conf = new Configuration(false);
+    conf.set("testkey", "testvalue");
+    UserPayload userPayload = TezUtils.createUserPayloadFromConf(conf);
+
+    AppContext appContext = mock(AppContext.class);
+    TaskAttemptListener tal = mock(TaskAttemptListener.class);
+
+    String customLauncherName = "customLauncher";
+    List<NamedEntityDescriptor> launcherDescriptors = new LinkedList<>();
+    ByteBuffer bb = ByteBuffer.allocate(4);
+    bb.putInt(0, 3);
+    UserPayload customPayload = UserPayload.create(bb);
+    launcherDescriptors.add(
+        new NamedEntityDescriptor(customLauncherName, FakeContainerLauncher.class.getName())
+            .setUserPayload(customPayload));
+    launcherDescriptors
+        .add(new NamedEntityDescriptor(TezConstants.getTezYarnServicePluginName(), null)
+            .setUserPayload(userPayload));
+
+    ContainerLaucherRouterForMultipleLauncherTest clr =
+        new ContainerLaucherRouterForMultipleLauncherTest(appContext, tal, null,
+            launcherDescriptors,
+            true);
+    try {
+      clr.init(conf);
+      clr.start();
+
+      assertEquals(2, clr.getNumContainerLaunchers());
+      assertTrue(clr.getYarnContainerLauncherCreated());
+      assertFalse(clr.getUberContainerLauncherCreated());
+      assertEquals(customLauncherName, clr.getContainerLauncherName(0));
+      assertEquals(bb, clr.getContainerLauncherContext(0).getInitialUserPayload().getPayload());
+
+      assertEquals(TezConstants.getTezYarnServicePluginName(), clr.getContainerLauncherName(1));
+      Configuration confParsed = TezUtils
+          .createConfFromUserPayload(clr.getContainerLauncherContext(1).getInitialUserPayload());
+      assertEquals("testvalue", confParsed.get("testkey"));
+    } finally {
+      clr.stop();
+    }
+  }
+
+  @Test(timeout = 5000)
+  public void testEventRouting() throws Exception {
+    Configuration conf = new Configuration(false);
+    UserPayload userPayload = TezUtils.createUserPayloadFromConf(conf);
+
+    AppContext appContext = mock(AppContext.class);
+    TaskAttemptListener tal = mock(TaskAttemptListener.class);
+
+    String customLauncherName = "customLauncher";
+    List<NamedEntityDescriptor> launcherDescriptors = new LinkedList<>();
+    ByteBuffer bb = ByteBuffer.allocate(4);
+    bb.putInt(0, 3);
+    UserPayload customPayload = UserPayload.create(bb);
+    launcherDescriptors.add(
+        new NamedEntityDescriptor(customLauncherName, FakeContainerLauncher.class.getName())
+            .setUserPayload(customPayload));
+    launcherDescriptors
+        .add(new NamedEntityDescriptor(TezConstants.getTezYarnServicePluginName(), null)
+            .setUserPayload(userPayload));
+
+    ContainerLaucherRouterForMultipleLauncherTest clr =
+        new ContainerLaucherRouterForMultipleLauncherTest(appContext, tal, null,
+            launcherDescriptors,
+            true);
+    try {
+      clr.init(conf);
+      clr.start();
+
+      assertEquals(2, clr.getNumContainerLaunchers());
+      assertTrue(clr.getYarnContainerLauncherCreated());
+      assertFalse(clr.getUberContainerLauncherCreated());
+      assertEquals(customLauncherName, clr.getContainerLauncherName(0));
+      assertEquals(TezConstants.getTezYarnServicePluginName(), clr.getContainerLauncherName(1));
+
+      verify(clr.getTestContainerLauncher(0)).initialize();
+      verify(clr.getTestContainerLauncher(0)).start();
+      verify(clr.getTestContainerLauncher(1)).initialize();
+      verify(clr.getTestContainerLauncher(1)).start();
+
+      ContainerLaunchContext clc1 = mock(ContainerLaunchContext.class);
+      Container container1 = mock(Container.class);
+
+      ContainerLaunchContext clc2 = mock(ContainerLaunchContext.class);
+      Container container2 = mock(Container.class);
+
+      NMCommunicatorLaunchRequestEvent launchRequestEvent1 =
+          new NMCommunicatorLaunchRequestEvent(clc1, container1, 0, 0, 0);
+      NMCommunicatorLaunchRequestEvent launchRequestEvent2 =
+          new NMCommunicatorLaunchRequestEvent(clc2, container2, 1, 0, 0);
+
+      clr.handle(launchRequestEvent1);
+
+
+      ArgumentCaptor<ContainerLaunchRequest> captor =
+          ArgumentCaptor.forClass(ContainerLaunchRequest.class);
+      verify(clr.getTestContainerLauncher(0)).launchContainer(captor.capture());
+      assertEquals(1, captor.getAllValues().size());
+      ContainerLaunchRequest launchRequest1 = captor.getValue();
+      assertEquals(clc1, launchRequest1.getContainerLaunchContext());
+
+      clr.handle(launchRequestEvent2);
+      captor = ArgumentCaptor.forClass(ContainerLaunchRequest.class);
+      verify(clr.getTestContainerLauncher(1)).launchContainer(captor.capture());
+      assertEquals(1, captor.getAllValues().size());
+      ContainerLaunchRequest launchRequest2 = captor.getValue();
+      assertEquals(clc2, launchRequest2.getContainerLaunchContext());
+
+    } finally {
+      clr.stop();
+      verify(clr.getTestContainerLauncher(0)).shutdown();
+      verify(clr.getTestContainerLauncher(1)).shutdown();
+    }
+  }
+
+  private static class ContainerLaucherRouterForMultipleLauncherTest
+      extends ContainerLauncherRouter {
+
+    // All variables setup as static since methods being overridden are invoked by the ContainerLauncherRouter ctor,
+    // and regular variables will not be initialized at this point.
+    private static final AtomicInteger numContainerLaunchers = new AtomicInteger(0);
+    private static final Set<Integer> containerLauncherIndices = new HashSet<>();
+    private static final ContainerLauncher yarnContainerLauncher = mock(ContainerLauncher.class);
+    private static final ContainerLauncher uberContainerlauncher = mock(ContainerLauncher.class);
+    private static final AtomicBoolean yarnContainerLauncherCreated = new AtomicBoolean(false);
+    private static final AtomicBoolean uberContainerLauncherCreated = new AtomicBoolean(false);
+
+    private static final List<ContainerLauncherContext> containerLauncherContexts =
+        new LinkedList<>();
+    private static final List<String> containerLauncherNames = new LinkedList<>();
+    private static final List<ContainerLauncher> testContainerLaunchers = new LinkedList<>();
+
+
+    public static void reset() {
+      numContainerLaunchers.set(0);
+      containerLauncherIndices.clear();
+      yarnContainerLauncherCreated.set(false);
+      uberContainerLauncherCreated.set(false);
+      containerLauncherContexts.clear();
+      containerLauncherNames.clear();
+      testContainerLaunchers.clear();
+    }
+
+    public ContainerLaucherRouterForMultipleLauncherTest(AppContext context,
+                                                         TaskAttemptListener taskAttemptListener,
+                                                         String workingDirectory,
+                                                         List<NamedEntityDescriptor> containerLauncherDescriptors,
+                                                         boolean isPureLocalMode) throws
+        UnknownHostException {
+      super(context, taskAttemptListener, workingDirectory,
+          containerLauncherDescriptors, isPureLocalMode);
+    }
+
+    @Override
+    ContainerLauncher createContainerLauncher(NamedEntityDescriptor containerLauncherDescriptor,
+                                              AppContext context,
+                                              ContainerLauncherContext containerLauncherContext,
+                                              TaskAttemptListener taskAttemptListener,
+                                              String workingDirectory,
+                                              int containerLauncherIndex,
+                                              boolean isPureLocalMode) throws
+        UnknownHostException {
+      numContainerLaunchers.incrementAndGet();
+      boolean added = containerLauncherIndices.add(containerLauncherIndex);
+      assertTrue("Cannot add multiple launchers with the same index", added);
+      containerLauncherNames.add(containerLauncherDescriptor.getEntityName());
+      containerLauncherContexts.add(containerLauncherContext);
+      return super
+          .createContainerLauncher(containerLauncherDescriptor, context, containerLauncherContext,
+              taskAttemptListener, workingDirectory, containerLauncherIndex, isPureLocalMode);
+    }
+
+    @Override
+    ContainerLauncher createYarnContainerLauncher(
+        ContainerLauncherContext containerLauncherContext) {
+      yarnContainerLauncherCreated.set(true);
+      testContainerLaunchers.add(yarnContainerLauncher);
+      return yarnContainerLauncher;
+    }
+
+    @Override
+    ContainerLauncher createUberContainerLauncher(ContainerLauncherContext containerLauncherContext,
+                                                  AppContext context,
+                                                  TaskAttemptListener taskAttemptListener,
+                                                  String workingDirectory,
+                                                  boolean isPureLocalMode) throws
+        UnknownHostException {
+      uberContainerLauncherCreated.set(true);
+      testContainerLaunchers.add(uberContainerlauncher);
+      return uberContainerlauncher;
+    }
+
+    @Override
+    ContainerLauncher createCustomContainerLauncher(
+        ContainerLauncherContext containerLauncherContext,
+        NamedEntityDescriptor containerLauncherDescriptor) {
+      ContainerLauncher spyLauncher = spy(super.createCustomContainerLauncher(
+          containerLauncherContext, containerLauncherDescriptor));
+      testContainerLaunchers.add(spyLauncher);
+      return spyLauncher;
+    }
+
+    public int getNumContainerLaunchers() {
+      return numContainerLaunchers.get();
+    }
+
+    public boolean getYarnContainerLauncherCreated() {
+      return yarnContainerLauncherCreated.get();
+    }
+
+    public boolean getUberContainerLauncherCreated() {
+      return uberContainerLauncherCreated.get();
+    }
+
+    public String getContainerLauncherName(int containerLauncherIndex) {
+      return containerLauncherNames.get(containerLauncherIndex);
+    }
+
+    public ContainerLauncher getTestContainerLauncher(int containerLauncherIndex) {
+      return testContainerLaunchers.get(containerLauncherIndex);
+    }
+
+    public ContainerLauncherContext getContainerLauncherContext(int containerLauncherIndex) {
+      return containerLauncherContexts.get(containerLauncherIndex);
+    }
+  }
+
+  private static class FakeContainerLauncher extends ContainerLauncher {
+
+    public FakeContainerLauncher(
+        ContainerLauncherContext containerLauncherContext) {
+      super(containerLauncherContext);
+    }
+
+    @Override
+    public void launchContainer(ContainerLaunchRequest launchRequest) {
+
+    }
+
+    @Override
+    public void stopContainer(ContainerStopRequest stopRequest) {
+
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/tez/blob/267fe737/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 f8aa1e2..3e68a4c 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
@@ -19,22 +19,30 @@
 package org.apache.tez.dag.app.rm;
 
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.eq;
 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.spy;
 import static org.mockito.Mockito.times;
 import static org.mockito.Mockito.verify;
 import static org.mockito.Mockito.when;
 
 import java.io.IOException;
 import java.net.InetSocketAddress;
+import java.nio.ByteBuffer;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.LinkedList;
 import java.util.List;
+import java.util.Set;
 import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.security.Credentials;
@@ -44,6 +52,7 @@ import org.apache.hadoop.yarn.api.records.ContainerExitStatus;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.LocalResource;
+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.event.Event;
@@ -53,12 +62,13 @@ import org.apache.tez.common.TezUtils;
 import org.apache.tez.dag.api.NamedEntityDescriptor;
 import org.apache.tez.dag.api.TaskLocationHint;
 import org.apache.tez.dag.api.TezConfiguration;
-import org.apache.tez.dag.api.TezUncheckedException;
+import org.apache.tez.dag.api.TezConstants;
 import org.apache.tez.dag.api.UserPayload;
 import org.apache.tez.dag.api.client.DAGClientServer;
 import org.apache.tez.dag.app.AppContext;
 import org.apache.tez.dag.app.ContainerContext;
 import org.apache.tez.dag.app.ServicePluginLifecycleAbstractService;
+import org.apache.tez.dag.app.dag.TaskAttempt;
 import org.apache.tez.dag.app.dag.impl.TaskAttemptImpl;
 import org.apache.tez.dag.app.dag.impl.TaskImpl;
 import org.apache.tez.dag.app.dag.impl.VertexImpl;
@@ -70,8 +80,14 @@ import org.apache.tez.dag.app.rm.container.AMContainerMap;
 import org.apache.tez.dag.app.rm.container.AMContainerState;
 import org.apache.tez.dag.app.web.WebUIService;
 import org.apache.tez.dag.records.TaskAttemptTerminationCause;
+import org.apache.tez.dag.records.TezDAGID;
 import org.apache.tez.dag.records.TezTaskAttemptID;
+import org.apache.tez.dag.records.TezTaskID;
+import org.apache.tez.dag.records.TezVertexID;
+import org.apache.tez.runtime.api.impl.TaskSpec;
+import org.apache.tez.serviceplugins.api.TaskAttemptEndReason;
 import org.apache.tez.serviceplugins.api.TaskScheduler;
+import org.apache.tez.serviceplugins.api.TaskSchedulerContext;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
@@ -95,10 +111,9 @@ public class TestTaskSchedulerEventHandler {
     
     public MockTaskSchedulerEventHandler(AppContext appContext,
         DAGClientServer clientService, EventHandler eventHandler,
-        ContainerSignatureMatcher containerSignatureMatcher, WebUIService webUI,
-        UserPayload defaultPayload) {
+        ContainerSignatureMatcher containerSignatureMatcher, WebUIService webUI) {
       super(appContext, clientService, eventHandler, containerSignatureMatcher, webUI,
-          new LinkedList<NamedEntityDescriptor>(), defaultPayload, false);
+          Lists.newArrayList(new NamedEntityDescriptor("FakeDescriptor", null)), false);
     }
 
     @Override
@@ -140,14 +155,8 @@ public class TestTaskSchedulerEventHandler {
     when(mockAppContext.getAllContainers()).thenReturn(mockAMContainerMap);
     when(mockClientService.getBindAddress()).thenReturn(new InetSocketAddress(10000));
     Configuration conf = new Configuration(false);
-    UserPayload userPayload;
-    try {
-      userPayload = TezUtils.createUserPayloadFromConf(conf);
-    } catch (IOException e) {
-      throw new TezUncheckedException(e);
-    }
     schedulerHandler = new MockTaskSchedulerEventHandler(
-        mockAppContext, mockClientService, mockEventHandler, mockSigMatcher, mockWebUIService, userPayload);
+        mockAppContext, mockClientService, mockEventHandler, mockSigMatcher, mockWebUIService);
   }
 
   @Test(timeout = 5000)
@@ -272,7 +281,7 @@ public class TestTaskSchedulerEventHandler {
     when(mockAmContainer.getContainerLauncherIdentifier()).thenReturn(0);
     when(mockAmContainer.getTaskCommunicatorIdentifier()).thenReturn(0);
     ContainerId mockCId = mock(ContainerId.class);
-    verify(mockTaskScheduler, times(0)).deallocateContainer((ContainerId)any());
+    verify(mockTaskScheduler, times(0)).deallocateContainer((ContainerId) any());
     when(mockAMContainerMap.get(mockCId)).thenReturn(mockAmContainer);
     schedulerHandler.preemptContainer(0, mockCId);
     verify(mockTaskScheduler, times(1)).deallocateContainer(mockCId);
@@ -400,5 +409,300 @@ public class TestTaskSchedulerEventHandler {
 
   }
 
-  // TODO TEZ-2003. Add tests with multiple schedulers, and ensuring that events go out with correct IDs.
+  @Test(timeout = 5000)
+  public void testNoSchedulerSpecified() throws IOException {
+    try {
+      TSEHForMultipleSchedulersTest tseh =
+          new TSEHForMultipleSchedulersTest(mockAppContext, mockClientService, mockEventHandler,
+              mockSigMatcher, mockWebUIService, null, false);
+      fail("Expecting an IllegalStateException with no schedulers specified");
+    } catch (IllegalArgumentException e) {
+    }
+  }
+
+  // Verified via statics
+  @Test(timeout = 5000)
+  public void testCustomTaskSchedulerSetup() throws IOException {
+    Configuration conf = new Configuration(false);
+    conf.set("testkey", "testval");
+    UserPayload defaultPayload = TezUtils.createUserPayloadFromConf(conf);
+
+    String customSchedulerName = "fakeScheduler";
+    List<NamedEntityDescriptor> taskSchedulers = new LinkedList<>();
+    ByteBuffer bb = ByteBuffer.allocate(4);
+    bb.putInt(0, 3);
+    UserPayload userPayload = UserPayload.create(bb);
+    taskSchedulers.add(
+        new NamedEntityDescriptor(customSchedulerName, FakeTaskScheduler.class.getName())
+            .setUserPayload(userPayload));
+    taskSchedulers.add(new NamedEntityDescriptor(TezConstants.getTezYarnServicePluginName(), null)
+        .setUserPayload(defaultPayload));
+
+    TSEHForMultipleSchedulersTest tseh =
+        new TSEHForMultipleSchedulersTest(mockAppContext, mockClientService, mockEventHandler,
+            mockSigMatcher, mockWebUIService, taskSchedulers, false);
+
+    tseh.init(conf);
+    tseh.start();
+
+    // Verify that the YARN task scheduler is installed by default
+    assertTrue(tseh.getYarnSchedulerCreated());
+    assertFalse(tseh.getUberSchedulerCreated());
+    assertEquals(2, tseh.getNumCreateInvocations());
+
+    // Verify the order of the schedulers
+    assertEquals(customSchedulerName, tseh.getTaskSchedulerName(0));
+    assertEquals(TezConstants.getTezYarnServicePluginName(), tseh.getTaskSchedulerName(1));
+
+    // Verify the payload setup for the custom task scheduler
+    assertNotNull(tseh.getTaskSchedulerContext(0));
+    assertEquals(bb, tseh.getTaskSchedulerContext(0).getInitialUserPayload().getPayload());
+
+    // Verify the payload on the yarn scheduler
+    assertNotNull(tseh.getTaskSchedulerContext(1));
+    Configuration parsed = TezUtils.createConfFromUserPayload(tseh.getTaskSchedulerContext(1).getInitialUserPayload());
+    assertEquals("testval", parsed.get("testkey"));
+  }
+
+  @Test(timeout = 5000)
+  public void testTaskSchedulerRouting() throws Exception {
+    Configuration conf = new Configuration(false);
+    UserPayload defaultPayload = TezUtils.createUserPayloadFromConf(conf);
+
+    String customSchedulerName = "fakeScheduler";
+    List<NamedEntityDescriptor> taskSchedulers = new LinkedList<>();
+    ByteBuffer bb = ByteBuffer.allocate(4);
+    bb.putInt(0, 3);
+    UserPayload userPayload = UserPayload.create(bb);
+    taskSchedulers.add(
+        new NamedEntityDescriptor(customSchedulerName, FakeTaskScheduler.class.getName())
+            .setUserPayload(userPayload));
+    taskSchedulers.add(new NamedEntityDescriptor(TezConstants.getTezYarnServicePluginName(), null)
+        .setUserPayload(defaultPayload));
+
+    TSEHForMultipleSchedulersTest tseh =
+        new TSEHForMultipleSchedulersTest(mockAppContext, mockClientService, mockEventHandler,
+            mockSigMatcher, mockWebUIService, taskSchedulers, false);
+
+    tseh.init(conf);
+    tseh.start();
+
+    // Verify that the YARN task scheduler is installed by default
+    assertTrue(tseh.getYarnSchedulerCreated());
+    assertFalse(tseh.getUberSchedulerCreated());
+    assertEquals(2, tseh.getNumCreateInvocations());
+
+    // Verify the order of the schedulers
+    assertEquals(customSchedulerName, tseh.getTaskSchedulerName(0));
+    assertEquals(TezConstants.getTezYarnServicePluginName(), tseh.getTaskSchedulerName(1));
+
+    verify(tseh.getTestTaskScheduler(0)).initialize();
+    verify(tseh.getTestTaskScheduler(0)).start();
+
+    ApplicationId appId = ApplicationId.newInstance(1000, 1);
+    TezDAGID dagId = TezDAGID.getInstance(appId, 1);
+    TezVertexID vertexID = TezVertexID.getInstance(dagId, 1);
+    TezTaskID taskId1 = TezTaskID.getInstance(vertexID, 1);
+    TezTaskAttemptID attemptId11 = TezTaskAttemptID.getInstance(taskId1, 1);
+    TezTaskID taskId2 = TezTaskID.getInstance(vertexID, 2);
+    TezTaskAttemptID attemptId21 = TezTaskAttemptID.getInstance(taskId2, 1);
+
+    Resource resource = Resource.newInstance(1024, 1);
+
+    TaskAttempt mockTaskAttempt1 = mock(TaskAttempt.class);
+    TaskAttempt mockTaskAttempt2 = mock(TaskAttempt.class);
+
+    AMSchedulerEventTALaunchRequest launchRequest1 =
+        new AMSchedulerEventTALaunchRequest(attemptId11, resource, mock(TaskSpec.class),
+            mockTaskAttempt1, mock(TaskLocationHint.class), 1, mock(ContainerContext.class), 0, 0,
+            0);
+
+    tseh.handle(launchRequest1);
+
+    verify(tseh.getTestTaskScheduler(0)).allocateTask(eq(mockTaskAttempt1), eq(resource),
+        any(String[].class), any(String[].class), any(Priority.class), any(Object.class),
+        eq(launchRequest1));
+
+    AMSchedulerEventTALaunchRequest launchRequest2 =
+        new AMSchedulerEventTALaunchRequest(attemptId21, resource, mock(TaskSpec.class),
+            mockTaskAttempt2, mock(TaskLocationHint.class), 1, mock(ContainerContext.class), 1, 0,
+            0);
+    tseh.handle(launchRequest2);
+    verify(tseh.getTestTaskScheduler(1)).allocateTask(eq(mockTaskAttempt2), eq(resource),
+        any(String[].class), any(String[].class), any(Priority.class), any(Object.class),
+        eq(launchRequest2));
+  }
+
+  private static class TSEHForMultipleSchedulersTest extends TaskSchedulerEventHandler {
+
+    private final TaskScheduler yarnTaskScheduler;
+    private final TaskScheduler uberTaskScheduler;
+    private final AtomicBoolean uberSchedulerCreated = new AtomicBoolean(false);
+    private final AtomicBoolean yarnSchedulerCreated = new AtomicBoolean(false);
+    private final AtomicInteger numCreateInvocations = new AtomicInteger(0);
+    private final Set<Integer> seenSchedulers = new HashSet<>();
+    private final List<TaskSchedulerContext> taskSchedulerContexts = new LinkedList<>();
+    private final List<String> taskSchedulerNames = new LinkedList<>();
+    private final List<TaskScheduler> testTaskSchedulers = new LinkedList<>();
+
+    public TSEHForMultipleSchedulersTest(AppContext appContext,
+                                         DAGClientServer clientService,
+                                         EventHandler eventHandler,
+                                         ContainerSignatureMatcher containerSignatureMatcher,
+                                         WebUIService webUI,
+                                         List<NamedEntityDescriptor> schedulerDescriptors,
+                                         boolean isPureLocalMode) {
+      super(appContext, clientService, eventHandler, containerSignatureMatcher, webUI,
+          schedulerDescriptors, isPureLocalMode);
+      yarnTaskScheduler = mock(TaskScheduler.class);
+      uberTaskScheduler = mock(TaskScheduler.class);
+    }
+
+    @Override
+    TaskScheduler createTaskScheduler(String host, int port, String trackingUrl,
+                                      AppContext appContext,
+                                      NamedEntityDescriptor taskSchedulerDescriptor,
+                                      long customAppIdIdentifier,
+                                      int schedulerId) {
+
+      numCreateInvocations.incrementAndGet();
+      boolean added = seenSchedulers.add(schedulerId);
+      assertTrue("Cannot add multiple schedulers with the same schedulerId", added);
+      taskSchedulerNames.add(taskSchedulerDescriptor.getEntityName());
+      return super.createTaskScheduler(host, port, trackingUrl, appContext, taskSchedulerDescriptor,
+          customAppIdIdentifier, schedulerId);
+    }
+
+    @Override
+    TaskSchedulerContext wrapTaskSchedulerContext(TaskSchedulerContext rawContext) {
+      // Avoid wrapping in threads
+      return rawContext;
+    }
+
+    @Override
+    TaskScheduler createYarnTaskScheduler(TaskSchedulerContext taskSchedulerContext, int schedulerId) {
+      taskSchedulerContexts.add(taskSchedulerContext);
+      testTaskSchedulers.add(yarnTaskScheduler);
+      yarnSchedulerCreated.set(true);
+      return yarnTaskScheduler;
+    }
+
+    @Override
+    TaskScheduler createUberTaskScheduler(TaskSchedulerContext taskSchedulerContext, int schedulerId) {
+      taskSchedulerContexts.add(taskSchedulerContext);
+      uberSchedulerCreated.set(true);
+      testTaskSchedulers.add(yarnTaskScheduler);
+      return uberTaskScheduler;
+    }
+
+    @Override
+    TaskScheduler createCustomTaskScheduler(TaskSchedulerContext taskSchedulerContext,
+                                            NamedEntityDescriptor taskSchedulerDescriptor, int schedulerId) {
+      taskSchedulerContexts.add(taskSchedulerContext);
+      TaskScheduler taskScheduler = spy(super.createCustomTaskScheduler(taskSchedulerContext, taskSchedulerDescriptor, schedulerId));
+      testTaskSchedulers.add(taskScheduler);
+      return taskScheduler;
+    }
+
+    @Override
+    // Inline handling of events.
+    public void handle(AMSchedulerEvent event) {
+      handleEvent(event);
+    }
+
+    public boolean getUberSchedulerCreated() {
+      return uberSchedulerCreated.get();
+    }
+
+    public boolean getYarnSchedulerCreated() {
+      return yarnSchedulerCreated.get();
+    }
+
+    public int getNumCreateInvocations() {
+      return numCreateInvocations.get();
+    }
+
+    public TaskSchedulerContext getTaskSchedulerContext(int schedulerId) {
+      return taskSchedulerContexts.get(schedulerId);
+    }
+
+    public String getTaskSchedulerName(int schedulerId) {
+      return taskSchedulerNames.get(schedulerId);
+    }
+
+    public TaskScheduler getTestTaskScheduler(int schedulerId) {
+      return testTaskSchedulers.get(schedulerId);
+    }
+  }
+
+  public static class FakeTaskScheduler extends TaskScheduler {
+
+    public FakeTaskScheduler(
+        TaskSchedulerContext taskSchedulerContext) {
+      super(taskSchedulerContext);
+    }
+
+    @Override
+    public Resource getAvailableResources() {
+      return null;
+    }
+
+    @Override
+    public int getClusterNodeCount() {
+      return 0;
+    }
+
+    @Override
+    public void dagComplete() {
+
+    }
+
+    @Override
+    public Resource getTotalResources() {
+      return null;
+    }
+
+    @Override
+    public void blacklistNode(NodeId nodeId) {
+
+    }
+
+    @Override
+    public void unblacklistNode(NodeId nodeId) {
+
+    }
+
+    @Override
+    public void allocateTask(Object task, Resource capability, String[] hosts, String[] racks,
+                             Priority priority, Object containerSignature, Object clientCookie) {
+
+    }
+
+    @Override
+    public void allocateTask(Object task, Resource capability, ContainerId containerId,
+                             Priority priority, Object containerSignature, Object clientCookie) {
+
+    }
+
+    @Override
+    public boolean deallocateTask(Object task, boolean taskSucceeded,
+                                  TaskAttemptEndReason endReason) {
+      return false;
+    }
+
+    @Override
+    public Object deallocateContainer(ContainerId containerId) {
+      return null;
+    }
+
+    @Override
+    public void setShouldUnregister() {
+
+    }
+
+    @Override
+    public boolean hasUnregistered() {
+      return false;
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/267fe737/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 59ab00a..0746507 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
@@ -42,6 +42,7 @@ import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
 
 import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.service.AbstractService;
@@ -138,7 +139,8 @@ class TestTaskSchedulerHelpers {
         ContainerSignatureMatcher containerSignatureMatcher,
         UserPayload defaultPayload) {
       super(appContext, null, eventHandler, containerSignatureMatcher, null,
-          new LinkedList<NamedEntityDescriptor>(), defaultPayload, false);
+          Lists.newArrayList(new NamedEntityDescriptor("FakeScheduler", null)),
+          false);
       this.amrmClientAsync = amrmClientAsync;
       this.containerSignatureMatcher = containerSignatureMatcher;
       this.defaultPayload = defaultPayload;


[16/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/c44b3379
Tree: http://git-wip-us.apache.org/repos/asf/tez/tree/c44b3379
Diff: http://git-wip-us.apache.org/repos/asf/tez/diff/c44b3379

Branch: refs/heads/TEZ-2003
Commit: c44b3379b9886eea4b0914c375c49cca41952b44
Parents: 5e66ee7
Author: Siddharth Seth <ss...@apache.org>
Authored: Mon Apr 20 13:17:31 2015 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Fri Aug 14 13:46:43 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   |  2 +
 .../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, 145 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/c44b3379/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/c44b3379/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/c44b3379/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/c44b3379/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/c44b3379/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 47b56f2..a011b61 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
@@ -72,4 +72,6 @@ public interface Task {
   public TaskSpec getBaseTaskSpec();
   
   public TaskLocationHint getTaskLocationHint();
+
+  long getFirstAttemptStartTime();
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/c44b3379/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 4360cc3..cbe72c1 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
@@ -125,6 +125,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/c44b3379/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 ef2df78..e37fc2f 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
@@ -702,6 +702,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/c44b3379/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 dfb2618..cb26c55 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
@@ -134,6 +134,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;
@@ -699,6 +700,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 {
@@ -1060,6 +1071,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/c44b3379/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 e6027f5..93b4c3f 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
@@ -1529,7 +1529,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 static class KillTransition
     implements SingleArcTransition<TaskImpl, TaskEvent> {
     @Override


[03/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/699634f7
Tree: http://git-wip-us.apache.org/repos/asf/tez/tree/699634f7
Diff: http://git-wip-us.apache.org/repos/asf/tez/diff/699634f7

Branch: refs/heads/TEZ-2003
Commit: 699634f7b7fec5f80c41ad5d1da78e436778169f
Parents: d25f9af
Author: Siddharth Seth <ss...@apache.org>
Authored: Fri Feb 20 11:59:03 2015 -0800
Committer: Siddharth Seth <ss...@apache.org>
Committed: Fri Aug 14 13:46:42 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   |  27 ++--
 .../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   |  38 +++--
 .../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  |  25 ++-
 .../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, 630 insertions(+), 357 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/699634f7/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/699634f7/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 431a8b2..a304b37 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
@@ -517,7 +517,7 @@ public class DAGAppMaster extends AbstractService {
 
     this.taskSchedulerEventHandler = new TaskSchedulerEventHandler(context,
         clientRpcServer, dispatcher.getEventHandler(), containerSignatureMatcher, webUIService,
-        taskSchedulerClassIdentifiers);
+        taskSchedulerClassIdentifiers, isLocal);
     addIfService(taskSchedulerEventHandler, true);
 
     if (enableWebUIService()) {
@@ -2341,6 +2341,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);
 
@@ -2378,6 +2379,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/699634f7/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/699634f7/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 3d9abdf..05c4623 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
@@ -272,11 +272,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
@@ -296,7 +291,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");
     }
@@ -306,11 +301,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);
     }
@@ -318,12 +314,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: "
@@ -353,13 +349,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");
@@ -373,7 +369,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) {

http://git-wip-us.apache.org/repos/asf/tez/blob/699634f7/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 36e84f0..e408345 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
@@ -1227,7 +1227,7 @@ public class TaskAttemptImpl implements TaskAttempt,
       // Inform the scheduler
       if (sendSchedulerEvent()) {
         ta.sendEvent(new AMSchedulerEventTAEnded(ta, ta.containerId, helper
-            .getTaskAttemptState()));
+            .getTaskAttemptState(), ta.getVertex().getTaskSchedulerIdentifier()));
       }
     }
   }
@@ -1309,7 +1309,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/699634f7/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/699634f7/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 0583a0b..7733ef9 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
@@ -4536,7 +4536,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/699634f7/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 9476860..247b92f 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
@@ -294,7 +294,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/699634f7/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/699634f7/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 d9d668f..3c27678 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);
@@ -223,7 +221,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/699634f7/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/699634f7/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/699634f7/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/699634f7/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/699634f7/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/699634f7/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/699634f7/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/699634f7/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 cb109ae..ae83730 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/699634f7/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 095a15a..4b2d528 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 long getCurrentTaskAttemptAllocationTime();
+  public int getTaskSchedulerIdentifier();
+  public int getContainerLauncherIdentifier();
+  public int getTaskCommunicatorIdentifier();
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/699634f7/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/699634f7/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 54f24d6..8685556 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>();
@@ -303,7 +306,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();
@@ -315,6 +318,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);
   }
 
@@ -374,6 +380,20 @@ public class AMContainerImpl implements AMContainer {
     }
   }
 
+  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;
   }
@@ -443,7 +463,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());
 
@@ -1026,7 +1046,7 @@ public class AMContainerImpl implements AMContainer {
   }
   
   protected void deAllocate() {
-    sendEvent(new AMSchedulerEventDeallocateContainer(containerId));
+    sendEvent(new AMSchedulerEventDeallocateContainer(containerId, schedulerId));
   }
 
   protected void sendTerminatedToTaskAttempt(
@@ -1056,28 +1076,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/699634f7/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/699634f7/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/699634f7/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 8763a0c..b70b9ea 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
@@ -198,7 +198,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/699634f7/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 219217f..200e737 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
@@ -125,11 +125,10 @@ public class TestTaskAttemptListenerImplTezDag {
     doReturn(container).when(amContainer).getContainer();
 
     taskAttemptListener = new TaskAttemptListenerImplForTest(appContext,
-        mock(TaskHeartbeatHandler.class), mock(ContainerHeartbeatHandler.class), null);
+        mock(TaskHeartbeatHandler.class), mock(ContainerHeartbeatHandler.class), null, null);
     TezTaskCommunicatorImpl taskCommunicator = (TezTaskCommunicatorImpl)taskAttemptListener.getTaskCommunicator();
     TezTaskUmbilicalProtocol tezUmbilical = taskCommunicator.getUmbilical();
 
-
     taskSpec = mock(TaskSpec.class);
     doReturn(taskAttemptID).when(taskSpec).getTaskAttemptID();
     amContainerTask = new AMContainerTask(taskSpec, null, null, false, 0);
@@ -139,7 +138,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);
@@ -149,55 +148,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/699634f7/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 d21f715..d6fc46e 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;
@@ -285,8 +286,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);
@@ -335,8 +337,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);
@@ -357,7 +360,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();
@@ -436,8 +439,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);
@@ -458,7 +462,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();
@@ -501,8 +505,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);
@@ -523,7 +528,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();
@@ -593,8 +598,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);
@@ -616,7 +622,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();
@@ -724,8 +730,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);
@@ -747,7 +754,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();
@@ -816,8 +823,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);
@@ -838,7 +846,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();
@@ -911,8 +919,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);
@@ -933,7 +942,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();
@@ -1014,8 +1023,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);
@@ -1036,7 +1046,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();
@@ -1114,8 +1124,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);
@@ -1136,7 +1147,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/699634f7/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 8864e9f..6ee741a 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
@@ -2196,7 +2196,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();
     doReturn(clock).when(appContext).getClock();
@@ -3229,7 +3229,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));
@@ -3264,7 +3264,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));
@@ -3300,7 +3300,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));


[44/50] [abbrv] tez git commit: TEZ-2126. Add unit tests for verifying multiple schedulers, launchers, communicators. (sseth)

Posted by ss...@apache.org.
TEZ-2126. Add unit tests for verifying multiple schedulers, launchers,
communicators. (sseth)


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

Branch: refs/heads/TEZ-2003
Commit: 267fe73727b9638bd19979d82174f925de0c2a38
Parents: f30a3fe
Author: Siddharth Seth <ss...@apache.org>
Authored: Thu Aug 6 01:04:31 2015 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Fri Aug 14 13:47:10 2015 -0700

----------------------------------------------------------------------
 TEZ-2003-CHANGES.txt                            |   1 +
 .../tez/dag/api/NamedEntityDescriptor.java      |   7 +
 .../org/apache/tez/dag/app/DAGAppMaster.java    | 163 ++++----
 .../dag/app/TaskAttemptListenerImpTezDag.java   |  94 ++---
 .../apache/tez/dag/app/dag/impl/VertexImpl.java |   9 +-
 .../app/launcher/ContainerLauncherRouter.java   | 126 ++++---
 .../dag/app/rm/TaskSchedulerEventHandler.java   | 137 +++----
 .../apache/tez/dag/app/MockDAGAppMaster.java    |   3 +-
 .../apache/tez/dag/app/TestDAGAppMaster.java    | 300 +++++++++++++++
 .../app/TestTaskAttemptListenerImplTezDag.java  |  44 ++-
 .../app/TestTaskAttemptListenerImplTezDag2.java |   6 +-
 .../dag/app/TestTaskCommunicatorManager.java    | 369 +++++++++++++++++++
 .../tez/dag/app/dag/impl/TestVertexImpl2.java   | 279 ++++++++++++--
 .../launcher/TestContainerLauncherRouter.java   | 361 ++++++++++++++++++
 .../app/rm/TestTaskSchedulerEventHandler.java   | 330 ++++++++++++++++-
 .../dag/app/rm/TestTaskSchedulerHelpers.java    |   4 +-
 16 files changed, 1907 insertions(+), 326 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/267fe737/TEZ-2003-CHANGES.txt
----------------------------------------------------------------------
diff --git a/TEZ-2003-CHANGES.txt b/TEZ-2003-CHANGES.txt
index c7a3dcc..f921739 100644
--- a/TEZ-2003-CHANGES.txt
+++ b/TEZ-2003-CHANGES.txt
@@ -42,5 +42,6 @@ ALL CHANGES:
   TEZ-2441. Add tests for TezTaskRunner2.
   TEZ-2657. Add tests for client side changes - specifying plugins, etc.
   TEZ-2626. Fix log lines with DEBUG in messages, consolidate TEZ-2003 TODOs.
+  TEZ-2126. Add unit tests for verifying multiple schedulers, launchers, communicators.
 
 INCOMPATIBLE CHANGES:

http://git-wip-us.apache.org/repos/asf/tez/blob/267fe737/tez-api/src/main/java/org/apache/tez/dag/api/NamedEntityDescriptor.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/NamedEntityDescriptor.java b/tez-api/src/main/java/org/apache/tez/dag/api/NamedEntityDescriptor.java
index 723d43f..17c8c6c 100644
--- a/tez-api/src/main/java/org/apache/tez/dag/api/NamedEntityDescriptor.java
+++ b/tez-api/src/main/java/org/apache/tez/dag/api/NamedEntityDescriptor.java
@@ -35,4 +35,11 @@ public class NamedEntityDescriptor<T extends NamedEntityDescriptor<T>> extends E
     super.setUserPayload(userPayload);
     return (T) this;
   }
+
+  @Override
+  public String toString() {
+    boolean hasPayload =
+        getUserPayload() == null ? false : getUserPayload().getPayload() == null ? false : true;
+    return "EntityName=" + entityName + ", ClassName=" + getClassName() + ", hasPayload=" + hasPayload;
+  }
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/267fe737/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 9b16a90..ed4f520 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
@@ -59,6 +59,7 @@ import java.util.regex.Pattern;
 
 import com.google.common.collect.BiMap;
 import com.google.common.collect.HashBiMap;
+import com.google.common.collect.Lists;
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.cli.GnuParser;
 import org.apache.commons.cli.Options;
@@ -389,42 +390,16 @@ public class DAGAppMaster extends AbstractService {
     this.isLocal = conf.getBoolean(TezConfiguration.TEZ_LOCAL_MODE,
         TezConfiguration.TEZ_LOCAL_MODE_DEFAULT);
 
-    List<NamedEntityDescriptor> taskSchedulerDescriptors;
-    List<NamedEntityDescriptor> containerLauncherDescriptors;
-    List<NamedEntityDescriptor> taskCommunicatorDescriptors;
-    boolean tezYarnEnabled = true;
-    boolean uberEnabled = false;
-
-    if (!isLocal) {
-      if (amPluginDescriptorProto == null) {
-        tezYarnEnabled = true;
-        uberEnabled = false;
-      } else {
-        tezYarnEnabled = amPluginDescriptorProto.getContainersEnabled();
-        uberEnabled = amPluginDescriptorProto.getUberEnabled();
-      }
-    } else {
-      tezYarnEnabled = false;
-      uberEnabled = true;
-    }
+    UserPayload defaultPayload = TezUtils.createUserPayloadFromConf(amConf);
 
-    taskSchedulerDescriptors = parsePlugin(taskSchedulers,
-        (amPluginDescriptorProto == null || amPluginDescriptorProto.getTaskSchedulersCount() == 0 ?
-            null :
-            amPluginDescriptorProto.getTaskSchedulersList()),
-        tezYarnEnabled, uberEnabled);
+    List<NamedEntityDescriptor> taskSchedulerDescriptors = Lists.newLinkedList();
+    List<NamedEntityDescriptor> containerLauncherDescriptors = Lists.newLinkedList();
+    List<NamedEntityDescriptor> taskCommunicatorDescriptors = Lists.newLinkedList();
 
-    containerLauncherDescriptors = parsePlugin(containerLaunchers,
-        (amPluginDescriptorProto == null ||
-            amPluginDescriptorProto.getContainerLaunchersCount() == 0 ? null :
-            amPluginDescriptorProto.getContainerLaunchersList()),
-        tezYarnEnabled, uberEnabled);
+    parseAllPlugins(taskSchedulerDescriptors, taskSchedulers, containerLauncherDescriptors,
+        containerLaunchers, taskCommunicatorDescriptors, taskCommunicators, amPluginDescriptorProto,
+        isLocal, defaultPayload);
 
-    taskCommunicatorDescriptors = parsePlugin(taskCommunicators,
-        (amPluginDescriptorProto == null ||
-            amPluginDescriptorProto.getTaskCommunicatorsCount() == 0 ? null :
-            amPluginDescriptorProto.getTaskCommunicatorsList()),
-        tezYarnEnabled, uberEnabled);
 
 
     LOG.info(buildPluginComponentLog(taskSchedulerDescriptors, taskSchedulers, "TaskSchedulers"));
@@ -494,12 +469,11 @@ public class DAGAppMaster extends AbstractService {
     jobTokenSecretManager.addTokenForJob(
         appAttemptID.getApplicationId().toString(), sessionToken);
 
-    UserPayload defaultPayload = TezUtils.createUserPayloadFromConf(amConf);
+
 
     //service to handle requests to TaskUmbilicalProtocol
     taskAttemptListener = createTaskAttemptListener(context,
-        taskHeartbeatHandler, containerHeartbeatHandler, taskCommunicatorDescriptors,
-        defaultPayload, isLocal);
+        taskHeartbeatHandler, containerHeartbeatHandler, taskCommunicatorDescriptors);
     addIfService(taskAttemptListener, true);
 
     containerSignatureMatcher = createContainerSignatureMatcher();
@@ -549,7 +523,7 @@ public class DAGAppMaster extends AbstractService {
 
     this.taskSchedulerEventHandler = new TaskSchedulerEventHandler(context,
         clientRpcServer, dispatcher.getEventHandler(), containerSignatureMatcher, webUIService,
-        taskSchedulerDescriptors, defaultPayload, isLocal);
+        taskSchedulerDescriptors, isLocal);
     addIfService(taskSchedulerEventHandler, true);
 
     if (enableWebUIService()) {
@@ -567,7 +541,7 @@ public class DAGAppMaster extends AbstractService {
         taskSchedulerEventHandler);
     addIfServiceDependency(taskSchedulerEventHandler, clientRpcServer);
 
-    this.containerLauncherRouter = createContainerLauncherRouter(defaultPayload, containerLauncherDescriptors, isLocal);
+    this.containerLauncherRouter = createContainerLauncherRouter(containerLauncherDescriptors, isLocal);
     addIfService(containerLauncherRouter, true);
     dispatcher.register(NMCommunicatorEventType.class, containerLauncherRouter);
 
@@ -1077,12 +1051,9 @@ public class DAGAppMaster extends AbstractService {
   protected TaskAttemptListener createTaskAttemptListener(AppContext context,
                                                           TaskHeartbeatHandler thh,
                                                           ContainerHeartbeatHandler chh,
-                                                          List<NamedEntityDescriptor> entityDescriptors,
-                                                          UserPayload defaultUserPayload,
-                                                          boolean isLocal) {
+                                                          List<NamedEntityDescriptor> entityDescriptors) {
     TaskAttemptListener lis =
-        new TaskAttemptListenerImpTezDag(context, thh, chh,
-            entityDescriptors, defaultUserPayload, isLocal);
+        new TaskAttemptListenerImpTezDag(context, thh, chh, entityDescriptors);
     return lis;
   }
 
@@ -1103,11 +1074,10 @@ public class DAGAppMaster extends AbstractService {
     return chh;
   }
 
-  protected ContainerLauncherRouter createContainerLauncherRouter(UserPayload defaultPayload,
-                                                                  List<NamedEntityDescriptor> containerLauncherDescriptors,
+  protected ContainerLauncherRouter createContainerLauncherRouter(List<NamedEntityDescriptor> containerLauncherDescriptors,
                                                                   boolean isLocal) throws
       UnknownHostException {
-    return new ContainerLauncherRouter(defaultPayload, context, taskAttemptListener, workingDirectory,
+    return new ContainerLauncherRouter(context, taskAttemptListener, workingDirectory,
         containerLauncherDescriptors, isLocal);
   }
 
@@ -2401,41 +2371,106 @@ public class DAGAppMaster extends AbstractService {
         TezConfiguration.TEZ_AM_WEBSERVICE_ENABLE_DEFAULT);
   }
 
-  private static List<NamedEntityDescriptor> parsePlugin(
-      BiMap<String, Integer> pluginMap, List<TezNamedEntityDescriptorProto> namedEntityDescriptorProtos,
-      boolean tezYarnEnabled, boolean uberEnabled) {
 
-    int index = 0;
+  @VisibleForTesting
+  static void parseAllPlugins(
+      List<NamedEntityDescriptor> taskSchedulerDescriptors, BiMap<String, Integer> taskSchedulerPluginMap,
+      List<NamedEntityDescriptor> containerLauncherDescriptors, BiMap<String, Integer> containerLauncherPluginMap,
+      List<NamedEntityDescriptor> taskCommDescriptors, BiMap<String, Integer> taskCommPluginMap,
+      AMPluginDescriptorProto amPluginDescriptorProto, boolean isLocal, UserPayload defaultPayload) {
+
+    boolean tezYarnEnabled;
+    boolean uberEnabled;
+    if (!isLocal) {
+      if (amPluginDescriptorProto == null) {
+        tezYarnEnabled = true;
+        uberEnabled = false;
+      } else {
+        tezYarnEnabled = amPluginDescriptorProto.getContainersEnabled();
+        uberEnabled = amPluginDescriptorProto.getUberEnabled();
+      }
+    } else {
+      tezYarnEnabled = false;
+      uberEnabled = true;
+    }
+
+    parsePlugin(taskSchedulerDescriptors, taskSchedulerPluginMap,
+        (amPluginDescriptorProto == null || amPluginDescriptorProto.getTaskSchedulersCount() == 0 ?
+            null :
+            amPluginDescriptorProto.getTaskSchedulersList()),
+        tezYarnEnabled, uberEnabled, defaultPayload);
+    processSchedulerDescriptors(taskSchedulerDescriptors, isLocal, defaultPayload, taskSchedulerPluginMap);
 
-    List<NamedEntityDescriptor> resultList = new LinkedList<>();
+    parsePlugin(containerLauncherDescriptors, containerLauncherPluginMap,
+        (amPluginDescriptorProto == null ||
+            amPluginDescriptorProto.getContainerLaunchersCount() == 0 ? null :
+            amPluginDescriptorProto.getContainerLaunchersList()),
+        tezYarnEnabled, uberEnabled, defaultPayload);
+
+    parsePlugin(taskCommDescriptors, taskCommPluginMap,
+        (amPluginDescriptorProto == null ||
+            amPluginDescriptorProto.getTaskCommunicatorsCount() == 0 ? null :
+            amPluginDescriptorProto.getTaskCommunicatorsList()),
+        tezYarnEnabled, uberEnabled, defaultPayload);
+  }
+
+
+  @VisibleForTesting
+  static void parsePlugin(List<NamedEntityDescriptor> resultList,
+      BiMap<String, Integer> pluginMap, List<TezNamedEntityDescriptorProto> namedEntityDescriptorProtos,
+      boolean tezYarnEnabled, boolean uberEnabled, UserPayload defaultPayload) {
 
     if (tezYarnEnabled) {
       // Default classnames will be populated by individual components
       NamedEntityDescriptor r = new NamedEntityDescriptor(
-          TezConstants.getTezYarnServicePluginName(), null);
-      resultList.add(r);
-      pluginMap.put(TezConstants.getTezYarnServicePluginName(), index);
-      index++;
+          TezConstants.getTezYarnServicePluginName(), null).setUserPayload(defaultPayload);
+      addDescriptor(resultList, pluginMap, r);
     }
 
     if (uberEnabled) {
       // Default classnames will be populated by individual components
       NamedEntityDescriptor r = new NamedEntityDescriptor(
-          TezConstants.getTezUberServicePluginName(), null);
-      resultList.add(r);
-      pluginMap.put(TezConstants.getTezUberServicePluginName(), index);
-      index++;
+          TezConstants.getTezUberServicePluginName(), null).setUserPayload(defaultPayload);
+      addDescriptor(resultList, pluginMap, r);
     }
 
     if (namedEntityDescriptorProtos != null) {
       for (TezNamedEntityDescriptorProto namedEntityDescriptorProto : namedEntityDescriptorProtos) {
-        resultList.add(DagTypeConverters
-            .convertNamedDescriptorFromProto(namedEntityDescriptorProto));
-        pluginMap.put(resultList.get(index).getEntityName(), index);
-        index++;
+        NamedEntityDescriptor namedEntityDescriptor = DagTypeConverters
+            .convertNamedDescriptorFromProto(namedEntityDescriptorProto);
+        addDescriptor(resultList, pluginMap, namedEntityDescriptor);
+      }
+    }
+  }
+
+  @VisibleForTesting
+  static void addDescriptor(List<NamedEntityDescriptor> list, BiMap<String, Integer> pluginMap,
+                            NamedEntityDescriptor namedEntityDescriptor) {
+    list.add(namedEntityDescriptor);
+    pluginMap.put(list.get(list.size() - 1).getEntityName(), list.size() - 1);
+  }
+
+  @VisibleForTesting
+  static void processSchedulerDescriptors(List<NamedEntityDescriptor> descriptors, boolean isLocal,
+                                          UserPayload defaultPayload,
+                                          BiMap<String, Integer> schedulerPluginMap) {
+    if (isLocal) {
+      Preconditions.checkState(descriptors.size() == 1 &&
+          descriptors.get(0).getEntityName().equals(TezConstants.getTezUberServicePluginName()));
+    } else {
+      boolean foundYarn = false;
+      for (int i = 0; i < descriptors.size(); i++) {
+        if (descriptors.get(i).getEntityName().equals(TezConstants.getTezYarnServicePluginName())) {
+          foundYarn = true;
+        }
+      }
+      if (!foundYarn) {
+        NamedEntityDescriptor yarnDescriptor =
+            new NamedEntityDescriptor(TezConstants.getTezYarnServicePluginName(), null)
+                .setUserPayload(defaultPayload);
+        addDescriptor(descriptors, schedulerPluginMap, yarnDescriptor);
       }
     }
-    return resultList;
   }
 
   String buildPluginComponentLog(List<NamedEntityDescriptor> namedEntityDescriptors, BiMap<String, Integer> map,

http://git-wip-us.apache.org/repos/asf/tez/blob/267fe737/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 941e583..7b97738 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
@@ -27,7 +27,7 @@ import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 
 import com.google.common.annotations.VisibleForTesting;
-import com.google.common.collect.Lists;
+import com.google.common.base.Preconditions;
 import org.apache.commons.collections4.ListUtils;
 import org.apache.tez.dag.api.NamedEntityDescriptor;
 import org.apache.tez.dag.api.TezConstants;
@@ -102,35 +102,19 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
 
   public TaskAttemptListenerImpTezDag(AppContext context,
                                       TaskHeartbeatHandler thh, ContainerHeartbeatHandler chh,
-                                      List<NamedEntityDescriptor> taskCommunicatorDescriptors,
-                                      UserPayload defaultUserPayload,
-                                      boolean isPureLocalMode) {
+                                      List<NamedEntityDescriptor> taskCommunicatorDescriptors) {
     super(TaskAttemptListenerImpTezDag.class.getName());
     this.context = context;
     this.taskHeartbeatHandler = thh;
     this.containerHeartbeatHandler = chh;
-    if (taskCommunicatorDescriptors == null || taskCommunicatorDescriptors.isEmpty()) {
-      if (isPureLocalMode) {
-        taskCommunicatorDescriptors = Lists.newArrayList(new NamedEntityDescriptor(
-            TezConstants.getTezUberServicePluginName(), null).setUserPayload(defaultUserPayload));
-      } else {
-        taskCommunicatorDescriptors = Lists.newArrayList(new NamedEntityDescriptor(
-            TezConstants.getTezYarnServicePluginName(), null).setUserPayload(defaultUserPayload));
-      }
-    }
+    Preconditions.checkArgument(
+        taskCommunicatorDescriptors != null && !taskCommunicatorDescriptors.isEmpty(),
+        "TaskCommunicators must be specified");
     this.taskCommunicators = new TaskCommunicator[taskCommunicatorDescriptors.size()];
     this.taskCommunicatorContexts = new TaskCommunicatorContext[taskCommunicatorDescriptors.size()];
     this.taskCommunicatorServiceWrappers = new ServicePluginLifecycleAbstractService[taskCommunicatorDescriptors.size()];
     for (int i = 0 ; i < taskCommunicatorDescriptors.size() ; i++) {
-      UserPayload userPayload;
-      if (taskCommunicatorDescriptors.get(i).getEntityName()
-          .equals(TezConstants.getTezYarnServicePluginName()) ||
-          taskCommunicatorDescriptors.get(i).getEntityName()
-              .equals(TezConstants.getTezUberServicePluginName())) {
-        userPayload = defaultUserPayload;
-      } else {
-        userPayload = taskCommunicatorDescriptors.get(i).getUserPayload();
-      }
+      UserPayload userPayload = taskCommunicatorDescriptors.get(i).getUserPayload();
       taskCommunicatorContexts[i] = new TaskCommunicatorContextImpl(context, this, userPayload, i);
       taskCommunicators[i] = createTaskCommunicator(taskCommunicatorDescriptors.get(i), i);
       taskCommunicatorServiceWrappers[i] = new ServicePluginLifecycleAbstractService(taskCommunicators[i]);
@@ -154,36 +138,54 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
     }
   }
 
-  private TaskCommunicator createTaskCommunicator(NamedEntityDescriptor taskCommDescriptor, int taskCommIndex) {
+  @VisibleForTesting
+  TaskCommunicator createTaskCommunicator(NamedEntityDescriptor taskCommDescriptor,
+                                          int taskCommIndex) {
     if (taskCommDescriptor.getEntityName().equals(TezConstants.getTezYarnServicePluginName())) {
-      LOG.info("Using Default Task Communicator");
-      return createTezTaskCommunicator(taskCommunicatorContexts[taskCommIndex]);
-    } else if (taskCommDescriptor.getEntityName().equals(TezConstants.getTezUberServicePluginName())) {
-      LOG.info("Using Default Local Task Communicator");
-      return new TezLocalTaskCommunicatorImpl(taskCommunicatorContexts[taskCommIndex]);
+      return createDefaultTaskCommunicator(taskCommunicatorContexts[taskCommIndex]);
+    } else if (taskCommDescriptor.getEntityName()
+        .equals(TezConstants.getTezUberServicePluginName())) {
+      return createUberTaskCommunicator(taskCommunicatorContexts[taskCommIndex]);
     } else {
-      LOG.info("Using TaskCommunicator {}:{} " + taskCommDescriptor.getEntityName(), taskCommDescriptor.getClassName());
-      Class<? extends TaskCommunicator> taskCommClazz = (Class<? extends TaskCommunicator>) ReflectionUtils
-          .getClazz(taskCommDescriptor.getClassName());
-      try {
-        Constructor<? extends TaskCommunicator> ctor = taskCommClazz.getConstructor(TaskCommunicatorContext.class);
-        ctor.setAccessible(true);
-        return ctor.newInstance(taskCommunicatorContexts[taskCommIndex]);
-      } 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);
-      }
+      return createCustomTaskCommunicator(taskCommunicatorContexts[taskCommIndex],
+          taskCommDescriptor);
     }
   }
 
   @VisibleForTesting
-  protected TezTaskCommunicatorImpl createTezTaskCommunicator(TaskCommunicatorContext context) {
-    return new TezTaskCommunicatorImpl(context);
+  TaskCommunicator createDefaultTaskCommunicator(TaskCommunicatorContext taskCommunicatorContext) {
+    LOG.info("Using Default Task Communicator");
+    return new TezTaskCommunicatorImpl(taskCommunicatorContext);
+  }
+
+  @VisibleForTesting
+  TaskCommunicator createUberTaskCommunicator(TaskCommunicatorContext taskCommunicatorContext) {
+    LOG.info("Using Default Local Task Communicator");
+    return new TezLocalTaskCommunicatorImpl(taskCommunicatorContext);
+  }
+
+  @VisibleForTesting
+  TaskCommunicator createCustomTaskCommunicator(TaskCommunicatorContext taskCommunicatorContext,
+                                                NamedEntityDescriptor taskCommDescriptor) {
+    LOG.info("Using TaskCommunicator {}:{} " + taskCommDescriptor.getEntityName(),
+        taskCommDescriptor.getClassName());
+    Class<? extends TaskCommunicator> taskCommClazz =
+        (Class<? extends TaskCommunicator>) ReflectionUtils
+            .getClazz(taskCommDescriptor.getClassName());
+    try {
+      Constructor<? extends TaskCommunicator> ctor =
+          taskCommClazz.getConstructor(TaskCommunicatorContext.class);
+      ctor.setAccessible(true);
+      return ctor.newInstance(taskCommunicatorContext);
+    } 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);
+    }
   }
 
   public TaskHeartbeatResponse heartbeat(TaskHeartbeatRequest request)

http://git-wip-us.apache.org/repos/asf/tez/blob/267fe737/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 2e8f218..3cc439f 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
@@ -233,9 +233,12 @@ 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;
+  @VisibleForTesting
+  final int taskSchedulerIdentifier;
+  @VisibleForTesting
+  final int containerLauncherIdentifier;
+  @VisibleForTesting
+  final int taskCommunicatorIdentifier;
 
   //fields initialized in init
 

http://git-wip-us.apache.org/repos/asf/tez/blob/267fe737/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 2d56bfe..57b4aee 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
@@ -20,7 +20,7 @@ import java.net.UnknownHostException;
 import java.util.List;
 
 import com.google.common.annotations.VisibleForTesting;
-import com.google.common.collect.Lists;
+import com.google.common.base.Preconditions;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.service.AbstractService;
 import org.apache.hadoop.yarn.event.EventHandler;
@@ -48,8 +48,10 @@ public class ContainerLauncherRouter extends AbstractService
 
   static final Logger LOG = LoggerFactory.getLogger(ContainerLauncherImpl.class);
 
-  private final ContainerLauncher containerLaunchers[];
-  private final ContainerLauncherContext containerLauncherContexts[];
+  @VisibleForTesting
+  final ContainerLauncher containerLaunchers[];
+  @VisibleForTesting
+  final ContainerLauncherContext containerLauncherContexts[];
   protected final ServicePluginLifecycleAbstractService[] containerLauncherServiceWrappers;
   private final AppContext appContext;
 
@@ -64,7 +66,7 @@ public class ContainerLauncherRouter extends AbstractService
   }
 
   // Accepting conf to setup final parameters, if required.
-  public ContainerLauncherRouter(UserPayload defaultUserPayload, AppContext context,
+  public ContainerLauncherRouter(AppContext context,
                                  TaskAttemptListener taskAttemptListener,
                                  String workingDirectory,
                                  List<NamedEntityDescriptor> containerLauncherDescriptors,
@@ -72,79 +74,91 @@ public class ContainerLauncherRouter extends AbstractService
     super(ContainerLauncherRouter.class.getName());
 
     this.appContext = context;
-    if (containerLauncherDescriptors == null || containerLauncherDescriptors.isEmpty()) {
-      if (isPureLocalMode) {
-        containerLauncherDescriptors = Lists.newArrayList(new NamedEntityDescriptor(
-            TezConstants.getTezUberServicePluginName(), null).setUserPayload(defaultUserPayload));
-      } else {
-        containerLauncherDescriptors = Lists.newArrayList(new NamedEntityDescriptor(
-            TezConstants.getTezYarnServicePluginName(), null).setUserPayload(defaultUserPayload));
-      }
-    }
+    Preconditions.checkArgument(
+        containerLauncherDescriptors != null && !containerLauncherDescriptors.isEmpty(),
+        "ContainerLauncherDescriptors must be specified");
     containerLauncherContexts = new ContainerLauncherContext[containerLauncherDescriptors.size()];
     containerLaunchers = new ContainerLauncher[containerLauncherDescriptors.size()];
     containerLauncherServiceWrappers = new ServicePluginLifecycleAbstractService[containerLauncherDescriptors.size()];
 
 
     for (int i = 0; i < containerLauncherDescriptors.size(); i++) {
-      UserPayload userPayload;
-      if (containerLauncherDescriptors.get(i).getEntityName()
-          .equals(TezConstants.getTezYarnServicePluginName()) ||
-          containerLauncherDescriptors.get(i).getEntityName()
-              .equals(TezConstants.getTezUberServicePluginName())) {
-        userPayload = defaultUserPayload;
-      } else {
-        userPayload = containerLauncherDescriptors.get(i).getUserPayload();
-      }
+      UserPayload userPayload = containerLauncherDescriptors.get(i).getUserPayload();
       ContainerLauncherContext containerLauncherContext =
           new ContainerLauncherContextImpl(context, taskAttemptListener, userPayload);
       containerLauncherContexts[i] = containerLauncherContext;
       containerLaunchers[i] = createContainerLauncher(containerLauncherDescriptors.get(i), context,
-          containerLauncherContext, taskAttemptListener, workingDirectory, isPureLocalMode);
+          containerLauncherContext, taskAttemptListener, workingDirectory, i, isPureLocalMode);
       containerLauncherServiceWrappers[i] = new ServicePluginLifecycleAbstractService(containerLaunchers[i]);
     }
   }
 
-  private ContainerLauncher createContainerLauncher(NamedEntityDescriptor containerLauncherDescriptor,
-                                                    AppContext context,
-                                                    ContainerLauncherContext containerLauncherContext,
-                                                    TaskAttemptListener taskAttemptListener,
-                                                    String workingDirectory,
-                                                    boolean isPureLocalMode) throws
+  @VisibleForTesting
+  ContainerLauncher createContainerLauncher(
+      NamedEntityDescriptor containerLauncherDescriptor,
+      AppContext context,
+      ContainerLauncherContext containerLauncherContext,
+      TaskAttemptListener taskAttemptListener,
+      String workingDirectory,
+      int containerLauncherIndex,
+      boolean isPureLocalMode) throws
       UnknownHostException {
     if (containerLauncherDescriptor.getEntityName().equals(
         TezConstants.getTezYarnServicePluginName())) {
-      LOG.info("Creating DefaultContainerLauncher");
-      return new ContainerLauncherImpl(containerLauncherContext);
+      return createYarnContainerLauncher(containerLauncherContext);
     } else if (containerLauncherDescriptor.getEntityName()
         .equals(TezConstants.getTezUberServicePluginName())) {
-      LOG.info("Creating LocalContainerLauncher");
-      // TODO Post TEZ-2003. LocalContainerLauncher is special cased, since it makes use of
-      // extensive internals which are only available at runtime. Will likely require
-      // some kind of runtime binding of parameters in the payload to work correctly.
-      return
-          new LocalContainerLauncher(containerLauncherContext, context, taskAttemptListener, workingDirectory, isPureLocalMode);
+      return createUberContainerLauncher(containerLauncherContext, context, taskAttemptListener,
+          workingDirectory, isPureLocalMode);
     } else {
-      LOG.info("Creating container launcher {}:{} ", containerLauncherDescriptor.getEntityName(), containerLauncherDescriptor.getClassName());
-      Class<? extends ContainerLauncher> containerLauncherClazz =
-          (Class<? extends ContainerLauncher>) ReflectionUtils.getClazz(
-              containerLauncherDescriptor.getClassName());
-      try {
-        Constructor<? extends ContainerLauncher> ctor = containerLauncherClazz
-            .getConstructor(ContainerLauncherContext.class);
-        ctor.setAccessible(true);
-        return ctor.newInstance(containerLauncherContext);
-      } 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);
-      }
+      return createCustomContainerLauncher(containerLauncherContext, containerLauncherDescriptor);
+    }
+  }
+
+  @VisibleForTesting
+  ContainerLauncher createYarnContainerLauncher(ContainerLauncherContext containerLauncherContext) {
+    LOG.info("Creating DefaultContainerLauncher");
+    return new ContainerLauncherImpl(containerLauncherContext);
+  }
+
+  @VisibleForTesting
+  ContainerLauncher createUberContainerLauncher(ContainerLauncherContext containerLauncherContext,
+                                                AppContext context,
+                                                TaskAttemptListener taskAttemptListener,
+                                                String workingDirectory,
+                                                boolean isPureLocalMode) throws
+      UnknownHostException {
+    LOG.info("Creating LocalContainerLauncher");
+    // TODO Post TEZ-2003. LocalContainerLauncher is special cased, since it makes use of
+    // extensive internals which are only available at runtime. Will likely require
+    // some kind of runtime binding of parameters in the payload to work correctly.
+    return
+        new LocalContainerLauncher(containerLauncherContext, context, taskAttemptListener,
+            workingDirectory, isPureLocalMode);
+  }
+
+  @VisibleForTesting
+  ContainerLauncher createCustomContainerLauncher(ContainerLauncherContext containerLauncherContext,
+                                                  NamedEntityDescriptor containerLauncherDescriptor) {
+    LOG.info("Creating container launcher {}:{} ", containerLauncherDescriptor.getEntityName(),
+        containerLauncherDescriptor.getClassName());
+    Class<? extends ContainerLauncher> containerLauncherClazz =
+        (Class<? extends ContainerLauncher>) ReflectionUtils.getClazz(
+            containerLauncherDescriptor.getClassName());
+    try {
+      Constructor<? extends ContainerLauncher> ctor = containerLauncherClazz
+          .getConstructor(ContainerLauncherContext.class);
+      ctor.setAccessible(true);
+      return ctor.newInstance(containerLauncherContext);
+    } 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

http://git-wip-us.apache.org/repos/asf/tez/blob/267fe737/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 c86f638..7c36232 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
@@ -22,7 +22,6 @@ import java.lang.reflect.Constructor;
 import java.lang.reflect.InvocationTargetException;
 import java.net.InetSocketAddress;
 import java.nio.ByteBuffer;
-import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
 import java.util.concurrent.BlockingQueue;
@@ -34,10 +33,8 @@ import java.util.concurrent.atomic.AtomicBoolean;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.util.concurrent.ThreadFactoryBuilder;
-import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.tez.dag.api.NamedEntityDescriptor;
 import org.apache.tez.dag.api.TezConstants;
-import org.apache.tez.dag.api.UserPayload;
 import org.apache.tez.dag.app.ServicePluginLifecycleAbstractService;
 import org.apache.tez.serviceplugins.api.TaskScheduler;
 import org.apache.tez.serviceplugins.api.TaskSchedulerContext;
@@ -126,9 +123,8 @@ public class TaskSchedulerEventHandler extends AbstractService implements
   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
+  // Splitting registration and heartbeats is not straight-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;
@@ -153,9 +149,10 @@ public class TaskSchedulerEventHandler extends AbstractService implements
   public TaskSchedulerEventHandler(AppContext appContext,
       DAGClientServer clientService, EventHandler eventHandler, 
       ContainerSignatureMatcher containerSignatureMatcher, WebUIService webUI,
-      List<NamedEntityDescriptor> schedulerDescriptors, UserPayload defaultPayload,
-                                   boolean isPureLocalMode) {
+      List<NamedEntityDescriptor> schedulerDescriptors, boolean isPureLocalMode) {
     super(TaskSchedulerEventHandler.class.getName());
+    Preconditions.checkArgument(schedulerDescriptors != null && !schedulerDescriptors.isEmpty(),
+        "TaskSchedulerDescriptors must be specified");
     this.appContext = appContext;
     this.eventHandler = eventHandler;
     this.clientService = clientService;
@@ -168,50 +165,8 @@ public class TaskSchedulerEventHandler extends AbstractService implements
       this.webUI.setHistoryUrl(this.historyUrl);
     }
 
-    // Override everything for pure local mode
-    if (isPureLocalMode) {
-      this.taskSchedulerDescriptors = new NamedEntityDescriptor[]{
-          new NamedEntityDescriptor(TezConstants.getTezUberServicePluginName(), null)
-              .setUserPayload(defaultPayload)};
-      this.yarnTaskSchedulerIndex = -1;
-    } else {
-      if (schedulerDescriptors == null || schedulerDescriptors.isEmpty()) {
-        this.taskSchedulerDescriptors = new NamedEntityDescriptor[]{
-            new NamedEntityDescriptor(TezConstants.getTezYarnServicePluginName(), null)
-                .setUserPayload(defaultPayload)};
-        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;
-
-        List<NamedEntityDescriptor> schedulerDescriptorList = new LinkedList<>();
-        for (int i = 0 ; i < schedulerDescriptors.size() ; i++) {
-          if (schedulerDescriptors.get(i).getEntityName().equals(
-              TezConstants.getTezYarnServicePluginName())) {
-            schedulerDescriptorList.add(
-                new NamedEntityDescriptor(schedulerDescriptors.get(i).getEntityName(), null)
-                    .setUserPayload(
-                        defaultPayload));
-            foundYarnTaskSchedulerIndex = i;
-          } else if (schedulerDescriptors.get(i).getEntityName().equals(
-              TezConstants.getTezUberServicePluginName())) {
-            schedulerDescriptorList.add(
-                new NamedEntityDescriptor(schedulerDescriptors.get(i).getEntityName(), null)
-                    .setUserPayload(
-                        defaultPayload));
-          } else {
-            schedulerDescriptorList.add(schedulerDescriptors.get(i));
-          }
-        }
-        if (foundYarnTaskSchedulerIndex == -1) {
-          schedulerDescriptorList.add(new NamedEntityDescriptor(TezConstants.getTezYarnServicePluginName(), null).setUserPayload(
-              defaultPayload));
-          foundYarnTaskSchedulerIndex = schedulerDescriptorList.size() -1;
-        }
-        this.taskSchedulerDescriptors = schedulerDescriptorList.toArray(new NamedEntityDescriptor[schedulerDescriptorList.size()]);
-        this.yarnTaskSchedulerIndex = foundYarnTaskSchedulerIndex;
-      }
-    }
+    this.taskSchedulerDescriptors = schedulerDescriptors.toArray(new NamedEntityDescriptor[schedulerDescriptors.size()]);
+
     taskSchedulers = new TaskScheduler[this.taskSchedulerDescriptors.length];
     taskSchedulerServiceWrappers = new ServicePluginLifecycleAbstractService[this.taskSchedulerDescriptors.length];
   }
@@ -239,7 +194,8 @@ public class TaskSchedulerEventHandler extends AbstractService implements
 
   private ExecutorService createAppCallbackExecutorService() {
     return Executors.newSingleThreadExecutor(
-        new ThreadFactoryBuilder().setNameFormat("TaskSchedulerAppCallbackExecutor #%d").setDaemon(true)
+        new ThreadFactoryBuilder().setNameFormat("TaskSchedulerAppCallbackExecutor #%d")
+            .setDaemon(true)
             .build());
   }
 
@@ -428,7 +384,8 @@ public class TaskSchedulerEventHandler extends AbstractService implements
         event);
   }
 
-  private TaskScheduler createTaskScheduler(String host, int port, String trackingUrl,
+  @VisibleForTesting
+  TaskScheduler createTaskScheduler(String host, int port, String trackingUrl,
                                                    AppContext appContext,
                                                    NamedEntityDescriptor taskSchedulerDescriptor,
                                                    long customAppIdIdentifier,
@@ -436,32 +393,57 @@ public class TaskSchedulerEventHandler extends AbstractService implements
     TaskSchedulerContext rawContext =
         new TaskSchedulerContextImpl(this, appContext, schedulerId, trackingUrl,
             customAppIdIdentifier, host, port, taskSchedulerDescriptor.getUserPayload());
-    TaskSchedulerContext wrappedContext = new TaskSchedulerContextImplWrapper(rawContext, appCallbackExecutor);
+    TaskSchedulerContext wrappedContext = wrapTaskSchedulerContext(rawContext);
     String schedulerName = taskSchedulerDescriptor.getEntityName();
     if (schedulerName.equals(TezConstants.getTezYarnServicePluginName())) {
-      LOG.info("Creating TaskScheduler: YarnTaskSchedulerService");
-      return new YarnTaskSchedulerService(wrappedContext);
+      return createYarnTaskScheduler(wrappedContext, schedulerId);
     } else if (schedulerName.equals(TezConstants.getTezUberServicePluginName())) {
-      LOG.info("Creating TaskScheduler: Local TaskScheduler");
-      return new LocalTaskSchedulerService(wrappedContext);
+      return createUberTaskScheduler(wrappedContext, schedulerId);
     } else {
-      LOG.info("Creating custom TaskScheduler {}:{}", taskSchedulerDescriptor.getEntityName(), taskSchedulerDescriptor.getClassName());
-      Class<? extends TaskScheduler> taskSchedulerClazz =
-          (Class<? extends TaskScheduler>) ReflectionUtils.getClazz(taskSchedulerDescriptor.getClassName());
-      try {
-        Constructor<? extends TaskScheduler> ctor = taskSchedulerClazz
-            .getConstructor(TaskSchedulerContext.class);
-        ctor.setAccessible(true);
-        return ctor.newInstance(wrappedContext);
-      } 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);
-      }
+      return createCustomTaskScheduler(wrappedContext, taskSchedulerDescriptor, schedulerId);
+    }
+  }
+
+  @VisibleForTesting
+  TaskSchedulerContext wrapTaskSchedulerContext(TaskSchedulerContext rawContext) {
+    return new TaskSchedulerContextImplWrapper(rawContext, appCallbackExecutor);
+  }
+
+  @VisibleForTesting
+  TaskScheduler createYarnTaskScheduler(TaskSchedulerContext taskSchedulerContext,
+                                        int schedulerId) {
+    LOG.info("Creating TaskScheduler: YarnTaskSchedulerService");
+    return new YarnTaskSchedulerService(taskSchedulerContext);
+  }
+
+  @VisibleForTesting
+  TaskScheduler createUberTaskScheduler(TaskSchedulerContext taskSchedulerContext,
+                                        int schedulerId) {
+    LOG.info("Creating TaskScheduler: Local TaskScheduler");
+    return new LocalTaskSchedulerService(taskSchedulerContext);
+  }
+
+  TaskScheduler createCustomTaskScheduler(TaskSchedulerContext taskSchedulerContext,
+                                          NamedEntityDescriptor taskSchedulerDescriptor,
+                                          int schedulerId) {
+    LOG.info("Creating custom TaskScheduler {}:{}", taskSchedulerDescriptor.getEntityName(),
+        taskSchedulerDescriptor.getClassName());
+    Class<? extends TaskScheduler> taskSchedulerClazz =
+        (Class<? extends TaskScheduler>) ReflectionUtils
+            .getClazz(taskSchedulerDescriptor.getClassName());
+    try {
+      Constructor<? extends TaskScheduler> ctor = taskSchedulerClazz
+          .getConstructor(TaskSchedulerContext.class);
+      ctor.setAccessible(true);
+      return ctor.newInstance(taskSchedulerContext);
+    } 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);
     }
   }
 
@@ -797,9 +779,4 @@ public class TaskSchedulerEventHandler extends AbstractService implements
     return historyUrl;
   }
 
-  @VisibleForTesting
-  @InterfaceAudience.Private
-  ExecutorService getContextExecutorService() {
-    return appCallbackExecutor;
-  }
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/267fe737/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 0723dbc..2e6e568 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
@@ -511,8 +511,7 @@ public class MockDAGAppMaster extends DAGAppMaster {
 
   // use mock container launcher for tests
   @Override
-  protected ContainerLauncherRouter createContainerLauncherRouter(final UserPayload defaultUserPayload,
-                                                                  List<NamedEntityDescriptor> containerLauncherDescirptors,
+  protected ContainerLauncherRouter createContainerLauncherRouter(List<NamedEntityDescriptor> containerLauncherDescirptors,
                                                                   boolean isLocal)
       throws UnknownHostException {
     return new ContainerLauncherRouter(containerLauncher, getContext());

http://git-wip-us.apache.org/repos/asf/tez/blob/267fe737/tez-dag/src/test/java/org/apache/tez/dag/app/TestDAGAppMaster.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/TestDAGAppMaster.java b/tez-dag/src/test/java/org/apache/tez/dag/app/TestDAGAppMaster.java
new file mode 100644
index 0000000..fa5d87c
--- /dev/null
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/TestDAGAppMaster.java
@@ -0,0 +1,300 @@
+/*
+ * 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.assertNull;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.LinkedList;
+import java.util.List;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.BiMap;
+import com.google.common.collect.HashBiMap;
+import com.google.common.collect.Lists;
+import com.google.protobuf.ByteString;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.tez.common.TezUtils;
+import org.apache.tez.dag.api.NamedEntityDescriptor;
+import org.apache.tez.dag.api.TezConstants;
+import org.apache.tez.dag.api.UserPayload;
+import org.apache.tez.dag.api.records.DAGProtos;
+import org.apache.tez.dag.api.records.DAGProtos.AMPluginDescriptorProto;
+import org.apache.tez.dag.api.records.DAGProtos.TezNamedEntityDescriptorProto;
+import org.apache.tez.dag.api.records.DAGProtos.TezUserPayloadProto;
+import org.junit.Test;
+
+public class TestDAGAppMaster {
+
+  private static final String TEST_KEY = "TEST_KEY";
+  private static final String TEST_VAL = "TEST_VAL";
+  private static final String TS_NAME = "TS";
+  private static final String CL_NAME = "CL";
+  private static final String TC_NAME = "TC";
+  private static final String CLASS_SUFFIX = "_CLASS";
+
+  @Test(timeout = 5000)
+  public void testPluginParsing() throws IOException {
+    BiMap<String, Integer> pluginMap = HashBiMap.create();
+    Configuration conf = new Configuration(false);
+    conf.set("testkey", "testval");
+    UserPayload defaultPayload = TezUtils.createUserPayloadFromConf(conf);
+
+    List<TezNamedEntityDescriptorProto> entityDescriptors = new LinkedList<>();
+    List<NamedEntityDescriptor> entities;
+
+    // Test empty descriptor list, yarn enabled
+    pluginMap.clear();
+    entities = new LinkedList<>();
+    DAGAppMaster.parsePlugin(entities, pluginMap, null, true, false, defaultPayload);
+    assertEquals(1, pluginMap.size());
+    assertEquals(1, entities.size());
+    assertTrue(pluginMap.containsKey(TezConstants.getTezYarnServicePluginName()));
+    assertTrue(0 == pluginMap.get(TezConstants.getTezYarnServicePluginName()));
+    assertEquals("testval",
+        TezUtils.createConfFromUserPayload(entities.get(0).getUserPayload()).get("testkey"));
+
+    // Test empty descriptor list, uber enabled
+    pluginMap.clear();
+    entities = new LinkedList<>();
+    DAGAppMaster.parsePlugin(entities, pluginMap, null, false, true, defaultPayload);
+    assertEquals(1, pluginMap.size());
+    assertEquals(1, entities.size());
+    assertTrue(pluginMap.containsKey(TezConstants.getTezUberServicePluginName()));
+    assertTrue(0 == pluginMap.get(TezConstants.getTezUberServicePluginName()));
+    assertEquals("testval",
+        TezUtils.createConfFromUserPayload(entities.get(0).getUserPayload()).get("testkey"));
+
+    // Test empty descriptor list, yarn enabled, uber enabled
+    pluginMap.clear();
+    entities = new LinkedList<>();
+    DAGAppMaster.parsePlugin(entities, pluginMap, null, true, true, defaultPayload);
+    assertEquals(2, pluginMap.size());
+    assertEquals(2, entities.size());
+    assertTrue(pluginMap.containsKey(TezConstants.getTezYarnServicePluginName()));
+    assertTrue(0 == pluginMap.get(TezConstants.getTezYarnServicePluginName()));
+    assertTrue(pluginMap.containsKey(TezConstants.getTezUberServicePluginName()));
+    assertTrue(1 == pluginMap.get(TezConstants.getTezUberServicePluginName()));
+
+
+    String pluginName = "d1";
+    ByteBuffer bb = ByteBuffer.allocate(4);
+    bb.putInt(0, 3);
+    TezNamedEntityDescriptorProto d1 =
+        TezNamedEntityDescriptorProto.newBuilder().setName(pluginName).setEntityDescriptor(
+            DAGProtos.TezEntityDescriptorProto.newBuilder().setClassName("d1Class")
+                .setTezUserPayload(
+                    TezUserPayloadProto.newBuilder()
+                        .setUserPayload(ByteString.copyFrom(bb)))).build();
+    entityDescriptors.add(d1);
+
+    // Test descriptor, no yarn, no uber
+    pluginMap.clear();
+    entities = new LinkedList<>();
+    DAGAppMaster.parsePlugin(entities, pluginMap, entityDescriptors, false, false, defaultPayload);
+    assertEquals(1, pluginMap.size());
+    assertEquals(1, entities.size());
+    assertTrue(pluginMap.containsKey(pluginName));
+    assertTrue(0 == pluginMap.get(pluginName));
+
+    // Test descriptor, yarn and uber
+    pluginMap.clear();
+    entities = new LinkedList<>();
+    DAGAppMaster.parsePlugin(entities, pluginMap, entityDescriptors, true, true, defaultPayload);
+    assertEquals(3, pluginMap.size());
+    assertEquals(3, entities.size());
+    assertTrue(pluginMap.containsKey(TezConstants.getTezYarnServicePluginName()));
+    assertTrue(0 == pluginMap.get(TezConstants.getTezYarnServicePluginName()));
+    assertTrue(pluginMap.containsKey(TezConstants.getTezUberServicePluginName()));
+    assertTrue(1 == pluginMap.get(TezConstants.getTezUberServicePluginName()));
+    assertTrue(pluginMap.containsKey(pluginName));
+    assertTrue(2 == pluginMap.get(pluginName));
+    entityDescriptors.clear();
+  }
+
+
+  @Test(timeout = 5000)
+  public void testParseAllPluginsNoneSpecified() throws IOException {
+    Configuration conf = new Configuration(false);
+    conf.set(TEST_KEY, TEST_VAL);
+    UserPayload defaultPayload = TezUtils.createUserPayloadFromConf(conf);
+
+    List<NamedEntityDescriptor> tsDescriptors;
+    BiMap<String, Integer> tsMap;
+    List<NamedEntityDescriptor> clDescriptors;
+    BiMap<String, Integer> clMap;
+    List<NamedEntityDescriptor> tcDescriptors;
+    BiMap<String, Integer> tcMap;
+
+
+    // No plugins. Non local
+    tsDescriptors = Lists.newLinkedList();
+    tsMap = HashBiMap.create();
+    clDescriptors = Lists.newLinkedList();
+    clMap = HashBiMap.create();
+    tcDescriptors = Lists.newLinkedList();
+    tcMap = HashBiMap.create();
+    DAGAppMaster.parseAllPlugins(tsDescriptors, tsMap, clDescriptors, clMap, tcDescriptors, tcMap,
+        null, false, defaultPayload);
+    verifyDescAndMap(tsDescriptors, tsMap, 1, true, TezConstants.getTezYarnServicePluginName());
+    verifyDescAndMap(clDescriptors, clMap, 1, true, TezConstants.getTezYarnServicePluginName());
+    verifyDescAndMap(tcDescriptors, tcMap, 1, true, TezConstants.getTezYarnServicePluginName());
+
+    // No plugins. Local
+    tsDescriptors = Lists.newLinkedList();
+    tsMap = HashBiMap.create();
+    clDescriptors = Lists.newLinkedList();
+    clMap = HashBiMap.create();
+    tcDescriptors = Lists.newLinkedList();
+    tcMap = HashBiMap.create();
+    DAGAppMaster.parseAllPlugins(tsDescriptors, tsMap, clDescriptors, clMap, tcDescriptors, tcMap,
+        null, true, defaultPayload);
+    verifyDescAndMap(tsDescriptors, tsMap, 1, true, TezConstants.getTezUberServicePluginName());
+    verifyDescAndMap(clDescriptors, clMap, 1, true, TezConstants.getTezUberServicePluginName());
+    verifyDescAndMap(tcDescriptors, tcMap, 1, true, TezConstants.getTezUberServicePluginName());
+  }
+
+  @Test(timeout = 5000)
+  public void testParseAllPluginsOnlyCustomSpecified() throws IOException {
+    Configuration conf = new Configuration(false);
+    conf.set(TEST_KEY, TEST_VAL);
+    UserPayload defaultPayload = TezUtils.createUserPayloadFromConf(conf);
+    TezUserPayloadProto payloadProto = TezUserPayloadProto.newBuilder()
+        .setUserPayload(ByteString.copyFrom(defaultPayload.getPayload())).build();
+
+    AMPluginDescriptorProto proto = createAmPluginDescriptor(false, false, true, payloadProto);
+
+    List<NamedEntityDescriptor> tsDescriptors;
+    BiMap<String, Integer> tsMap;
+    List<NamedEntityDescriptor> clDescriptors;
+    BiMap<String, Integer> clMap;
+    List<NamedEntityDescriptor> tcDescriptors;
+    BiMap<String, Integer> tcMap;
+
+
+    // Only plugin, Yarn.
+    tsDescriptors = Lists.newLinkedList();
+    tsMap = HashBiMap.create();
+    clDescriptors = Lists.newLinkedList();
+    clMap = HashBiMap.create();
+    tcDescriptors = Lists.newLinkedList();
+    tcMap = HashBiMap.create();
+    DAGAppMaster.parseAllPlugins(tsDescriptors, tsMap, clDescriptors, clMap, tcDescriptors, tcMap,
+        proto, false, defaultPayload);
+    verifyDescAndMap(tsDescriptors, tsMap, 2, true, TS_NAME,
+        TezConstants.getTezYarnServicePluginName());
+    verifyDescAndMap(clDescriptors, clMap, 1, true, CL_NAME);
+    verifyDescAndMap(tcDescriptors, tcMap, 1, true, TC_NAME);
+    assertEquals(TS_NAME + CLASS_SUFFIX, tsDescriptors.get(0).getClassName());
+    assertEquals(CL_NAME + CLASS_SUFFIX, clDescriptors.get(0).getClassName());
+    assertEquals(TC_NAME + CLASS_SUFFIX, tcDescriptors.get(0).getClassName());
+  }
+
+  @Test(timeout = 5000)
+  public void testParseAllPluginsCustomAndYarnSpecified() throws IOException {
+    Configuration conf = new Configuration(false);
+    conf.set(TEST_KEY, TEST_VAL);
+    UserPayload defaultPayload = TezUtils.createUserPayloadFromConf(conf);
+    TezUserPayloadProto payloadProto = TezUserPayloadProto.newBuilder()
+        .setUserPayload(ByteString.copyFrom(defaultPayload.getPayload())).build();
+
+    AMPluginDescriptorProto proto = createAmPluginDescriptor(true, false, true, payloadProto);
+
+    List<NamedEntityDescriptor> tsDescriptors;
+    BiMap<String, Integer> tsMap;
+    List<NamedEntityDescriptor> clDescriptors;
+    BiMap<String, Integer> clMap;
+    List<NamedEntityDescriptor> tcDescriptors;
+    BiMap<String, Integer> tcMap;
+
+
+    // Only plugin, Yarn.
+    tsDescriptors = Lists.newLinkedList();
+    tsMap = HashBiMap.create();
+    clDescriptors = Lists.newLinkedList();
+    clMap = HashBiMap.create();
+    tcDescriptors = Lists.newLinkedList();
+    tcMap = HashBiMap.create();
+    DAGAppMaster.parseAllPlugins(tsDescriptors, tsMap, clDescriptors, clMap, tcDescriptors, tcMap,
+        proto, false, defaultPayload);
+    verifyDescAndMap(tsDescriptors, tsMap, 2, true, TezConstants.getTezYarnServicePluginName(),
+        TS_NAME);
+    verifyDescAndMap(clDescriptors, clMap, 2, true, TezConstants.getTezYarnServicePluginName(),
+        CL_NAME);
+    verifyDescAndMap(tcDescriptors, tcMap, 2, true, TezConstants.getTezYarnServicePluginName(),
+        TC_NAME);
+    assertNull(tsDescriptors.get(0).getClassName());
+    assertNull(clDescriptors.get(0).getClassName());
+    assertNull(tcDescriptors.get(0).getClassName());
+    assertEquals(TS_NAME + CLASS_SUFFIX, tsDescriptors.get(1).getClassName());
+    assertEquals(CL_NAME + CLASS_SUFFIX, clDescriptors.get(1).getClassName());
+    assertEquals(TC_NAME + CLASS_SUFFIX, tcDescriptors.get(1).getClassName());
+  }
+
+  private void verifyDescAndMap(List<NamedEntityDescriptor> descriptors, BiMap<String, Integer> map,
+                                int numExpected, boolean verifyPayload,
+                                String... expectedNames) throws
+      IOException {
+    Preconditions.checkArgument(expectedNames.length == numExpected);
+    assertEquals(numExpected, descriptors.size());
+    assertEquals(numExpected, map.size());
+    for (int i = 0; i < numExpected; i++) {
+      assertEquals(expectedNames[i], descriptors.get(i).getEntityName());
+      if (verifyPayload) {
+        assertEquals(TEST_VAL,
+            TezUtils.createConfFromUserPayload(descriptors.get(0).getUserPayload()).get(TEST_KEY));
+      }
+      assertTrue(map.get(expectedNames[i]) == i);
+      assertTrue(map.inverse().get(i) == expectedNames[i]);
+    }
+  }
+
+  private AMPluginDescriptorProto createAmPluginDescriptor(boolean enableYarn, boolean enableUber,
+                                                           boolean addCustom,
+                                                           TezUserPayloadProto payloadProto) {
+    AMPluginDescriptorProto.Builder builder = AMPluginDescriptorProto.newBuilder()
+        .setUberEnabled(enableUber)
+        .setContainersEnabled(enableYarn);
+    if (addCustom) {
+      builder.addTaskSchedulers(
+          TezNamedEntityDescriptorProto.newBuilder()
+              .setName(TS_NAME)
+              .setEntityDescriptor(
+                  DAGProtos.TezEntityDescriptorProto.newBuilder()
+                      .setClassName(TS_NAME + CLASS_SUFFIX)
+                      .setTezUserPayload(payloadProto)))
+          .addContainerLaunchers(
+              TezNamedEntityDescriptorProto.newBuilder()
+                  .setName(CL_NAME)
+                  .setEntityDescriptor(
+                      DAGProtos.TezEntityDescriptorProto.newBuilder()
+                          .setClassName(CL_NAME + CLASS_SUFFIX)
+                          .setTezUserPayload(payloadProto)))
+          .addTaskCommunicators(
+              TezNamedEntityDescriptorProto.newBuilder()
+                  .setName(TC_NAME)
+                  .setEntityDescriptor(
+                      DAGProtos.TezEntityDescriptorProto.newBuilder()
+                          .setClassName(TC_NAME + CLASS_SUFFIX)
+                          .setTezUserPayload(payloadProto)));
+    }
+    return builder.build();
+  }
+
+
+}

http://git-wip-us.apache.org/repos/asf/tez/blob/267fe737/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 1cb69a8..639c487 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
@@ -34,6 +34,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Random;
 
+import com.google.common.collect.Lists;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.security.Credentials;
@@ -52,7 +53,9 @@ 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.NamedEntityDescriptor;
+import org.apache.tez.dag.api.TaskCommunicator;
 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.UserPayload;
 import org.apache.tez.serviceplugins.api.ContainerEndReason;
@@ -146,7 +149,10 @@ public class TestTaskAttemptListenerImplTezDag {
       throw new TezUncheckedException(e);
     }
     taskAttemptListener = new TaskAttemptListenerImplForTest(appContext,
-        mock(TaskHeartbeatHandler.class), mock(ContainerHeartbeatHandler.class), null, defaultPayload, false);
+        mock(TaskHeartbeatHandler.class), mock(ContainerHeartbeatHandler.class),
+        Lists.newArrayList(
+            new NamedEntityDescriptor(TezConstants.getTezYarnServicePluginName(), null)
+                .setUserPayload(defaultPayload)));
     TezTaskCommunicatorImpl taskCommunicator = (TezTaskCommunicatorImpl)taskAttemptListener.getTaskCommunicator();
     TezTaskUmbilicalProtocol tezUmbilical = taskCommunicator.getUmbilical();
 
@@ -301,7 +307,7 @@ public class TestTaskAttemptListenerImplTezDag {
 
   // TODO TEZ-2003 Move this into TestTezTaskCommunicator. Potentially other tests as well.
   @Test (timeout= 5000)
-  public void testPortRange_NotSpecified() {
+  public void testPortRange_NotSpecified() throws IOException {
     Configuration conf = new Configuration();
     JobTokenIdentifier identifier = new JobTokenIdentifier(new Text(
         "fakeIdentifier"));
@@ -309,14 +315,11 @@ public class TestTaskAttemptListenerImplTezDag {
         new JobTokenSecretManager());
     sessionToken.setService(identifier.getJobId());
     TokenCache.setSessionToken(sessionToken, credentials);
-    UserPayload userPayload = null;
-    try {
-      userPayload = TezUtils.createUserPayloadFromConf(conf);
-    } catch (IOException e) {
-      throw new TezUncheckedException(e);
-    }
+    UserPayload userPayload = TezUtils.createUserPayloadFromConf(conf);
     taskAttemptListener = new TaskAttemptListenerImpTezDag(appContext,
-        mock(TaskHeartbeatHandler.class), mock(ContainerHeartbeatHandler.class), null, userPayload, false);
+        mock(TaskHeartbeatHandler.class), mock(ContainerHeartbeatHandler.class), Lists.newArrayList(
+        new NamedEntityDescriptor(TezConstants.getTezYarnServicePluginName(), null)
+            .setUserPayload(userPayload)));
     // no exception happen, should started properly
     taskAttemptListener.init(conf);
     taskAttemptListener.start();
@@ -335,14 +338,12 @@ public class TestTaskAttemptListenerImplTezDag {
       TokenCache.setSessionToken(sessionToken, credentials);
 
       conf.set(TezConfiguration.TEZ_AM_TASK_AM_PORT_RANGE, port + "-" + port);
-      UserPayload userPayload = null;
-      try {
-        userPayload = TezUtils.createUserPayloadFromConf(conf);
-      } catch (IOException e) {
-        throw new TezUncheckedException(e);
-      }
+      UserPayload userPayload = TezUtils.createUserPayloadFromConf(conf);
+
       taskAttemptListener = new TaskAttemptListenerImpTezDag(appContext,
-          mock(TaskHeartbeatHandler.class), mock(ContainerHeartbeatHandler.class), null, userPayload, false);
+          mock(TaskHeartbeatHandler.class), mock(ContainerHeartbeatHandler.class), Lists
+          .newArrayList(new NamedEntityDescriptor(TezConstants.getTezYarnServicePluginName(), null)
+              .setUserPayload(userPayload)));
       taskAttemptListener.init(conf);
       taskAttemptListener.start();
       int resultedPort = taskAttemptListener.getTaskCommunicator(0).getAddress().getPort();
@@ -398,16 +399,13 @@ public class TestTaskAttemptListenerImplTezDag {
     public TaskAttemptListenerImplForTest(AppContext context,
                                           TaskHeartbeatHandler thh,
                                           ContainerHeartbeatHandler chh,
-                                          List<NamedEntityDescriptor> taskCommDescriptors,
-                                          UserPayload userPayload,
-                                          boolean isPureLocalMode) {
-      super(context, thh, chh, taskCommDescriptors, userPayload,
-          isPureLocalMode);
+                                          List<NamedEntityDescriptor> taskCommDescriptors) {
+      super(context, thh, chh, taskCommDescriptors);
     }
 
     @Override
-    protected TezTaskCommunicatorImpl createTezTaskCommunicator(TaskCommunicatorContext context) {
-      return new TezTaskCommunicatorImplForTest(context);
+    TaskCommunicator createDefaultTaskCommunicator(TaskCommunicatorContext taskCommunicatorContext) {
+      return new TezTaskCommunicatorImplForTest(taskCommunicatorContext);
     }
 
   }

http://git-wip-us.apache.org/repos/asf/tez/blob/267fe737/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
index 1c82bd8..abb5e42 100644
--- 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
@@ -26,6 +26,7 @@ import java.io.IOException;
 import java.util.HashMap;
 import java.util.Map;
 
+import com.google.common.collect.Lists;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
@@ -37,7 +38,9 @@ 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.common.TezUtils;
+import org.apache.tez.dag.api.NamedEntityDescriptor;
 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.UserPayload;
 import org.apache.tez.serviceplugins.api.TaskAttemptEndReason;
@@ -83,7 +86,8 @@ public class TestTaskAttemptListenerImplTezDag2 {
     UserPayload userPayload = TezUtils.createUserPayloadFromConf(conf);
     TaskAttemptListenerImpTezDag taskAttemptListener =
         new TaskAttemptListenerImpTezDag(appContext, mock(TaskHeartbeatHandler.class),
-            mock(ContainerHeartbeatHandler.class), null, userPayload, false);
+            mock(ContainerHeartbeatHandler.class), Lists.newArrayList(new NamedEntityDescriptor(
+            TezConstants.getTezYarnServicePluginName(), null).setUserPayload(userPayload)));
 
     TaskSpec taskSpec1 = mock(TaskSpec.class);
     TezTaskAttemptID taskAttemptId1 = mock(TezTaskAttemptID.class);

http://git-wip-us.apache.org/repos/asf/tez/blob/267fe737/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskCommunicatorManager.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskCommunicatorManager.java b/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskCommunicatorManager.java
new file mode 100644
index 0000000..c76aa50
--- /dev/null
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskCommunicatorManager.java
@@ -0,0 +1,369 @@
+/*
+ * 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.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.eq;
+import static org.mockito.Mockito.RETURNS_DEEP_STUBS;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.nio.ByteBuffer;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.Credentials;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.LocalResource;
+import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.tez.common.TezUtils;
+import org.apache.tez.dag.api.NamedEntityDescriptor;
+import org.apache.tez.dag.api.TaskCommunicator;
+import org.apache.tez.dag.api.TaskCommunicatorContext;
+import org.apache.tez.dag.api.TezConstants;
+import org.apache.tez.dag.api.UserPayload;
+import org.apache.tez.dag.api.event.VertexStateUpdate;
+import org.apache.tez.dag.records.TezTaskAttemptID;
+import org.apache.tez.runtime.api.impl.TaskSpec;
+import org.apache.tez.serviceplugins.api.ContainerEndReason;
+import org.apache.tez.serviceplugins.api.TaskAttemptEndReason;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestTaskCommunicatorManager {
+
+  @Before
+  @After
+  public void reset() {
+    TaskCommManagerForMultipleCommTest.reset();
+  }
+
+  @Test(timeout = 5000)
+  public void testNoTaskCommSpecified() throws IOException {
+
+    AppContext appContext = mock(AppContext.class);
+    TaskHeartbeatHandler thh = mock(TaskHeartbeatHandler.class);
+    ContainerHeartbeatHandler chh = mock(ContainerHeartbeatHandler.class);
+
+    try {
+      new TaskCommManagerForMultipleCommTest(appContext, thh, chh, null);
+      fail("Initialization should have failed without a TaskComm specified");
+    } catch (IllegalArgumentException e) {
+
+    }
+
+
+  }
+
+  @Test(timeout = 5000)
+  public void testCustomTaskCommSpecified() throws IOException {
+
+    AppContext appContext = mock(AppContext.class);
+    TaskHeartbeatHandler thh = mock(TaskHeartbeatHandler.class);
+    ContainerHeartbeatHandler chh = mock(ContainerHeartbeatHandler.class);
+
+    String customTaskCommName = "customTaskComm";
+    List<NamedEntityDescriptor> taskCommDescriptors = new LinkedList<>();
+    ByteBuffer bb = ByteBuffer.allocate(4);
+    bb.putInt(0, 3);
+    UserPayload customPayload = UserPayload.create(bb);
+    taskCommDescriptors.add(
+        new NamedEntityDescriptor(customTaskCommName, FakeTaskComm.class.getName())
+            .setUserPayload(customPayload));
+
+    TaskCommManagerForMultipleCommTest tcm =
+        new TaskCommManagerForMultipleCommTest(appContext, thh, chh, taskCommDescriptors);
+
+    try {
+      tcm.init(new Configuration(false));
+      tcm.start();
+
+      assertEquals(1, tcm.getNumTaskComms());
+      assertFalse(tcm.getYarnTaskCommCreated());
+      assertFalse(tcm.getUberTaskCommCreated());
+
+      assertEquals(customTaskCommName, tcm.getTaskCommName(0));
+      assertEquals(bb, tcm.getTaskCommContext(0).getInitialUserPayload().getPayload());
+
+    } finally {
+      tcm.stop();
+    }
+  }
+
+  @Test(timeout = 5000)
+  public void testMultipleTaskComms() throws IOException {
+
+    AppContext appContext = mock(AppContext.class);
+    TaskHeartbeatHandler thh = mock(TaskHeartbeatHandler.class);
+    ContainerHeartbeatHandler chh = mock(ContainerHeartbeatHandler.class);
+    Configuration conf = new Configuration(false);
+    conf.set("testkey", "testvalue");
+    UserPayload defaultPayload = TezUtils.createUserPayloadFromConf(conf);
+
+    String customTaskCommName = "customTaskComm";
+    List<NamedEntityDescriptor> taskCommDescriptors = new LinkedList<>();
+    ByteBuffer bb = ByteBuffer.allocate(4);
+    bb.putInt(0, 3);
+    UserPayload customPayload = UserPayload.create(bb);
+    taskCommDescriptors.add(
+        new NamedEntityDescriptor(customTaskCommName, FakeTaskComm.class.getName())
+            .setUserPayload(customPayload));
+    taskCommDescriptors
+        .add(new NamedEntityDescriptor(TezConstants.getTezYarnServicePluginName(), null).setUserPayload(defaultPayload));
+
+    TaskCommManagerForMultipleCommTest tcm =
+        new TaskCommManagerForMultipleCommTest(appContext, thh, chh, taskCommDescriptors);
+
+    try {
+      tcm.init(new Configuration(false));
+      tcm.start();
+
+      assertEquals(2, tcm.getNumTaskComms());
+      assertTrue(tcm.getYarnTaskCommCreated());
+      assertFalse(tcm.getUberTaskCommCreated());
+
+      assertEquals(customTaskCommName, tcm.getTaskCommName(0));
+      assertEquals(bb, tcm.getTaskCommContext(0).getInitialUserPayload().getPayload());
+
+      assertEquals(TezConstants.getTezYarnServicePluginName(), tcm.getTaskCommName(1));
+      Configuration confParsed = TezUtils
+          .createConfFromUserPayload(tcm.getTaskCommContext(1).getInitialUserPayload());
+      assertEquals("testvalue", confParsed.get("testkey"));
+    } finally {
+      tcm.stop();
+    }
+  }
+
+  @Test(timeout = 5000)
+  public void testEventRouting() throws Exception {
+
+    AppContext appContext = mock(AppContext.class, RETURNS_DEEP_STUBS);
+    NodeId nodeId = NodeId.newInstance("host1", 3131);
+    when(appContext.getAllContainers().get(any(ContainerId.class)).getContainer().getNodeId())
+        .thenReturn(nodeId);
+    TaskHeartbeatHandler thh = mock(TaskHeartbeatHandler.class);
+    ContainerHeartbeatHandler chh = mock(ContainerHeartbeatHandler.class);
+    Configuration conf = new Configuration(false);
+    conf.set("testkey", "testvalue");
+    UserPayload defaultPayload = TezUtils.createUserPayloadFromConf(conf);
+
+    String customTaskCommName = "customTaskComm";
+    List<NamedEntityDescriptor> taskCommDescriptors = new LinkedList<>();
+    ByteBuffer bb = ByteBuffer.allocate(4);
+    bb.putInt(0, 3);
+    UserPayload customPayload = UserPayload.create(bb);
+    taskCommDescriptors.add(
+        new NamedEntityDescriptor(customTaskCommName, FakeTaskComm.class.getName())
+            .setUserPayload(customPayload));
+    taskCommDescriptors
+        .add(new NamedEntityDescriptor(TezConstants.getTezYarnServicePluginName(), null).setUserPayload(defaultPayload));
+
+    TaskCommManagerForMultipleCommTest tcm =
+        new TaskCommManagerForMultipleCommTest(appContext, thh, chh, taskCommDescriptors);
+
+    try {
+      tcm.init(new Configuration(false));
+      tcm.start();
+
+      assertEquals(2, tcm.getNumTaskComms());
+      assertTrue(tcm.getYarnTaskCommCreated());
+      assertFalse(tcm.getUberTaskCommCreated());
+
+      verify(tcm.getTestTaskComm(0)).initialize();
+      verify(tcm.getTestTaskComm(0)).start();
+      verify(tcm.getTestTaskComm(1)).initialize();
+      verify(tcm.getTestTaskComm(1)).start();
+
+
+      ContainerId containerId1 = mock(ContainerId.class);
+      tcm.registerRunningContainer(containerId1, 0);
+      verify(tcm.getTestTaskComm(0)).registerRunningContainer(eq(containerId1), eq("host1"),
+          eq(3131));
+
+      ContainerId containerId2 = mock(ContainerId.class);
+      tcm.registerRunningContainer(containerId2, 1);
+      verify(tcm.getTestTaskComm(1)).registerRunningContainer(eq(containerId2), eq("host1"),
+          eq(3131));
+
+    } finally {
+      tcm.stop();
+      verify(tcm.getTaskCommunicator(0)).shutdown();
+      verify(tcm.getTaskCommunicator(1)).shutdown();
+    }
+  }
+
+
+  static class TaskCommManagerForMultipleCommTest extends TaskAttemptListenerImpTezDag {
+
+    // All variables setup as static since methods being overridden are invoked by the ContainerLauncherRouter ctor,
+    // and regular variables will not be initialized at this point.
+    private static final AtomicInteger numTaskComms = new AtomicInteger(0);
+    private static final Set<Integer> taskCommIndices = new HashSet<>();
+    private static final TaskCommunicator yarnTaskComm = mock(TaskCommunicator.class);
+    private static final TaskCommunicator uberTaskComm = mock(TaskCommunicator.class);
+    private static final AtomicBoolean yarnTaskCommCreated = new AtomicBoolean(false);
+    private static final AtomicBoolean uberTaskCommCreated = new AtomicBoolean(false);
+
+    private static final List<TaskCommunicatorContext> taskCommContexts =
+        new LinkedList<>();
+    private static final List<String> taskCommNames = new LinkedList<>();
+    private static final List<TaskCommunicator> testTaskComms = new LinkedList<>();
+
+
+    public static void reset() {
+      numTaskComms.set(0);
+      taskCommIndices.clear();
+      yarnTaskCommCreated.set(false);
+      uberTaskCommCreated.set(false);
+      taskCommContexts.clear();
+      taskCommNames.clear();
+      testTaskComms.clear();
+    }
+
+    public TaskCommManagerForMultipleCommTest(AppContext context,
+                                              TaskHeartbeatHandler thh,
+                                              ContainerHeartbeatHandler chh,
+                                              List<NamedEntityDescriptor> taskCommunicatorDescriptors) {
+      super(context, thh, chh, taskCommunicatorDescriptors);
+    }
+
+    @Override
+    TaskCommunicator createTaskCommunicator(NamedEntityDescriptor taskCommDescriptor,
+                                            int taskCommIndex) {
+      numTaskComms.incrementAndGet();
+      boolean added = taskCommIndices.add(taskCommIndex);
+      assertTrue("Cannot add multiple taskComms with the same index", added);
+      taskCommNames.add(taskCommDescriptor.getEntityName());
+      return super.createTaskCommunicator(taskCommDescriptor, taskCommIndex);
+    }
+
+    @Override
+    TaskCommunicator createDefaultTaskCommunicator(
+        TaskCommunicatorContext taskCommunicatorContext) {
+      taskCommContexts.add(taskCommunicatorContext);
+      yarnTaskCommCreated.set(true);
+      testTaskComms.add(yarnTaskComm);
+      return yarnTaskComm;
+    }
+
+    @Override
+    TaskCommunicator createUberTaskCommunicator(TaskCommunicatorContext taskCommunicatorContext) {
+      taskCommContexts.add(taskCommunicatorContext);
+      uberTaskCommCreated.set(true);
+      testTaskComms.add(uberTaskComm);
+      return uberTaskComm;
+    }
+
+    @Override
+    TaskCommunicator createCustomTaskCommunicator(TaskCommunicatorContext taskCommunicatorContext,
+                                                  NamedEntityDescriptor taskCommDescriptor) {
+      taskCommContexts.add(taskCommunicatorContext);
+      TaskCommunicator spyComm =
+          spy(super.createCustomTaskCommunicator(taskCommunicatorContext, taskCommDescriptor));
+      testTaskComms.add(spyComm);
+      return spyComm;
+    }
+
+    public static int getNumTaskComms() {
+      return numTaskComms.get();
+    }
+
+    public static boolean getYarnTaskCommCreated() {
+      return yarnTaskCommCreated.get();
+    }
+
+    public static boolean getUberTaskCommCreated() {
+      return uberTaskCommCreated.get();
+    }
+
+    public static TaskCommunicatorContext getTaskCommContext(int taskCommIndex) {
+      return taskCommContexts.get(taskCommIndex);
+    }
+
+    public static String getTaskCommName(int taskCommIndex) {
+      return taskCommNames.get(taskCommIndex);
+    }
+
+    public static TaskCommunicator getTestTaskComm(int taskCommIndex) {
+      return testTaskComms.get(taskCommIndex);
+    }
+  }
+
+  public static class FakeTaskComm extends TaskCommunicator {
+
+    public FakeTaskComm(TaskCommunicatorContext taskCommunicatorContext) {
+      super(taskCommunicatorContext);
+    }
+
+    @Override
+    public void registerRunningContainer(ContainerId containerId, String hostname, int port) {
+
+    }
+
+    @Override
+    public void registerContainerEnd(ContainerId containerId, ContainerEndReason endReason) {
+
+    }
+
+    @Override
+    public void registerRunningTaskAttempt(ContainerId containerId, TaskSpec taskSpec,
+                                           Map<String, LocalResource> additionalResources,
+                                           Credentials credentials, boolean credentialsChanged,
+                                           int priority) {
+
+    }
+
+    @Override
+    public void unregisterRunningTaskAttempt(TezTaskAttemptID taskAttemptID,
+                                             TaskAttemptEndReason endReason) {
+
+    }
+
+    @Override
+    public InetSocketAddress getAddress() {
+      return null;
+    }
+
+    @Override
+    public void onVertexStateUpdated(VertexStateUpdate stateUpdate) throws Exception {
+
+    }
+
+    @Override
+    public void dagComplete(String dagName) {
+
+    }
+
+    @Override
+    public Object getMetaInfo() {
+      return null;
+    }
+  }
+}


[35/50] [abbrv] tez git commit: TEZ-2441. Add tests for TezTaskRunner2. (sseth)

Posted by ss...@apache.org.
TEZ-2441. Add tests for TezTaskRunner2. (sseth)


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

Branch: refs/heads/TEZ-2003
Commit: 25dd309778166c2f7db829234bd12aae2e76ecb2
Parents: 17feebc
Author: Siddharth Seth <ss...@apache.org>
Authored: Wed Jul 29 18:25:18 2015 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Fri Aug 14 13:47:09 2015 -0700

----------------------------------------------------------------------
 TEZ-2003-CHANGES.txt                            |   1 +
 .../org/apache/tez/runtime/task/TezChild.java   |   5 +-
 .../apache/tez/runtime/task/TezTaskRunner.java  |   2 +-
 .../apache/tez/runtime/task/TezTaskRunner2.java |  42 +-
 .../runtime/task/TaskExecutionTestHelpers.java  | 451 +++++++++++++
 .../runtime/task/TestContainerExecution.java    |  59 ++
 .../tez/runtime/task/TestTaskExecution.java     | 400 +-----------
 .../tez/runtime/task/TestTaskExecution2.java    | 638 +++++++++++++++++++
 .../src/test/resources/log4j.properties         |  19 +
 9 files changed, 1213 insertions(+), 404 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/25dd3097/TEZ-2003-CHANGES.txt
----------------------------------------------------------------------
diff --git a/TEZ-2003-CHANGES.txt b/TEZ-2003-CHANGES.txt
index b88044b..9d72d92 100644
--- a/TEZ-2003-CHANGES.txt
+++ b/TEZ-2003-CHANGES.txt
@@ -39,5 +39,6 @@ ALL CHANGES:
   TEZ-2651. Pluggable services should not extend AbstractService.
   TEZ-2652. Cleanup the way services are specified for an AM and vertices.
   TEZ-2653. Change service contexts to expose a user specified payload instead of the AM configuration.
+  TEZ-2441. Add tests for TezTaskRunner2.
 
 INCOMPATIBLE CHANGES:

http://git-wip-us.apache.org/repos/asf/tez/blob/25dd3097/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 353fe23..b64ec37 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
@@ -49,7 +49,6 @@ import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.yarn.YarnUncaughtExceptionHandler;
 import org.apache.hadoop.yarn.api.ApplicationConstants;
 import org.apache.hadoop.yarn.api.ApplicationConstants.Environment;
-import org.apache.log4j.LogManager;
 import org.apache.tez.common.ContainerContext;
 import org.apache.tez.common.ContainerTask;
 import org.apache.tez.common.TezCommonUtils;
@@ -68,7 +67,6 @@ 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.apache.tez.runtime.internals.api.TaskReporterInterface;
 import org.slf4j.Logger;
@@ -256,6 +254,7 @@ public class TezChild {
         boolean shouldDie;
         try {
           TaskRunner2Result result = taskRunner.run();
+          LOG.info("TaskRunner2Result: {}", result);
           shouldDie = result.isContainerShutdownRequested();
           if (shouldDie) {
             LOG.info("Got a shouldDie notification via heartbeats for container {}. Shutting down", containerIdString);
@@ -377,8 +376,6 @@ public class TezChild {
       }
       if (ownUmbilical) {
         RPC.stopProxy(umbilical);
-        // TODO Temporary change. Revert. Ideally, move this over to the main method in TezChild if possible.
-//        LogManager.shutdown();
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/tez/blob/25dd3097/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 a82d87b..aebf6a9 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
@@ -250,7 +250,7 @@ public class TezTaskRunner implements TezUmbilical, ErrorReporter {
                   cause = ((UndeclaredThrowableException) cause).getCause();
                 }
                 maybeRegisterFirstException(cause);
-                LOG.info("Encounted an error while executing task: " + task.getTaskAttemptID(),
+                LOG.info("Encountered an error while executing task: " + task.getTaskAttemptID(),
                     cause);
                 try {
                   sendFailure(cause, "Failure while running task");

http://git-wip-us.apache.org/repos/asf/tez/blob/25dd3097/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezTaskRunner2.java
----------------------------------------------------------------------
diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezTaskRunner2.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezTaskRunner2.java
index a5fabb5..1a8828d 100644
--- a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezTaskRunner2.java
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezTaskRunner2.java
@@ -24,6 +24,7 @@ import java.util.concurrent.locks.Condition;
 import java.util.concurrent.locks.Lock;
 import java.util.concurrent.locks.ReentrantLock;
 
+import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Multimap;
 import com.google.common.util.concurrent.ListenableFuture;
@@ -48,6 +49,9 @@ import org.slf4j.LoggerFactory;
 
 public class TezTaskRunner2 {
 
+  // Behaviour changes as compared to TezTaskRunner
+  // - Exception not thrown. Instead returned in the result.
+  // - The actual exception is part of the result, instead of requiring a getCause().
 
   private static final Logger LOG = LoggerFactory.getLogger(TezTaskRunner2.class);
 
@@ -156,19 +160,7 @@ public class TezTaskRunner2 {
           }
         }
       }
-      if (executionResult != null) {
-        synchronized (this) {
-          if (isRunningState()) {
-            if (executionResult.error != null) {
-              trySettingEndReason(EndReason.TASK_ERROR);
-              registerFirstException(executionResult.error, null);
-            } else {
-              trySettingEndReason(EndReason.SUCCESS);
-              taskComplete.set(true);
-            }
-          }
-        }
-      }
+      processCallableResult(executionResult);
 
       switch (firstEndReason) {
         case SUCCESS:
@@ -249,6 +241,26 @@ public class TezTaskRunner2 {
     }
   }
 
+  // It's possible for the task to actually complete, and an alternate signal such as killTask/killContainer
+  // come in before the future has been processed by this thread. That condition is not handled - and
+  // the result of the execution will be determind by the thread order.
+  @VisibleForTesting
+  void processCallableResult(TaskRunner2CallableResult executionResult) {
+    if (executionResult != null) {
+      synchronized (this) {
+        if (isRunningState()) {
+          if (executionResult.error != null) {
+            trySettingEndReason(EndReason.TASK_ERROR);
+            registerFirstException(executionResult.error, null);
+          } else {
+            trySettingEndReason(EndReason.SUCCESS);
+            taskComplete.set(true);
+          }
+        }
+      }
+    }
+  }
+
   /**
    * Attempt to kill the running task, if it hasn't already completed for some other reason.
    * @return true if the task kill was honored, false otherwise
@@ -438,12 +450,12 @@ public class TezTaskRunner2 {
   private String getTaskDiagnosticsString(Throwable t, String message) {
     String diagnostics;
     if (t != null && message != null) {
-      diagnostics = "exceptionThrown=" + ExceptionUtils.getStackTrace(t) + ", errorMessage="
+      diagnostics = "Failure while running task: " + ExceptionUtils.getStackTrace(t) + ", errorMessage="
           + message;
     } else if (t == null && message == null) {
       diagnostics = "Unknown error";
     } else {
-      diagnostics = t != null ? "exceptionThrown=" + ExceptionUtils.getStackTrace(t)
+      diagnostics = t != null ? "Failure while running task: " + ExceptionUtils.getStackTrace(t)
           : " errorMessage=" + message;
     }
     return diagnostics;

http://git-wip-us.apache.org/repos/asf/tez/blob/25dd3097/tez-runtime-internals/src/test/java/org/apache/tez/runtime/task/TaskExecutionTestHelpers.java
----------------------------------------------------------------------
diff --git a/tez-runtime-internals/src/test/java/org/apache/tez/runtime/task/TaskExecutionTestHelpers.java b/tez-runtime-internals/src/test/java/org/apache/tez/runtime/task/TaskExecutionTestHelpers.java
new file mode 100644
index 0000000..fc42da3
--- /dev/null
+++ b/tez-runtime-internals/src/test/java/org/apache/tez/runtime/task/TaskExecutionTestHelpers.java
@@ -0,0 +1,451 @@
+/*
+ * 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.task;
+
+import static org.junit.Assert.fail;
+
+import java.io.IOException;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+
+import org.apache.hadoop.ipc.ProtocolSignature;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.tez.common.ContainerContext;
+import org.apache.tez.common.ContainerTask;
+import org.apache.tez.common.TezTaskUmbilicalProtocol;
+import org.apache.tez.dag.api.TezException;
+import org.apache.tez.dag.records.TezTaskAttemptID;
+import org.apache.tez.runtime.api.AbstractLogicalIOProcessor;
+import org.apache.tez.runtime.api.Event;
+import org.apache.tez.runtime.api.LogicalInput;
+import org.apache.tez.runtime.api.LogicalOutput;
+import org.apache.tez.runtime.api.ProcessorContext;
+import org.apache.tez.runtime.api.events.TaskAttemptCompletedEvent;
+import org.apache.tez.runtime.api.events.TaskAttemptFailedEvent;
+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.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class TaskExecutionTestHelpers {
+
+  public static final String HEARTBEAT_EXCEPTION_STRING = "HeartbeatException";
+
+  // Uses static fields for signaling. Ensure only used by one test at a time.
+  public static class TestProcessor extends AbstractLogicalIOProcessor {
+
+    public static final byte[] CONF_EMPTY = new byte[] { 0 };
+    public static final byte[] CONF_THROW_IO_EXCEPTION = new byte[] { 1 };
+    public static final byte[] CONF_THROW_TEZ_EXCEPTION = new byte[] { 2 };
+    public static final byte[] CONF_SIGNAL_FATAL_AND_THROW = new byte[] { 4 };
+    public static final byte[] CONF_SIGNAL_FATAL_AND_LOOP = new byte[] { 8 };
+    public static final byte[] CONF_SIGNAL_FATAL_AND_COMPLETE = new byte[] { 16 };
+
+    private static final Logger LOG = LoggerFactory.getLogger(TestProcessor.class);
+
+    private static final ReentrantLock processorLock = new ReentrantLock();
+    private static final Condition processorCondition = processorLock.newCondition();
+    private static final Condition loopCondition = processorLock.newCondition();
+    private static final Condition completionCondition = processorLock.newCondition();
+    private static final Condition runningCondition = processorLock.newCondition();
+    private static volatile boolean completed = false;
+    private static volatile boolean running = false;
+    private static volatile boolean looping = false;
+    private static volatile boolean signalled = false;
+
+    private static boolean receivedInterrupt = false;
+    private static volatile boolean wasAborted = false;
+
+    private boolean throwIOException = false;
+    private boolean throwTezException = false;
+    private boolean signalFatalAndThrow = false;
+    private boolean signalFatalAndLoop = false;
+    private boolean signalFatalAndComplete = false;
+
+    public TestProcessor(ProcessorContext context) {
+      super(context);
+    }
+
+    @Override
+    public void initialize() throws Exception {
+      parseConf(getContext().getUserPayload().deepCopyAsArray());
+    }
+
+    @Override
+    public void handleEvents(List<Event> processorEvents) {
+
+    }
+
+    @Override
+    public void close() throws Exception {
+
+    }
+
+    private void parseConf(byte[] bytes) {
+      byte b = bytes[0];
+      throwIOException = (b & 1) > 0;
+      throwTezException = (b & 2) > 0;
+      signalFatalAndThrow = (b & 4) > 0;
+      signalFatalAndLoop = (b & 8) > 0;
+      signalFatalAndComplete = (b & 16) > 0;
+    }
+
+    public static void reset() {
+      signalled = false;
+      receivedInterrupt = false;
+      completed = false;
+      running = false;
+      wasAborted = false;
+    }
+
+    public static void signal() {
+      LOG.info("Signalled");
+      processorLock.lock();
+      try {
+        signalled = true;
+        processorCondition.signal();
+      } finally {
+        processorLock.unlock();
+      }
+    }
+
+    public static void awaitStart() throws InterruptedException {
+      LOG.info("Awaiting Process run");
+      processorLock.lock();
+      try {
+        if (running) {
+          return;
+        }
+        runningCondition.await();
+      } finally {
+        processorLock.unlock();
+      }
+    }
+
+    public static void awaitLoop() throws InterruptedException {
+      LOG.info("Awaiting loop after signalling error");
+      processorLock.lock();
+      try {
+        if (looping) {
+          return;
+        }
+        loopCondition.await();
+      } finally {
+        processorLock.unlock();
+      }
+    }
+
+    public static void awaitCompletion() throws InterruptedException {
+      LOG.info("Await completion");
+      processorLock.lock();
+      try {
+        if (completed) {
+          return;
+        } else {
+          completionCondition.await();
+        }
+      } finally {
+        processorLock.unlock();
+      }
+    }
+
+    public static boolean wasInterrupted() {
+      processorLock.lock();
+      try {
+        return receivedInterrupt;
+      } finally {
+        processorLock.unlock();
+      }
+    }
+
+    public static boolean wasAborted() {
+      processorLock.lock();
+      try {
+        return wasAborted;
+      } finally {
+        processorLock.unlock();
+      }
+    }
+
+    @Override
+    public void abort() {
+      wasAborted = true;
+    }
+
+    @Override
+    public void run(Map<String, LogicalInput> inputs, Map<String, LogicalOutput> outputs) throws
+        Exception {
+      processorLock.lock();
+      running = true;
+      runningCondition.signal();
+      try {
+        try {
+          LOG.info("Signal is: " + signalled);
+          if (!signalled) {
+            LOG.info("Waiting for processor signal");
+            processorCondition.await();
+          }
+          if (Thread.currentThread().isInterrupted()) {
+            throw new InterruptedException();
+          }
+          LOG.info("Received processor signal");
+          if (throwIOException) {
+            throw createProcessorIOException();
+          } else if (throwTezException) {
+            throw createProcessorTezException();
+          } else if (signalFatalAndThrow) {
+            IOException io = new IOException("FATALERROR");
+            getContext().fatalError(io, "FATALERROR");
+            throw io;
+          } else if (signalFatalAndComplete) {
+            IOException io = new IOException("FATALERROR");
+            getContext().fatalError(io, "FATALERROR");
+            return;
+          } else if (signalFatalAndLoop) {
+            IOException io = createProcessorIOException();
+            getContext().fatalError(io, "FATALERROR");
+            LOG.info("looping");
+            looping = true;
+            loopCondition.signal();
+            LOG.info("Waiting for Processor signal again");
+            processorCondition.await();
+            LOG.info("Received second processor signal");
+          }
+        } catch (InterruptedException e) {
+          receivedInterrupt = true;
+        }
+      } finally {
+        completed = true;
+        completionCondition.signal();
+        processorLock.unlock();
+      }
+    }
+  }
+
+  public static TezException createProcessorTezException() {
+    return new TezException("TezException");
+  }
+
+  public static IOException createProcessorIOException() {
+    return new IOException("IOException");
+  }
+
+  public static class TezTaskUmbilicalForTest implements TezTaskUmbilicalProtocol {
+
+    private static final Logger LOG = LoggerFactory.getLogger(TezTaskUmbilicalForTest.class);
+
+    private final List<TezEvent> requestEvents = new LinkedList<TezEvent>();
+
+    private final ReentrantLock umbilicalLock = new ReentrantLock();
+    private final Condition eventCondition = umbilicalLock.newCondition();
+    private boolean pendingEvent = false;
+    private boolean eventEnacted = false;
+
+    volatile int getTaskInvocations = 0;
+
+    private boolean shouldThrowException = false;
+    private boolean shouldSendDieSignal = false;
+
+    public void signalThrowException() {
+      umbilicalLock.lock();
+      try {
+        shouldThrowException = true;
+        pendingEvent = true;
+      } finally {
+        umbilicalLock.unlock();
+      }
+    }
+
+    public void signalSendShouldDie() {
+      umbilicalLock.lock();
+      try {
+        shouldSendDieSignal = true;
+        pendingEvent = true;
+      } finally {
+        umbilicalLock.unlock();
+      }
+    }
+
+    public void awaitRegisteredEvent() throws InterruptedException {
+      umbilicalLock.lock();
+      try {
+        if (eventEnacted) {
+          return;
+        }
+        LOG.info("Awaiting event");
+        eventCondition.await();
+      } finally {
+        umbilicalLock.unlock();
+      }
+    }
+
+    public void resetTrackedEvents() {
+      umbilicalLock.lock();
+      try {
+        requestEvents.clear();
+      } finally {
+        umbilicalLock.unlock();
+      }
+    }
+
+    public void verifyNoCompletionEvents() {
+      umbilicalLock.lock();
+      try {
+        for (TezEvent event : requestEvents) {
+          if (event.getEvent() instanceof TaskAttemptFailedEvent) {
+            fail("Found a TaskAttemptFailedEvent when not expected");
+          }
+          if (event.getEvent() instanceof TaskAttemptCompletedEvent) {
+            fail("Found a TaskAttemptCompletedvent when not expected");
+          }
+        }
+      } finally {
+        umbilicalLock.unlock();
+      }
+    }
+
+    public void verifyTaskFailedEvent(String diagnostics) {
+      umbilicalLock.lock();
+      try {
+        for (TezEvent event : requestEvents) {
+          if (event.getEvent() instanceof TaskAttemptFailedEvent) {
+            TaskAttemptFailedEvent failedEvent = (TaskAttemptFailedEvent) event.getEvent();
+            if (failedEvent.getDiagnostics().startsWith(diagnostics)) {
+              return;
+            } else {
+              fail("Diagnostic message does not match expected message. Found [" +
+                  failedEvent.getDiagnostics() + "], Expected: [" + diagnostics + "]");
+            }
+          }
+        }
+        fail("No TaskAttemptFailedEvents sent over umbilical");
+      } finally {
+        umbilicalLock.unlock();
+      }
+    }
+
+    public void verifyTaskFailedEvent(String diagStart, String diagContains) {
+      umbilicalLock.lock();
+      try {
+        for (TezEvent event : requestEvents) {
+          if (event.getEvent() instanceof TaskAttemptFailedEvent) {
+            TaskAttemptFailedEvent failedEvent = (TaskAttemptFailedEvent) event.getEvent();
+            if (failedEvent.getDiagnostics().startsWith(diagStart)) {
+              if (diagContains != null) {
+                if (failedEvent.getDiagnostics().contains(diagContains)) {
+                  return;
+                } else {
+                  fail("Diagnostic message does not contain expected message. Found [" +
+                      failedEvent.getDiagnostics() + "], Expected: [" + diagContains + "]");
+                }
+              }
+            } else {
+              fail("Diagnostic message does not start with expected message. Found [" +
+                  failedEvent.getDiagnostics() + "], Expected: [" + diagStart + "]");
+            }
+          }
+        }
+        fail("No TaskAttemptFailedEvents sent over umbilical");
+      } finally {
+        umbilicalLock.unlock();
+      }
+    }
+
+    public void verifyTaskSuccessEvent() {
+      umbilicalLock.lock();
+      try {
+        for (TezEvent event : requestEvents) {
+          if (event.getEvent() instanceof TaskAttemptCompletedEvent) {
+            return;
+          }
+        }
+        fail("No TaskAttemptFailedEvents sent over umbilical");
+      } finally {
+        umbilicalLock.unlock();
+      }
+    }
+
+    @Override
+    public long getProtocolVersion(String protocol, long clientVersion) throws IOException {
+      return 0;
+    }
+
+    @Override
+    public ProtocolSignature getProtocolSignature(String protocol, long clientVersion,
+                                                  int clientMethodsHash) throws IOException {
+      return null;
+    }
+
+    @Override
+    public ContainerTask getTask(ContainerContext containerContext) throws IOException {
+      // Return shouldDie = true
+      getTaskInvocations++;
+      return new ContainerTask(null, true, null, null, false);
+    }
+
+    @Override
+    public boolean canCommit(TezTaskAttemptID taskid) throws IOException {
+      return true;
+    }
+
+    @Override
+    public TezHeartbeatResponse heartbeat(TezHeartbeatRequest request) throws IOException,
+        TezException {
+      umbilicalLock.lock();
+      if (request.getEvents() != null) {
+        requestEvents.addAll(request.getEvents());
+      }
+      try {
+        if (shouldThrowException) {
+          LOG.info("TestUmbilical throwing Exception");
+          throw new IOException(HEARTBEAT_EXCEPTION_STRING);
+        }
+        TezHeartbeatResponse response = new TezHeartbeatResponse();
+        response.setLastRequestId(request.getRequestId());
+        if (shouldSendDieSignal) {
+          LOG.info("TestUmbilical returning shouldDie=true");
+          response.setShouldDie();
+        }
+        return response;
+      } finally {
+        if (pendingEvent) {
+          eventEnacted = true;
+          LOG.info("Signalling Event");
+          eventCondition.signal();
+        }
+        umbilicalLock.unlock();
+      }
+    }
+  }
+
+  public static ContainerId createContainerId(ApplicationId appId) {
+    ApplicationAttemptId appAttemptId = ApplicationAttemptId.newInstance(appId, 1);
+    ContainerId containerId = ContainerId.newInstance(appAttemptId, 1);
+    return containerId;
+  }
+
+  public static TaskReporter createTaskReporter(ApplicationId appId, TezTaskUmbilicalForTest umbilical) {
+    TaskReporter taskReporter = new TaskReporter(umbilical, 100, 1000, 100, new AtomicLong(0),
+        createContainerId(appId).toString());
+    return taskReporter;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tez/blob/25dd3097/tez-runtime-internals/src/test/java/org/apache/tez/runtime/task/TestContainerExecution.java
----------------------------------------------------------------------
diff --git a/tez-runtime-internals/src/test/java/org/apache/tez/runtime/task/TestContainerExecution.java b/tez-runtime-internals/src/test/java/org/apache/tez/runtime/task/TestContainerExecution.java
new file mode 100644
index 0000000..c1616af
--- /dev/null
+++ b/tez-runtime-internals/src/test/java/org/apache/tez/runtime/task/TestContainerExecution.java
@@ -0,0 +1,59 @@
+/*
+ * 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.task;
+
+import static org.junit.Assert.assertEquals;
+
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+
+import com.google.common.util.concurrent.ListenableFuture;
+import com.google.common.util.concurrent.ListeningExecutorService;
+import com.google.common.util.concurrent.MoreExecutors;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.tez.common.ContainerContext;
+import org.apache.tez.common.ContainerTask;
+import org.junit.Test;
+
+public class TestContainerExecution {
+
+  @Test(timeout = 5000)
+  public void testGetTaskShouldDie() throws InterruptedException, ExecutionException {
+    ListeningExecutorService executor = null;
+    try {
+      ExecutorService rawExecutor = Executors.newFixedThreadPool(1);
+      executor = MoreExecutors.listeningDecorator(rawExecutor);
+      ApplicationId appId = ApplicationId.newInstance(10000, 1);
+      ApplicationAttemptId appAttemptId = ApplicationAttemptId.newInstance(appId, 1);
+      ContainerId containerId = ContainerId.newInstance(appAttemptId, 1);
+
+      TaskExecutionTestHelpers.TezTaskUmbilicalForTest
+          umbilical = new TaskExecutionTestHelpers.TezTaskUmbilicalForTest();
+      ContainerContext containerContext = new ContainerContext(containerId.toString());
+
+      ContainerReporter containerReporter = new ContainerReporter(umbilical, containerContext, 100);
+      ListenableFuture<ContainerTask> getTaskFuture = executor.submit(containerReporter);
+
+      getTaskFuture.get();
+      assertEquals(1, umbilical.getTaskInvocations);
+
+    } finally {
+      executor.shutdownNow();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/tez/blob/25dd3097/tez-runtime-internals/src/test/java/org/apache/tez/runtime/task/TestTaskExecution.java
----------------------------------------------------------------------
diff --git a/tez-runtime-internals/src/test/java/org/apache/tez/runtime/task/TestTaskExecution.java b/tez-runtime-internals/src/test/java/org/apache/tez/runtime/task/TestTaskExecution.java
index 1bcb337..a99416a 100644
--- a/tez-runtime-internals/src/test/java/org/apache/tez/runtime/task/TestTaskExecution.java
+++ b/tez-runtime-internals/src/test/java/org/apache/tez/runtime/task/TestTaskExecution.java
@@ -18,8 +18,8 @@
 
 package org.apache.tez.runtime.task;
 
+import static org.apache.tez.runtime.task.TaskExecutionTestHelpers.createTaskReporter;
 import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
@@ -28,30 +28,18 @@ import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.HashMap;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
 import java.util.UUID;
 import java.util.concurrent.Callable;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.Future;
-import java.util.concurrent.atomic.AtomicLong;
-import java.util.concurrent.locks.Condition;
-import java.util.concurrent.locks.ReentrantLock;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.ipc.ProtocolSignature;
 import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
-import org.apache.hadoop.yarn.api.records.ContainerId;
-import org.apache.tez.common.ContainerContext;
-import org.apache.tez.common.ContainerTask;
-import org.apache.tez.common.TezTaskUmbilicalProtocol;
 import org.apache.tez.dag.api.ProcessorDescriptor;
 import org.apache.tez.dag.api.TezConfiguration;
 import org.apache.tez.dag.api.TezException;
@@ -60,21 +48,13 @@ 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.AbstractLogicalIOProcessor;
-import org.apache.tez.runtime.api.Event;
-import org.apache.tez.runtime.api.LogicalInput;
-import org.apache.tez.runtime.api.LogicalOutput;
-import org.apache.tez.runtime.api.ProcessorContext;
-import org.apache.tez.runtime.api.events.TaskAttemptCompletedEvent;
-import org.apache.tez.runtime.api.events.TaskAttemptFailedEvent;
 import org.apache.tez.runtime.api.impl.ExecutionContextImpl;
 import org.apache.tez.runtime.api.impl.InputSpec;
 import org.apache.tez.runtime.api.impl.OutputSpec;
 import org.apache.tez.runtime.api.impl.TaskSpec;
-import org.apache.tez.runtime.api.impl.TezEvent;
-import org.apache.tez.runtime.api.impl.TezHeartbeatRequest;
-import org.apache.tez.runtime.api.impl.TezHeartbeatResponse;
 import org.apache.tez.runtime.common.resources.ScalingAllocator;
+import org.apache.tez.runtime.task.TaskExecutionTestHelpers.TestProcessor;
+import org.apache.tez.runtime.task.TaskExecutionTestHelpers.TezTaskUmbilicalForTest;
 import org.junit.AfterClass;
 import org.junit.Before;
 import org.junit.Test;
@@ -82,7 +62,6 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import com.google.common.collect.HashMultimap;
-import com.google.common.util.concurrent.ListenableFuture;
 import com.google.common.util.concurrent.ListeningExecutorService;
 import com.google.common.util.concurrent.MoreExecutors;
 
@@ -91,7 +70,7 @@ public class TestTaskExecution {
 
   private static final Logger LOG = LoggerFactory.getLogger(TestTaskExecution.class);
 
-  private static final String HEARTBEAT_EXCEPTION_STRING = "HeartbeatException";
+
 
   private static final Configuration defaultConf = new Configuration();
   private static final FileSystem localFs;
@@ -137,7 +116,7 @@ public class TestTaskExecution {
       TezTaskRunner taskRunner = createTaskRunner(appId, umbilical, taskReporter, executor,
           TestProcessor.CONF_EMPTY);
       // Setup the executor
-      Future<Boolean> taskRunnerFuture = taskExecutor.submit(new TaskRunnerCallable(taskRunner));
+      Future<Boolean> taskRunnerFuture = taskExecutor.submit(new TaskRunnerCallable1ForTest(taskRunner));
       // Signal the processor to go through
       TestProcessor.signal();
       boolean result = taskRunnerFuture.get();
@@ -164,7 +143,7 @@ public class TestTaskExecution {
       TezTaskRunner taskRunner = createTaskRunner(appId, umbilical, taskReporter, executor,
           TestProcessor.CONF_EMPTY);
       // Setup the executor
-      Future<Boolean> taskRunnerFuture = taskExecutor.submit(new TaskRunnerCallable(taskRunner));
+      Future<Boolean> taskRunnerFuture = taskExecutor.submit(new TaskRunnerCallable1ForTest(taskRunner));
       // Signal the processor to go through
       TestProcessor.signal();
       boolean result = taskRunnerFuture.get();
@@ -176,7 +155,7 @@ public class TestTaskExecution {
       taskRunner = createTaskRunner(appId, umbilical, taskReporter, executor,
           TestProcessor.CONF_EMPTY);
       // Setup the executor
-      taskRunnerFuture = taskExecutor.submit(new TaskRunnerCallable(taskRunner));
+      taskRunnerFuture = taskExecutor.submit(new TaskRunnerCallable1ForTest(taskRunner));
       // Signal the processor to go through
       TestProcessor.signal();
       result = taskRunnerFuture.get();
@@ -188,7 +167,7 @@ public class TestTaskExecution {
     }
   }
 
-  // test tasked failed due to exception in Processor
+  // test task failed due to exception in Processor
   @Test(timeout = 5000)
   public void testFailedTask() throws IOException, InterruptedException, TezException {
 
@@ -203,7 +182,7 @@ public class TestTaskExecution {
       TezTaskRunner taskRunner = createTaskRunner(appId, umbilical, taskReporter, executor,
           TestProcessor.CONF_THROW_TEZ_EXCEPTION);
       // Setup the executor
-      Future<Boolean> taskRunnerFuture = taskExecutor.submit(new TaskRunnerCallable(taskRunner));
+      Future<Boolean> taskRunnerFuture = taskExecutor.submit(new TaskRunnerCallable1ForTest(taskRunner));
       // Signal the processor to go through
       TestProcessor.awaitStart();
       TestProcessor.signal();
@@ -238,7 +217,7 @@ public class TestTaskExecution {
       TezTaskRunner taskRunner = createTaskRunner(appId, umbilical, taskReporter, executor,
           "NotExitedProcessor", TestProcessor.CONF_THROW_TEZ_EXCEPTION);
       // Setup the executor
-      Future<Boolean> taskRunnerFuture = taskExecutor.submit(new TaskRunnerCallable(taskRunner));
+      Future<Boolean> taskRunnerFuture = taskExecutor.submit(new TaskRunnerCallable1ForTest(taskRunner));
       try {
         taskRunnerFuture.get();
       } catch (ExecutionException e) {
@@ -268,7 +247,7 @@ public class TestTaskExecution {
       TezTaskRunner taskRunner = createTaskRunner(appId, umbilical, taskReporter, executor,
           TestProcessor.CONF_EMPTY);
       // Setup the executor
-      Future<Boolean> taskRunnerFuture = taskExecutor.submit(new TaskRunnerCallable(taskRunner));
+      Future<Boolean> taskRunnerFuture = taskExecutor.submit(new TaskRunnerCallable1ForTest(taskRunner));
       // Signal the processor to go through
       TestProcessor.awaitStart();
       umbilical.signalThrowException();
@@ -280,7 +259,7 @@ public class TestTaskExecution {
       } catch (ExecutionException e) {
         Throwable cause = e.getCause();
         assertTrue(cause instanceof IOException);
-        assertTrue(cause.getMessage().contains(HEARTBEAT_EXCEPTION_STRING));
+        assertTrue(cause.getMessage().contains(TaskExecutionTestHelpers.HEARTBEAT_EXCEPTION_STRING));
       }
       TestProcessor.awaitCompletion();
       assertTrue(TestProcessor.wasInterrupted());
@@ -307,7 +286,7 @@ public class TestTaskExecution {
       TezTaskRunner taskRunner = createTaskRunner(appId, umbilical, taskReporter, executor,
           TestProcessor.CONF_EMPTY);
       // Setup the executor
-      Future<Boolean> taskRunnerFuture = taskExecutor.submit(new TaskRunnerCallable(taskRunner));
+      Future<Boolean> taskRunnerFuture = taskExecutor.submit(new TaskRunnerCallable1ForTest(taskRunner));
       // Signal the processor to go through
       TestProcessor.awaitStart();
       umbilical.signalSendShouldDie();
@@ -329,38 +308,14 @@ public class TestTaskExecution {
     }
   }
 
-  @Test(timeout = 5000)
-  public void testGetTaskShouldDie() throws InterruptedException, ExecutionException {
-    ListeningExecutorService executor = null;
-    try {
-      ExecutorService rawExecutor = Executors.newFixedThreadPool(1);
-      executor = MoreExecutors.listeningDecorator(rawExecutor);
-      ApplicationId appId = ApplicationId.newInstance(10000, 1);
-      ApplicationAttemptId appAttemptId = ApplicationAttemptId.newInstance(appId, 1);
-      ContainerId containerId = ContainerId.newInstance(appAttemptId, 1);
-
-      TezTaskUmbilicalForTest umbilical = new TezTaskUmbilicalForTest();
-      ContainerContext containerContext = new ContainerContext(containerId.toString());
-
-      ContainerReporter containerReporter = new ContainerReporter(umbilical, containerContext, 100);
-      ListenableFuture<ContainerTask> getTaskFuture = executor.submit(containerReporter);
-
-      getTaskFuture.get();
-      assertEquals(1, umbilical.getTaskInvocations);
-
-    } finally {
-      executor.shutdownNow();
-    }
-  }
-
   // Potential new tests
   // Different states - initialization failure, close failure
   // getTask states
 
-  private static class TaskRunnerCallable implements Callable<Boolean> {
+  private static class TaskRunnerCallable1ForTest implements Callable<Boolean> {
     private final TezTaskRunner taskRunner;
 
-    public TaskRunnerCallable(TezTaskRunner taskRunner) {
+    public TaskRunnerCallable1ForTest(TezTaskRunner taskRunner) {
       this.taskRunner = taskRunner;
     }
 
@@ -370,328 +325,9 @@ public class TestTaskExecution {
     }
   }
 
-  // Uses static fields for signaling. Ensure only used by one test at a time.
-  public static class TestProcessor extends AbstractLogicalIOProcessor {
-
-    public static final byte[] CONF_EMPTY = new byte[] { 0 };
-    public static final byte[] CONF_THROW_IO_EXCEPTION = new byte[] { 1 };
-    public static final byte[] CONF_THROW_TEZ_EXCEPTION = new byte[] { 2 };
-    public static final byte[] CONF_SIGNAL_FATAL_AND_THROW = new byte[] { 4 };
-    public static final byte[] CONF_SIGNAL_FATAL_AND_LOOP = new byte[] { 8 };
-    public static final byte[] CONF_SIGNAL_FATAL_AND_COMPLETE = new byte[] { 16 };
-
-    private static final Logger LOG = LoggerFactory.getLogger(TestProcessor.class);
-
-    private static final ReentrantLock processorLock = new ReentrantLock();
-    private static final Condition processorCondition = processorLock.newCondition();
-    private static final Condition completionCondition = processorLock.newCondition();
-    private static final Condition runningCondition = processorLock.newCondition();
-    private static boolean completed = false;
-    private static boolean running = false;
-    private static boolean signalled = false;
-
-    public static boolean receivedInterrupt = false;
-
-    private boolean throwIOException = false;
-    private boolean throwTezException = false;
-    private boolean signalFatalAndThrow = false;
-    private boolean signalFatalAndLoop = false;
-    private boolean signalFatalAndComplete = false;
-
-    public TestProcessor(ProcessorContext context) {
-      super(context);
-    }
-
-    @Override
-    public void initialize() throws Exception {
-      parseConf(getContext().getUserPayload().deepCopyAsArray());
-    }
-
-    @Override
-    public void handleEvents(List<Event> processorEvents) {
-
-    }
-
-    @Override
-    public void close() throws Exception {
-
-    }
 
-    private void parseConf(byte[] bytes) {
-      byte b = bytes[0];
-      throwIOException = (b & 1) > 1;
-      throwTezException = (b & 2) > 1;
-      signalFatalAndThrow = (b & 4) > 1;
-      signalFatalAndLoop = (b & 8) > 1;
-      signalFatalAndComplete = (b & 16) > 1;
-    }
 
-    public static void reset() {
-      signalled = false;
-      receivedInterrupt = false;
-      completed = false;
-      running = false;
-    }
 
-    public static void signal() {
-      LOG.info("Signalled");
-      processorLock.lock();
-      try {
-        signalled = true;
-        processorCondition.signal();
-      } finally {
-        processorLock.unlock();
-      }
-    }
-
-    public static void awaitStart() throws InterruptedException {
-      LOG.info("Awaiting Process run");
-      processorLock.lock();
-      try {
-        if (running) {
-          return;
-        }
-        runningCondition.await();
-      } finally {
-        processorLock.unlock();
-      }
-    }
-
-    public static void awaitCompletion() throws InterruptedException {
-      LOG.info("Await completion");
-      processorLock.lock();
-      try {
-        if (completed) {
-          return;
-        } else {
-          completionCondition.await();
-        }
-      } finally {
-        processorLock.unlock();
-      }
-    }
-
-    public static boolean wasInterrupted() {
-      processorLock.lock();
-      try {
-        return receivedInterrupt;
-      } finally {
-        processorLock.unlock();
-      }
-    }
-
-    @Override
-    public void run(Map<String, LogicalInput> inputs, Map<String, LogicalOutput> outputs) throws
-        Exception {
-      processorLock.lock();
-      running = true;
-      runningCondition.signal();
-      try {
-        try {
-          LOG.info("Signal is: " + signalled);
-          if (!signalled) {
-            LOG.info("Waiting for processor signal");
-            processorCondition.await();
-          }
-          if (Thread.currentThread().isInterrupted()) {
-            throw new InterruptedException();
-          }
-          LOG.info("Received processor signal");
-          if (throwIOException) {
-            throw new IOException();
-          } else if (throwTezException) {
-            throw new TezException("TezException");
-          } else if (signalFatalAndThrow) {
-            IOException io = new IOException("FATALERROR");
-            getContext().fatalError(io, "FATALERROR");
-            throw io;
-          } else if (signalFatalAndComplete) {
-            IOException io = new IOException("FATALERROR");
-            getContext().fatalError(io, "FATALERROR");
-            return;
-          } else if (signalFatalAndLoop) {
-            IOException io = new IOException("FATALERROR");
-            getContext().fatalError(io, "FATALERROR");
-            LOG.info("Waiting for Processor signal again");
-            processorCondition.await();
-            LOG.info("Received second processor signal");
-          }
-        } catch (InterruptedException e) {
-          receivedInterrupt = true;
-        }
-      } finally {
-        completed = true;
-        completionCondition.signal();
-        processorLock.unlock();
-      }
-    }
-  }
-
-  private static class TezTaskUmbilicalForTest implements TezTaskUmbilicalProtocol {
-
-    private static final Logger LOG = LoggerFactory.getLogger(TezTaskUmbilicalForTest.class);
-
-    private final List<TezEvent> requestEvents = new LinkedList<TezEvent>();
-
-    private final ReentrantLock umbilicalLock = new ReentrantLock();
-    private final Condition eventCondition = umbilicalLock.newCondition();
-    private boolean pendingEvent = false;
-    private boolean eventEnacted = false;
-
-    volatile int getTaskInvocations = 0;
-
-    private boolean shouldThrowException = false;
-    private boolean shouldSendDieSignal = false;
-
-    public void signalThrowException() {
-      umbilicalLock.lock();
-      try {
-        shouldThrowException = true;
-        pendingEvent = true;
-      } finally {
-        umbilicalLock.unlock();
-      }
-    }
-
-    public void signalSendShouldDie() {
-      umbilicalLock.lock();
-      try {
-        shouldSendDieSignal = true;
-        pendingEvent = true;
-      } finally {
-        umbilicalLock.unlock();
-      }
-    }
-
-    public void awaitRegisteredEvent() throws InterruptedException {
-      umbilicalLock.lock();
-      try {
-        if (eventEnacted) {
-          return;
-        }
-        LOG.info("Awaiting event");
-        eventCondition.await();
-      } finally {
-        umbilicalLock.unlock();
-      }
-    }
-
-    public void resetTrackedEvents() {
-      umbilicalLock.lock();
-      try {
-        requestEvents.clear();
-      } finally {
-        umbilicalLock.unlock();
-      }
-    }
-
-    public void verifyNoCompletionEvents() {
-      umbilicalLock.lock();
-      try {
-        for (TezEvent event : requestEvents) {
-          if (event.getEvent() instanceof TaskAttemptFailedEvent) {
-            fail("Found a TaskAttemptFailedEvent when not expected");
-          }
-          if (event.getEvent() instanceof TaskAttemptCompletedEvent) {
-            fail("Found a TaskAttemptCompletedvent when not expected");
-          }
-        }
-      } finally {
-        umbilicalLock.unlock();
-      }
-    }
-
-    public void verifyTaskFailedEvent(String diagnostics) {
-      umbilicalLock.lock();
-      try {
-        for (TezEvent event : requestEvents) {
-          if (event.getEvent() instanceof TaskAttemptFailedEvent) {
-            TaskAttemptFailedEvent failedEvent = (TaskAttemptFailedEvent)event.getEvent();
-            if(failedEvent.getDiagnostics().startsWith(diagnostics)){
-              return ;
-            } else {
-              fail("No detailed diagnostics message in TaskAttemptFailedEvent");
-            }
-          }
-        }
-        fail("No TaskAttemptFailedEvents sent over umbilical");
-      } finally {
-        umbilicalLock.unlock();
-      }
-    }
-
-    public void verifyTaskSuccessEvent() {
-      umbilicalLock.lock();
-      try {
-        for (TezEvent event : requestEvents) {
-          if (event.getEvent() instanceof TaskAttemptCompletedEvent) {
-            return;
-          }
-        }
-        fail("No TaskAttemptFailedEvents sent over umbilical");
-      } finally {
-        umbilicalLock.unlock();
-      }
-    }
-
-    @Override
-    public long getProtocolVersion(String protocol, long clientVersion) throws IOException {
-      return 0;
-    }
-
-    @Override
-    public ProtocolSignature getProtocolSignature(String protocol, long clientVersion,
-        int clientMethodsHash) throws IOException {
-      return null;
-    }
-
-    @Override
-    public ContainerTask getTask(ContainerContext containerContext) throws IOException {
-      // Return shouldDie = true
-      getTaskInvocations++;
-      return new ContainerTask(null, true, null, null, false);
-    }
-
-    @Override
-    public boolean canCommit(TezTaskAttemptID taskid) throws IOException {
-      return true;
-    }
-
-    @Override
-    public TezHeartbeatResponse heartbeat(TezHeartbeatRequest request) throws IOException,
-        TezException {
-      umbilicalLock.lock();
-      if (request.getEvents() != null) {
-        requestEvents.addAll(request.getEvents());
-      }
-      try {
-        if (shouldThrowException) {
-          LOG.info("TestUmbilical throwing Exception");
-          throw new IOException(HEARTBEAT_EXCEPTION_STRING);
-        }
-        TezHeartbeatResponse response = new TezHeartbeatResponse();
-        response.setLastRequestId(request.getRequestId());
-        if (shouldSendDieSignal) {
-          LOG.info("TestUmbilical returning shouldDie=true");
-          response.setShouldDie();
-        }
-        return response;
-      } finally {
-        if (pendingEvent) {
-          eventEnacted = true;
-          LOG.info("Signalling Event");
-          eventCondition.signal();
-        }
-        umbilicalLock.unlock();
-      }
-    }
-  }
-
-  private TaskReporter createTaskReporter(ApplicationId appId, TezTaskUmbilicalForTest umbilical) {
-    TaskReporter taskReporter = new TaskReporter(umbilical, 100, 1000, 100, new AtomicLong(0),
-        createContainerId(appId).toString());
-    return taskReporter;
-  }
 
   private TezTaskRunner createTaskRunner(ApplicationId appId, TezTaskUmbilicalForTest umbilical,
       TaskReporter taskReporter, ListeningExecutorService executor, byte[] processorConf)
@@ -722,9 +358,5 @@ public class TestTaskExecution {
     return taskRunner;
   }
 
-  private ContainerId createContainerId(ApplicationId appId) {
-    ApplicationAttemptId appAttemptId = ApplicationAttemptId.newInstance(appId, 1);
-    ContainerId containerId = ContainerId.newInstance(appAttemptId, 1);
-    return containerId;
-  }
+
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/25dd3097/tez-runtime-internals/src/test/java/org/apache/tez/runtime/task/TestTaskExecution2.java
----------------------------------------------------------------------
diff --git a/tez-runtime-internals/src/test/java/org/apache/tez/runtime/task/TestTaskExecution2.java b/tez-runtime-internals/src/test/java/org/apache/tez/runtime/task/TestTaskExecution2.java
new file mode 100644
index 0000000..12d9d3f
--- /dev/null
+++ b/tez-runtime-internals/src/test/java/org/apache/tez/runtime/task/TestTaskExecution2.java
@@ -0,0 +1,638 @@
+/*
+ * 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.task;
+
+import static org.apache.tez.runtime.task.TaskExecutionTestHelpers.createProcessorIOException;
+import static org.apache.tez.runtime.task.TaskExecutionTestHelpers.createProcessorTezException;
+import static org.apache.tez.runtime.task.TaskExecutionTestHelpers.createTaskReporter;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.UUID;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.HashMultimap;
+import com.google.common.collect.Multimap;
+import com.google.common.util.concurrent.ListeningExecutorService;
+import com.google.common.util.concurrent.MoreExecutors;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.tez.dag.api.ProcessorDescriptor;
+import org.apache.tez.dag.api.TezConfiguration;
+import org.apache.tez.dag.api.TezException;
+import org.apache.tez.dag.api.TezUncheckedException;
+import org.apache.tez.dag.api.UserPayload;
+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.ExecutionContext;
+import org.apache.tez.runtime.api.ObjectRegistry;
+import org.apache.tez.runtime.api.impl.ExecutionContextImpl;
+import org.apache.tez.runtime.api.impl.InputSpec;
+import org.apache.tez.runtime.api.impl.OutputSpec;
+import org.apache.tez.runtime.api.impl.TaskSpec;
+import org.apache.tez.runtime.common.resources.ScalingAllocator;
+import org.apache.tez.runtime.internals.api.TaskReporterInterface;
+import org.apache.tez.runtime.task.TaskExecutionTestHelpers.TestProcessor;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class TestTaskExecution2 {
+
+  private static final Logger LOG = LoggerFactory.getLogger(TestTaskExecution2.class);
+
+  private static final Configuration defaultConf = new Configuration();
+  private static final FileSystem localFs;
+  private static final Path workDir;
+
+  private static final ExecutorService taskExecutor = Executors.newFixedThreadPool(1);
+
+  static {
+    defaultConf.set("fs.defaultFS", "file:///");
+    defaultConf.set(TezConfiguration.TEZ_TASK_SCALE_MEMORY_ALLOCATOR_CLASS,
+        ScalingAllocator.class.getName());
+    try {
+      localFs = FileSystem.getLocal(defaultConf);
+      Path wd = new Path(System.getProperty("test.build.data", "/tmp"),
+          TestTaskExecution.class.getSimpleName());
+      workDir = localFs.makeQualified(wd);
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  @Before
+  public void reset() {
+    TestProcessor.reset();
+  }
+
+  @AfterClass
+  public static void shutdown() {
+    taskExecutor.shutdownNow();
+  }
+
+  @Test(timeout = 5000)
+  public void testSingleSuccessfulTask() throws IOException, InterruptedException, TezException,
+      ExecutionException {
+    ListeningExecutorService executor = null;
+    try {
+      ExecutorService rawExecutor = Executors.newFixedThreadPool(1);
+      executor = MoreExecutors.listeningDecorator(rawExecutor);
+      ApplicationId appId = ApplicationId.newInstance(10000, 1);
+      TaskExecutionTestHelpers.TezTaskUmbilicalForTest
+          umbilical = new TaskExecutionTestHelpers.TezTaskUmbilicalForTest();
+      TaskReporter taskReporter = createTaskReporter(appId, umbilical);
+
+      TezTaskRunner2 taskRunner = createTaskRunner(appId, umbilical, taskReporter, executor,
+          TestProcessor.CONF_EMPTY);
+      // Setup the executor
+      Future<TaskRunner2Result> taskRunnerFuture = taskExecutor.submit(
+          new TaskRunnerCallable2ForTest(taskRunner));
+      // Signal the processor to go through
+      TestProcessor.signal();
+      TaskRunner2Result result = taskRunnerFuture.get();
+      verifyTaskRunnerResult(result, EndReason.SUCCESS, null, false);
+      assertNull(taskReporter.currentCallable);
+      umbilical.verifyTaskSuccessEvent();
+      assertFalse(TestProcessor.wasAborted());
+    } finally {
+      executor.shutdownNow();
+    }
+  }
+
+  @Test(timeout = 5000)
+  public void testMultipleSuccessfulTasks() throws IOException, InterruptedException, TezException,
+      ExecutionException {
+
+    ListeningExecutorService executor = null;
+    try {
+      ExecutorService rawExecutor = Executors.newFixedThreadPool(1);
+      executor = MoreExecutors.listeningDecorator(rawExecutor);
+      ApplicationId appId = ApplicationId.newInstance(10000, 1);
+      TaskExecutionTestHelpers.TezTaskUmbilicalForTest
+          umbilical = new TaskExecutionTestHelpers.TezTaskUmbilicalForTest();
+      TaskReporter taskReporter = createTaskReporter(appId, umbilical);
+
+      TezTaskRunner2 taskRunner = createTaskRunner(appId, umbilical, taskReporter, executor,
+          TestProcessor.CONF_EMPTY);
+      // Setup the executor
+      Future<TaskRunner2Result> taskRunnerFuture = taskExecutor.submit(
+          new TaskRunnerCallable2ForTest(taskRunner));
+      // Signal the processor to go through
+      TestProcessor.signal();
+      TaskRunner2Result result = taskRunnerFuture.get();
+      verifyTaskRunnerResult(result, EndReason.SUCCESS, null, false);
+      assertNull(taskReporter.currentCallable);
+      umbilical.verifyTaskSuccessEvent();
+      assertFalse(TestProcessor.wasAborted());
+      umbilical.resetTrackedEvents();
+
+      taskRunner = createTaskRunner(appId, umbilical, taskReporter, executor,
+          TestProcessor.CONF_EMPTY);
+      // Setup the executor
+      taskRunnerFuture = taskExecutor.submit(new TaskRunnerCallable2ForTest(taskRunner));
+      // Signal the processor to go through
+      TestProcessor.signal();
+      result = taskRunnerFuture.get();
+      verifyTaskRunnerResult(result, EndReason.SUCCESS, null, false);
+      assertNull(taskReporter.currentCallable);
+      umbilical.verifyTaskSuccessEvent();
+      assertFalse(TestProcessor.wasAborted());
+    } finally {
+      executor.shutdownNow();
+    }
+  }
+
+  // test task failed due to exception in Processor
+  @Test(timeout = 5000)
+  public void testFailedTaskTezException() throws IOException, InterruptedException, TezException,
+      ExecutionException {
+
+    ListeningExecutorService executor = null;
+    try {
+      ExecutorService rawExecutor = Executors.newFixedThreadPool(1);
+      executor = MoreExecutors.listeningDecorator(rawExecutor);
+      ApplicationId appId = ApplicationId.newInstance(10000, 1);
+      TaskExecutionTestHelpers.TezTaskUmbilicalForTest
+          umbilical = new TaskExecutionTestHelpers.TezTaskUmbilicalForTest();
+      TaskReporter taskReporter = createTaskReporter(appId, umbilical);
+
+      TezTaskRunner2 taskRunner = createTaskRunner(appId, umbilical, taskReporter, executor,
+          TestProcessor.CONF_THROW_TEZ_EXCEPTION);
+      // Setup the executor
+      Future<TaskRunner2Result> taskRunnerFuture =
+          taskExecutor.submit(new TaskRunnerCallable2ForTest(taskRunner));
+      // Signal the processor to go through
+      TestProcessor.awaitStart();
+      TestProcessor.signal();
+      TaskRunner2Result result = taskRunnerFuture.get();
+      verifyTaskRunnerResult(result, EndReason.TASK_ERROR, createProcessorTezException(), false);
+
+      assertNull(taskReporter.currentCallable);
+      umbilical.verifyTaskFailedEvent(
+          "Failure while running task",
+          TezException.class.getName() + ": " + TezException.class.getSimpleName());
+      // Failure detected as a result of fall off from the run method. abort isn't required.
+      assertFalse(TestProcessor.wasAborted());
+    } finally {
+      executor.shutdownNow();
+    }
+  }
+
+
+  // Test task failed due to Processor class not found
+  @Test(timeout = 5000)
+  public void testFailedTask2() throws IOException, InterruptedException, TezException,
+      ExecutionException {
+
+    ListeningExecutorService executor = null;
+    try {
+      ExecutorService rawExecutor = Executors.newFixedThreadPool(1);
+      executor = MoreExecutors.listeningDecorator(rawExecutor);
+      ApplicationId appId = ApplicationId.newInstance(10000, 1);
+      TaskExecutionTestHelpers.TezTaskUmbilicalForTest
+          umbilical = new TaskExecutionTestHelpers.TezTaskUmbilicalForTest();
+      TaskReporter taskReporter = createTaskReporter(appId, umbilical);
+
+      TezTaskRunner2 taskRunner = createTaskRunner(appId, umbilical, taskReporter, executor,
+          "NotExitedProcessor", TestProcessor.CONF_EMPTY, false);
+      // Setup the executor
+      Future<TaskRunner2Result> taskRunnerFuture =
+          taskExecutor.submit(new TaskRunnerCallable2ForTest(taskRunner));
+
+      TaskRunner2Result result = taskRunnerFuture.get();
+      verifyTaskRunnerResult(result, EndReason.TASK_ERROR,
+          new TezUncheckedException("Unchecked exception"), false);
+
+      assertNull(taskReporter.currentCallable);
+      umbilical.verifyTaskFailedEvent("Failure while running task",
+          ":org.apache.tez.dag.api.TezUncheckedException: "
+              + "Unable to load class: NotExitedProcessor");
+      // Failure detected as a result of fall off from the run method. abort isn't required.
+      assertFalse(TestProcessor.wasAborted());
+    } finally {
+      executor.shutdownNow();
+    }
+  }
+
+  // test task failed due to exception in Processor
+  @Test(timeout = 5000)
+  public void testFailedTaskIOException() throws IOException, InterruptedException, TezException,
+      ExecutionException {
+
+    ListeningExecutorService executor = null;
+    try {
+      ExecutorService rawExecutor = Executors.newFixedThreadPool(1);
+      executor = MoreExecutors.listeningDecorator(rawExecutor);
+      ApplicationId appId = ApplicationId.newInstance(10000, 1);
+      TaskExecutionTestHelpers.TezTaskUmbilicalForTest
+          umbilical = new TaskExecutionTestHelpers.TezTaskUmbilicalForTest();
+      TaskReporter taskReporter = createTaskReporter(appId, umbilical);
+
+      TezTaskRunner2 taskRunner = createTaskRunner(appId, umbilical, taskReporter, executor,
+          TestProcessor.CONF_THROW_IO_EXCEPTION);
+      // Setup the executor
+      Future<TaskRunner2Result> taskRunnerFuture =
+          taskExecutor.submit(new TaskRunnerCallable2ForTest(taskRunner));
+      // Signal the processor to go through
+      TestProcessor.awaitStart();
+      TestProcessor.signal();
+      TaskRunner2Result result = taskRunnerFuture.get();
+      verifyTaskRunnerResult(result, EndReason.TASK_ERROR, createProcessorIOException(), false);
+
+
+      assertNull(taskReporter.currentCallable);
+      umbilical.verifyTaskFailedEvent(
+          "Failure while running task",
+          IOException.class.getName() + ": " + IOException.class.getSimpleName());
+      // Failure detected as a result of fall off from the run method. abort isn't required.
+      assertFalse(TestProcessor.wasAborted());
+    } finally {
+      executor.shutdownNow();
+    }
+  }
+
+  @Test(timeout = 5000)
+  public void testHeartbeatException() throws IOException, InterruptedException, TezException,
+      ExecutionException {
+
+    ListeningExecutorService executor = null;
+    try {
+      ExecutorService rawExecutor = Executors.newFixedThreadPool(1);
+      executor = MoreExecutors.listeningDecorator(rawExecutor);
+      ApplicationId appId = ApplicationId.newInstance(10000, 1);
+      TaskExecutionTestHelpers.TezTaskUmbilicalForTest
+          umbilical = new TaskExecutionTestHelpers.TezTaskUmbilicalForTest();
+      TaskReporter taskReporter = createTaskReporter(appId, umbilical);
+
+      TezTaskRunner2 taskRunner = createTaskRunner(appId, umbilical, taskReporter, executor,
+          TestProcessor.CONF_EMPTY);
+      // Setup the executor
+      Future<TaskRunner2Result> taskRunnerFuture =
+          taskExecutor.submit(new TaskRunnerCallable2ForTest(taskRunner));
+      // Signal the processor to go through
+      TestProcessor.awaitStart();
+      umbilical.signalThrowException();
+      umbilical.awaitRegisteredEvent();
+      // Not signaling an actual start to verify task interruption
+
+      TaskRunner2Result result = taskRunnerFuture.get();
+      verifyTaskRunnerResult(result, EndReason.COMMUNICATION_FAILURE,
+          new IOException("IOException"),
+          TaskExecutionTestHelpers.HEARTBEAT_EXCEPTION_STRING, false);
+
+      TestProcessor.awaitCompletion();
+      assertTrue(TestProcessor.wasInterrupted());
+      assertNull(taskReporter.currentCallable);
+      // No completion events since umbilical communication already failed.
+      umbilical.verifyNoCompletionEvents();
+      assertTrue(TestProcessor.wasAborted());
+    } finally {
+      executor.shutdownNow();
+    }
+  }
+
+  @Test(timeout = 5000)
+  public void testHeartbeatShouldDie() throws IOException, InterruptedException, TezException,
+      ExecutionException {
+
+    ListeningExecutorService executor = null;
+    try {
+      ExecutorService rawExecutor = Executors.newFixedThreadPool(1);
+      executor = MoreExecutors.listeningDecorator(rawExecutor);
+      ApplicationId appId = ApplicationId.newInstance(10000, 1);
+      TaskExecutionTestHelpers.TezTaskUmbilicalForTest
+          umbilical = new TaskExecutionTestHelpers.TezTaskUmbilicalForTest();
+      TaskReporter taskReporter = createTaskReporter(appId, umbilical);
+
+      TezTaskRunner2 taskRunner = createTaskRunner(appId, umbilical, taskReporter, executor,
+          TestProcessor.CONF_EMPTY);
+      // Setup the executor
+      Future<TaskRunner2Result> taskRunnerFuture =
+          taskExecutor.submit(new TaskRunnerCallable2ForTest(taskRunner));
+      // Signal the processor to go through
+      TestProcessor.awaitStart();
+      umbilical.signalSendShouldDie();
+      umbilical.awaitRegisteredEvent();
+      // Not signaling an actual start to verify task interruption
+
+      TaskRunner2Result result = taskRunnerFuture.get();
+      verifyTaskRunnerResult(result, EndReason.CONTAINER_STOP_REQUESTED, null, true);
+
+
+      TestProcessor.awaitCompletion();
+      assertTrue(TestProcessor.wasInterrupted());
+      assertNull(taskReporter.currentCallable);
+      // TODO Is this statement correct ?
+      // No completion events since shouldDie was requested by the AM, which should have killed the
+      // task.
+      umbilical.verifyNoCompletionEvents();
+      assertTrue(TestProcessor.wasAborted());
+    } finally {
+      executor.shutdownNow();
+    }
+  }
+
+  @Test(timeout = 5000)
+  public void testSignalFatalErrorAndLoop() throws IOException, InterruptedException, TezException,
+      ExecutionException {
+
+    ListeningExecutorService executor = null;
+    try {
+      ExecutorService rawExecutor = Executors.newFixedThreadPool(1);
+      executor = MoreExecutors.listeningDecorator(rawExecutor);
+      ApplicationId appId = ApplicationId.newInstance(10000, 1);
+      TaskExecutionTestHelpers.TezTaskUmbilicalForTest
+          umbilical = new TaskExecutionTestHelpers.TezTaskUmbilicalForTest();
+      TaskReporter taskReporter = createTaskReporter(appId, umbilical);
+
+      TezTaskRunner2 taskRunner = createTaskRunner(appId, umbilical, taskReporter, executor,
+          TestProcessor.CONF_SIGNAL_FATAL_AND_LOOP);
+      // Setup the executor
+      Future<TaskRunner2Result> taskRunnerFuture =
+          taskExecutor.submit(new TaskRunnerCallable2ForTest(taskRunner));
+      // Signal the processor to go through
+      TestProcessor.awaitStart();
+      TestProcessor.signal();
+
+      TestProcessor.awaitLoop();
+      // The fatal error should have caused an interrupt.
+
+      TaskRunner2Result result = taskRunnerFuture.get();
+      verifyTaskRunnerResult(result, EndReason.TASK_ERROR, createProcessorIOException(), false);
+
+      TestProcessor.awaitCompletion();
+      assertTrue(TestProcessor.wasInterrupted());
+      assertNull(taskReporter.currentCallable);
+      umbilical.verifyTaskFailedEvent(
+          "Failure while running task",
+          IOException.class.getName() + ": " + IOException.class.getSimpleName());
+      // Signal fatal error should cause the processor to fail.
+      assertTrue(TestProcessor.wasAborted());
+    } finally {
+      executor.shutdownNow();
+    }
+  }
+
+  @Test(timeout = 5000)
+  public void testTaskKilled() throws IOException, InterruptedException, TezException,
+      ExecutionException {
+
+    ListeningExecutorService executor = null;
+    try {
+      ExecutorService rawExecutor = Executors.newFixedThreadPool(1);
+      executor = MoreExecutors.listeningDecorator(rawExecutor);
+      ApplicationId appId = ApplicationId.newInstance(10000, 1);
+      TaskExecutionTestHelpers.TezTaskUmbilicalForTest
+          umbilical = new TaskExecutionTestHelpers.TezTaskUmbilicalForTest();
+      TaskReporter taskReporter = createTaskReporter(appId, umbilical);
+
+      TezTaskRunner2 taskRunner = createTaskRunner(appId, umbilical, taskReporter, executor,
+          TestProcessor.CONF_EMPTY);
+      // Setup the executor
+      Future<TaskRunner2Result> taskRunnerFuture =
+          taskExecutor.submit(new TaskRunnerCallable2ForTest(taskRunner));
+      // Signal the processor to go through
+      TestProcessor.awaitStart();
+
+      taskRunner.killTask();
+
+      TaskRunner2Result result = taskRunnerFuture.get();
+      verifyTaskRunnerResult(result, EndReason.KILL_REQUESTED, null, false);
+
+      TestProcessor.awaitCompletion();
+      assertTrue(TestProcessor.wasInterrupted());
+      assertNull(taskReporter.currentCallable);
+      // Kill events are not sent over the umbilical at the moment.
+      umbilical.verifyNoCompletionEvents();
+    } finally {
+      executor.shutdownNow();
+    }
+  }
+
+  @Test(timeout = 5000)
+  public void testKilledAfterComplete() throws IOException, InterruptedException, TezException,
+      ExecutionException {
+
+    ListeningExecutorService executor = null;
+    try {
+      ExecutorService rawExecutor = Executors.newFixedThreadPool(1);
+      executor = MoreExecutors.listeningDecorator(rawExecutor);
+      ApplicationId appId = ApplicationId.newInstance(10000, 1);
+      TaskExecutionTestHelpers.TezTaskUmbilicalForTest
+          umbilical = new TaskExecutionTestHelpers.TezTaskUmbilicalForTest();
+      TaskReporter taskReporter = createTaskReporter(appId, umbilical);
+
+      TezTaskRunner2ForTest taskRunner =
+          createTaskRunnerForTest(appId, umbilical, taskReporter, executor,
+              TestProcessor.CONF_EMPTY);
+      // Setup the executor
+      Future<TaskRunner2Result> taskRunnerFuture =
+          taskExecutor.submit(new TaskRunnerCallable2ForTest(taskRunner));
+      // Signal the processor to go through
+      TestProcessor.awaitStart();
+      TestProcessor.signal();
+      TestProcessor.awaitCompletion();
+
+      taskRunner.awaitCallableCompletion();
+
+      taskRunner.killTask();
+      TaskRunner2Result result = taskRunnerFuture.get();
+      verifyTaskRunnerResult(result, EndReason.SUCCESS, null, false);
+
+      assertFalse(TestProcessor.wasInterrupted());
+      assertNull(taskReporter.currentCallable);
+      umbilical.verifyTaskSuccessEvent();
+    } finally {
+      executor.shutdownNow();
+    }
+  }
+
+
+  private void verifyTaskRunnerResult(TaskRunner2Result taskRunner2Result,
+                                      EndReason expectedEndReason, Throwable expectedThrowable,
+                                      boolean wasShutdownRequested) {
+    verifyTaskRunnerResult(taskRunner2Result, expectedEndReason, expectedThrowable, null,
+        wasShutdownRequested);
+  }
+
+  private void verifyTaskRunnerResult(TaskRunner2Result taskRunner2Result,
+                                      EndReason expectedEndReason, Throwable expectedThrowable,
+                                      String expectedExceptionMessage,
+                                      boolean wasShutdownRequested) {
+    assertEquals(expectedEndReason, taskRunner2Result.getEndReason());
+    if (expectedThrowable == null) {
+      assertNull(taskRunner2Result.getError());
+    } else {
+      assertNotNull(taskRunner2Result.getError());
+      Throwable cause = taskRunner2Result.getError();
+      LOG.info(cause.getClass().getName());
+      assertTrue(cause.getClass().isAssignableFrom(expectedThrowable.getClass()));
+
+      if (expectedExceptionMessage != null) {
+        assertTrue(cause.getMessage().contains(expectedExceptionMessage));
+      }
+
+    }
+    assertEquals(wasShutdownRequested, taskRunner2Result.isContainerShutdownRequested());
+  }
+
+
+  private static class TaskRunnerCallable2ForTest implements Callable<TaskRunner2Result> {
+    private final TezTaskRunner2 taskRunner;
+
+    public TaskRunnerCallable2ForTest(TezTaskRunner2 taskRunner) {
+      this.taskRunner = taskRunner;
+    }
+
+    @Override
+    public TaskRunner2Result call() throws Exception {
+      return taskRunner.run();
+    }
+  }
+
+  private TezTaskRunner2 createTaskRunner(ApplicationId appId,
+                                          TaskExecutionTestHelpers.TezTaskUmbilicalForTest umbilical,
+                                          TaskReporter taskReporter,
+                                          ListeningExecutorService executor, byte[] processorConf)
+      throws IOException {
+    return createTaskRunner(appId, umbilical, taskReporter, executor, TestProcessor.class.getName(),
+        processorConf, false);
+  }
+
+  private TezTaskRunner2ForTest createTaskRunnerForTest(ApplicationId appId,
+                                                        TaskExecutionTestHelpers.TezTaskUmbilicalForTest umbilical,
+                                                        TaskReporter taskReporter,
+                                                        ListeningExecutorService executor,
+                                                        byte[] processorConf)
+      throws IOException {
+    return (TezTaskRunner2ForTest) createTaskRunner(appId, umbilical, taskReporter, executor,
+        TestProcessor.class.getName(),
+        processorConf, true);
+  }
+
+  private TezTaskRunner2 createTaskRunner(ApplicationId appId,
+                                          TaskExecutionTestHelpers.TezTaskUmbilicalForTest umbilical,
+                                          TaskReporter taskReporter,
+                                          ListeningExecutorService executor, String processorClass,
+                                          byte[] processorConf, boolean testRunner) throws
+      IOException {
+    TezConfiguration tezConf = new TezConfiguration(defaultConf);
+    UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
+    Path testDir = new Path(workDir, UUID.randomUUID().toString());
+    String[] localDirs = new String[]{testDir.toString()};
+
+    TezDAGID dagId = TezDAGID.getInstance(appId, 1);
+    TezVertexID vertexId = TezVertexID.getInstance(dagId, 1);
+    TezTaskID taskId = TezTaskID.getInstance(vertexId, 1);
+    TezTaskAttemptID taskAttemptId = TezTaskAttemptID.getInstance(taskId, 1);
+    ProcessorDescriptor processorDescriptor = ProcessorDescriptor.create(processorClass)
+        .setUserPayload(UserPayload.create(ByteBuffer.wrap(processorConf)));
+    TaskSpec taskSpec =
+        new TaskSpec(taskAttemptId, "dagName", "vertexName", -1, processorDescriptor,
+            new ArrayList<InputSpec>(), new ArrayList<OutputSpec>(), null);
+
+    TezTaskRunner2 taskRunner;
+    if (testRunner) {
+      taskRunner = new TezTaskRunner2ForTest(tezConf, ugi, localDirs, taskSpec, 1,
+          new HashMap<String, ByteBuffer>(), new HashMap<String, String>(),
+          HashMultimap.<String, String>create(), taskReporter,
+          executor, null, "", new ExecutionContextImpl("localhost"),
+          Runtime.getRuntime().maxMemory());
+    } else {
+      taskRunner = new TezTaskRunner2(tezConf, ugi, localDirs, taskSpec, 1,
+          new HashMap<String, ByteBuffer>(), new HashMap<String, String>(),
+          HashMultimap.<String, String>create(), taskReporter,
+          executor, null, "", new ExecutionContextImpl("localhost"),
+          Runtime.getRuntime().maxMemory());
+    }
+
+    return taskRunner;
+  }
+
+  public static class TezTaskRunner2ForTest extends TezTaskRunner2 {
+
+    private final ReentrantLock testLock = new ReentrantLock();
+    private final Condition callableCompletionCondition = testLock.newCondition();
+
+    private final AtomicBoolean isCallableComplete = new AtomicBoolean(false);
+
+    public TezTaskRunner2ForTest(Configuration tezConf, UserGroupInformation ugi,
+                                 String[] localDirs,
+                                 TaskSpec taskSpec, int appAttemptNumber,
+                                 Map<String, ByteBuffer> serviceConsumerMetadata,
+                                 Map<String, String> serviceProviderEnvMap,
+                                 Multimap<String, String> startedInputsMap,
+                                 TaskReporterInterface taskReporter,
+                                 ListeningExecutorService executor,
+                                 ObjectRegistry objectRegistry,
+                                 String pid,
+                                 ExecutionContext executionContext,
+                                 long memAvailable) throws IOException {
+      super(tezConf, ugi, localDirs, taskSpec, appAttemptNumber, serviceConsumerMetadata,
+          serviceProviderEnvMap, startedInputsMap, taskReporter, executor, objectRegistry, pid,
+          executionContext, memAvailable);
+    }
+
+
+    @Override
+    @VisibleForTesting
+    void processCallableResult(TaskRunner2Callable.TaskRunner2CallableResult executionResult) {
+      testLock.lock();
+      try {
+        super.processCallableResult(executionResult);
+        isCallableComplete.set(true);
+        callableCompletionCondition.signal();
+      } finally {
+        testLock.unlock();
+      }
+    }
+
+    void awaitCallableCompletion() throws InterruptedException {
+      testLock.lock();
+      try {
+        while (!isCallableComplete.get()) {
+          callableCompletionCondition.await();
+        }
+      } finally {
+        testLock.unlock();
+      }
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/tez/blob/25dd3097/tez-runtime-internals/src/test/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/tez-runtime-internals/src/test/resources/log4j.properties b/tez-runtime-internals/src/test/resources/log4j.properties
new file mode 100644
index 0000000..531b68b
--- /dev/null
+++ b/tez-runtime-internals/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


[34/50] [abbrv] tez git commit: TEZ-2653. Change service contexts to expose a user specified payload instead of the AM configuration. (sseth)

Posted by ss...@apache.org.
TEZ-2653. Change service contexts to expose a user specified payload
instead of the AM configuration. (sseth)


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

Branch: refs/heads/TEZ-2003
Commit: 17feebc405eaa329c7525b007ad2e061f14ebe36
Parents: cfd625e
Author: Siddharth Seth <ss...@apache.org>
Authored: Tue Jul 28 14:56:56 2015 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Fri Aug 14 13:47:09 2015 -0700

----------------------------------------------------------------------
 TEZ-2003-CHANGES.txt                            |  1 +
 .../tez/dag/api/NamedEntityDescriptor.java      |  7 ++-
 .../api/ContainerLauncherContext.java           |  5 +-
 .../api/TaskSchedulerContext.java               |  5 +-
 .../tez/dag/api/TaskCommunicatorContext.java    |  4 +-
 .../dag/app/ContainerLauncherContextImpl.java   | 10 ++--
 .../org/apache/tez/dag/app/DAGAppMaster.java    | 20 +++++--
 .../dag/app/TaskAttemptListenerImpTezDag.java   | 19 ++++--
 .../dag/app/TaskCommunicatorContextImpl.java    | 12 ++--
 .../tez/dag/app/TezTaskCommunicatorImpl.java    | 12 +++-
 .../dag/app/launcher/ContainerLauncherImpl.java |  8 ++-
 .../app/launcher/ContainerLauncherRouter.java   | 24 +++++---
 .../app/launcher/LocalContainerLauncher.java    | 10 +++-
 .../dag/app/rm/LocalTaskSchedulerService.java   | 10 +++-
 .../dag/app/rm/TaskSchedulerContextImpl.java    | 12 ++--
 .../app/rm/TaskSchedulerContextImplWrapper.java |  6 +-
 .../dag/app/rm/TaskSchedulerEventHandler.java   | 61 ++++++++++++--------
 .../dag/app/rm/YarnTaskSchedulerService.java    | 21 +++++--
 .../apache/tez/dag/app/MockDAGAppMaster.java    | 15 ++++-
 .../app/TestTaskAttemptListenerImplTezDag.java  | 32 ++++++++--
 .../app/TestTaskAttemptListenerImplTezDag2.java | 12 +++-
 .../tez/dag/app/rm/TestContainerReuse.java      | 19 +++---
 .../tez/dag/app/rm/TestTaskScheduler.java       |  3 +-
 .../app/rm/TestTaskSchedulerEventHandler.java   | 18 +++++-
 .../dag/app/rm/TestTaskSchedulerHelpers.java    | 26 +++++++--
 .../TezTestServiceContainerLauncher.java        | 15 +++--
 .../rm/TezTestServiceTaskSchedulerService.java  | 10 +++-
 .../TezTestServiceTaskCommunicatorImpl.java     |  2 +-
 .../tez/tests/TestExternalTezServices.java      | 13 ++++-
 29 files changed, 294 insertions(+), 118 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/17feebc4/TEZ-2003-CHANGES.txt
----------------------------------------------------------------------
diff --git a/TEZ-2003-CHANGES.txt b/TEZ-2003-CHANGES.txt
index a201942..b88044b 100644
--- a/TEZ-2003-CHANGES.txt
+++ b/TEZ-2003-CHANGES.txt
@@ -38,5 +38,6 @@ ALL CHANGES:
   TEZ-2005. Define basic interface for pluggable TaskScheduler.
   TEZ-2651. Pluggable services should not extend AbstractService.
   TEZ-2652. Cleanup the way services are specified for an AM and vertices.
+  TEZ-2653. Change service contexts to expose a user specified payload instead of the AM configuration.
 
 INCOMPATIBLE CHANGES:

http://git-wip-us.apache.org/repos/asf/tez/blob/17feebc4/tez-api/src/main/java/org/apache/tez/dag/api/NamedEntityDescriptor.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/NamedEntityDescriptor.java b/tez-api/src/main/java/org/apache/tez/dag/api/NamedEntityDescriptor.java
index bad0d10..723d43f 100644
--- a/tez-api/src/main/java/org/apache/tez/dag/api/NamedEntityDescriptor.java
+++ b/tez-api/src/main/java/org/apache/tez/dag/api/NamedEntityDescriptor.java
@@ -17,7 +17,7 @@ package org.apache.tez.dag.api;
 import com.google.common.base.Preconditions;
 import org.apache.hadoop.classification.InterfaceAudience;
 
-public class NamedEntityDescriptor<T extends EntityDescriptor<T>> extends EntityDescriptor<T> {
+public class NamedEntityDescriptor<T extends NamedEntityDescriptor<T>> extends EntityDescriptor<NamedEntityDescriptor<T>>  {
   private final String entityName;
 
   @InterfaceAudience.Private
@@ -30,4 +30,9 @@ public class NamedEntityDescriptor<T extends EntityDescriptor<T>> extends Entity
   public String getEntityName() {
     return entityName;
   }
+
+  public T setUserPayload(UserPayload userPayload) {
+    super.setUserPayload(userPayload);
+    return (T) this;
+  }
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/17feebc4/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ContainerLauncherContext.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ContainerLauncherContext.java b/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ContainerLauncherContext.java
index 836dc4a..5da38b8 100644
--- a/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ContainerLauncherContext.java
+++ b/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ContainerLauncherContext.java
@@ -16,9 +16,9 @@ package org.apache.tez.serviceplugins.api;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.tez.dag.api.UserPayload;
 
 @InterfaceAudience.Public
 @InterfaceStability.Unstable
@@ -43,8 +43,7 @@ public interface ContainerLauncherContext {
 
   // Lookup APIs
 
-  // TODO TEZ-2003. To be replaced by getInitialPayload once the DAG API is changed.
-  Configuration getInitialConfiguration();
+  UserPayload getInitialUserPayload();
 
   int getNumNodes(String sourceName);
 

http://git-wip-us.apache.org/repos/asf/tez/blob/17feebc4/tez-api/src/main/java/org/apache/tez/serviceplugins/api/TaskSchedulerContext.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/serviceplugins/api/TaskSchedulerContext.java b/tez-api/src/main/java/org/apache/tez/serviceplugins/api/TaskSchedulerContext.java
index b2c8799..6f37641 100644
--- a/tez-api/src/main/java/org/apache/tez/serviceplugins/api/TaskSchedulerContext.java
+++ b/tez-api/src/main/java/org/apache/tez/serviceplugins/api/TaskSchedulerContext.java
@@ -20,7 +20,6 @@ import java.util.Map;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.Container;
@@ -30,6 +29,7 @@ import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
 import org.apache.hadoop.yarn.api.records.NodeReport;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.tez.common.ContainerSignatureMatcher;
+import org.apache.tez.dag.api.UserPayload;
 
 @InterfaceAudience.Public
 @InterfaceStability.Unstable
@@ -84,8 +84,7 @@ public interface TaskSchedulerContext {
 
   // Getters
 
-  // TODO TEZ-2003. To be replaced by getInitialPayload
-  public Configuration getInitialConfiguration();
+  public UserPayload getInitialUserPayload();
 
   public String getAppTrackingUrl();
 

http://git-wip-us.apache.org/repos/asf/tez/blob/17feebc4/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 ab32ec1..a1e94a3 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
@@ -18,7 +18,6 @@ import javax.annotation.Nullable;
 import java.io.IOException;
 import java.util.Set;
 
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
@@ -36,8 +35,7 @@ public interface TaskCommunicatorContext {
 
   // TODO TEZ-2003 Maybe add book-keeping as a helper library, instead of each impl tracking container to task etc.
 
-  // TODO TEZ-2003 To be replaced by getInitialPayload
-  Configuration getInitialConfiguration();
+  UserPayload getInitialUserPayload();
 
   ApplicationAttemptId getApplicationAttemptId();
   Credentials getCredentials();

http://git-wip-us.apache.org/repos/asf/tez/blob/17feebc4/tez-dag/src/main/java/org/apache/tez/dag/app/ContainerLauncherContextImpl.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/ContainerLauncherContextImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/ContainerLauncherContextImpl.java
index 997775a..92bbbdc 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/ContainerLauncherContextImpl.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/ContainerLauncherContextImpl.java
@@ -14,10 +14,10 @@
 
 package org.apache.tez.dag.app;
 
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.tez.common.TezUtilsInternal;
+import org.apache.tez.dag.api.UserPayload;
 import org.apache.tez.serviceplugins.api.ContainerLauncherContext;
 import org.apache.tez.serviceplugins.api.TaskAttemptEndReason;
 import org.apache.tez.dag.app.rm.container.AMContainerEvent;
@@ -33,10 +33,12 @@ public class ContainerLauncherContextImpl implements ContainerLauncherContext {
 
   private final AppContext context;
   private final TaskAttemptListener tal;
+  private final UserPayload initialUserPayload;
 
-  public ContainerLauncherContextImpl(AppContext appContext, TaskAttemptListener tal) {
+  public ContainerLauncherContextImpl(AppContext appContext, TaskAttemptListener tal, UserPayload initialUserPayload) {
     this.context = appContext;
     this.tal = tal;
+    this.initialUserPayload = initialUserPayload;
   }
 
   @Override
@@ -76,8 +78,8 @@ public class ContainerLauncherContextImpl implements ContainerLauncherContext {
   }
 
   @Override
-  public Configuration getInitialConfiguration() {
-    return context.getAMConf();
+  public UserPayload getInitialUserPayload() {
+    return initialUserPayload;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/tez/blob/17feebc4/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 8388cfb..4128841 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
@@ -62,8 +62,10 @@ import com.google.common.collect.HashBiMap;
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.cli.GnuParser;
 import org.apache.commons.cli.Options;
+import org.apache.tez.common.TezUtils;
 import org.apache.tez.dag.api.NamedEntityDescriptor;
 import org.apache.tez.dag.api.SessionNotRunning;
+import org.apache.tez.dag.api.UserPayload;
 import org.apache.tez.dag.api.records.DAGProtos.AMPluginDescriptorProto;
 import org.apache.tez.dag.api.records.DAGProtos.ConfigurationProto;
 import org.apache.tez.dag.api.records.DAGProtos.TezNamedEntityDescriptorProto;
@@ -492,9 +494,12 @@ public class DAGAppMaster extends AbstractService {
     jobTokenSecretManager.addTokenForJob(
         appAttemptID.getApplicationId().toString(), sessionToken);
 
+    UserPayload defaultPayload = TezUtils.createUserPayloadFromConf(amConf);
+
     //service to handle requests to TaskUmbilicalProtocol
     taskAttemptListener = createTaskAttemptListener(context,
-        taskHeartbeatHandler, containerHeartbeatHandler, taskCommunicatorDescriptors, isLocal);
+        taskHeartbeatHandler, containerHeartbeatHandler, taskCommunicatorDescriptors,
+        defaultPayload, isLocal);
     addIfService(taskAttemptListener, true);
 
     containerSignatureMatcher = createContainerSignatureMatcher();
@@ -540,9 +545,11 @@ public class DAGAppMaster extends AbstractService {
       }
     }
 
+
+
     this.taskSchedulerEventHandler = new TaskSchedulerEventHandler(context,
         clientRpcServer, dispatcher.getEventHandler(), containerSignatureMatcher, webUIService,
-        taskSchedulerDescriptors, isLocal);
+        taskSchedulerDescriptors, defaultPayload, isLocal);
     addIfService(taskSchedulerEventHandler, true);
 
     if (enableWebUIService()) {
@@ -560,7 +567,7 @@ public class DAGAppMaster extends AbstractService {
         taskSchedulerEventHandler);
     addIfServiceDependency(taskSchedulerEventHandler, clientRpcServer);
 
-    this.containerLauncherRouter = createContainerLauncherRouter(conf, containerLauncherDescriptors, isLocal);
+    this.containerLauncherRouter = createContainerLauncherRouter(defaultPayload, containerLauncherDescriptors, isLocal);
     addIfService(containerLauncherRouter, true);
     dispatcher.register(NMCommunicatorEventType.class, containerLauncherRouter);
 
@@ -1071,10 +1078,11 @@ public class DAGAppMaster extends AbstractService {
                                                           TaskHeartbeatHandler thh,
                                                           ContainerHeartbeatHandler chh,
                                                           List<NamedEntityDescriptor> entityDescriptors,
+                                                          UserPayload defaultUserPayload,
                                                           boolean isLocal) {
     TaskAttemptListener lis =
         new TaskAttemptListenerImpTezDag(context, thh, chh,
-            entityDescriptors, amConf, isLocal);
+            entityDescriptors, defaultUserPayload, isLocal);
     return lis;
   }
 
@@ -1095,11 +1103,11 @@ public class DAGAppMaster extends AbstractService {
     return chh;
   }
 
-  protected ContainerLauncherRouter createContainerLauncherRouter(Configuration conf,
+  protected ContainerLauncherRouter createContainerLauncherRouter(UserPayload defaultPayload,
                                                                   List<NamedEntityDescriptor> containerLauncherDescriptors,
                                                                   boolean isLocal) throws
       UnknownHostException {
-    return new ContainerLauncherRouter(conf, context, taskAttemptListener, workingDirectory,
+    return new ContainerLauncherRouter(defaultPayload, context, taskAttemptListener, workingDirectory,
         containerLauncherDescriptors, isLocal);
   }
 

http://git-wip-us.apache.org/repos/asf/tez/blob/17feebc4/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 1e34184..cc109a6 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
@@ -29,9 +29,9 @@ import java.util.concurrent.ConcurrentMap;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.Lists;
 import org.apache.commons.collections4.ListUtils;
-import org.apache.hadoop.conf.Configuration;
 import org.apache.tez.dag.api.NamedEntityDescriptor;
 import org.apache.tez.dag.api.TezConstants;
+import org.apache.tez.dag.api.UserPayload;
 import org.apache.tez.serviceplugins.api.ContainerEndReason;
 import org.apache.tez.dag.app.dag.event.TaskAttemptEvent;
 import org.apache.tez.dag.app.dag.event.TaskAttemptEventStatusUpdate;
@@ -103,7 +103,7 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
   public TaskAttemptListenerImpTezDag(AppContext context,
                                       TaskHeartbeatHandler thh, ContainerHeartbeatHandler chh,
                                       List<NamedEntityDescriptor> taskCommunicatorDescriptors,
-                                      Configuration conf,
+                                      UserPayload defaultUserPayload,
                                       boolean isPureLocalMode) {
     super(TaskAttemptListenerImpTezDag.class.getName());
     this.context = context;
@@ -112,17 +112,26 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
     if (taskCommunicatorDescriptors == null || taskCommunicatorDescriptors.isEmpty()) {
       if (isPureLocalMode) {
         taskCommunicatorDescriptors = Lists.newArrayList(new NamedEntityDescriptor(
-            TezConstants.getTezUberServicePluginName(), null));
+            TezConstants.getTezUberServicePluginName(), null).setUserPayload(defaultUserPayload));
       } else {
         taskCommunicatorDescriptors = Lists.newArrayList(new NamedEntityDescriptor(
-            TezConstants.getTezYarnServicePluginName(), null));
+            TezConstants.getTezYarnServicePluginName(), null).setUserPayload(defaultUserPayload));
       }
     }
     this.taskCommunicators = new TaskCommunicator[taskCommunicatorDescriptors.size()];
     this.taskCommunicatorContexts = new TaskCommunicatorContext[taskCommunicatorDescriptors.size()];
     this.taskCommunicatorServiceWrappers = new ServicePluginLifecycleAbstractService[taskCommunicatorDescriptors.size()];
     for (int i = 0 ; i < taskCommunicatorDescriptors.size() ; i++) {
-      taskCommunicatorContexts[i] = new TaskCommunicatorContextImpl(context, this, conf, i);
+      UserPayload userPayload;
+      if (taskCommunicatorDescriptors.get(i).getEntityName()
+          .equals(TezConstants.getTezYarnServicePluginName()) ||
+          taskCommunicatorDescriptors.get(i).getEntityName()
+              .equals(TezConstants.getTezUberServicePluginName())) {
+        userPayload = defaultUserPayload;
+      } else {
+        userPayload = taskCommunicatorDescriptors.get(i).getUserPayload();
+      }
+      taskCommunicatorContexts[i] = new TaskCommunicatorContextImpl(context, this, userPayload, i);
       taskCommunicators[i] = createTaskCommunicator(taskCommunicatorDescriptors.get(i), i);
       taskCommunicatorServiceWrappers[i] = new ServicePluginLifecycleAbstractService(taskCommunicators[i]);
     }

http://git-wip-us.apache.org/repos/asf/tez/blob/17feebc4/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 035db93..cc315b7 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
@@ -23,10 +23,10 @@ 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.conf.Configuration;
 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.UserPayload;
 import org.apache.tez.serviceplugins.api.TaskAttemptEndReason;
 import org.apache.tez.dag.api.TaskCommunicatorContext;
 import org.apache.tez.dag.api.TaskHeartbeatRequest;
@@ -49,17 +49,17 @@ public class TaskCommunicatorContextImpl implements TaskCommunicatorContext, Ver
   private final int taskCommunicatorIndex;
   private final ReentrantReadWriteLock.ReadLock dagChangedReadLock;
   private final ReentrantReadWriteLock.WriteLock dagChangedWriteLock;
-  private final Configuration conf;
+  private final UserPayload userPayload;
 
   private DAG dag;
 
   public TaskCommunicatorContextImpl(AppContext appContext,
                                      TaskAttemptListenerImpTezDag taskAttemptListener,
-                                     Configuration conf,
+                                     UserPayload userPayload,
                                      int taskCommunicatorIndex) {
     this.context = appContext;
     this.taskAttemptListener = taskAttemptListener;
-    this.conf = conf;
+    this.userPayload = userPayload;
     this.taskCommunicatorIndex = taskCommunicatorIndex;
 
     ReentrantReadWriteLock dagChangedLock = new ReentrantReadWriteLock();
@@ -68,8 +68,8 @@ public class TaskCommunicatorContextImpl implements TaskCommunicatorContext, Ver
   }
 
   @Override
-  public Configuration getInitialConfiguration() {
-    return conf;
+  public UserPayload getInitialUserPayload() {
+    return userPayload;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/tez/blob/17feebc4/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 93b5b43..2a5c80e 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
@@ -77,6 +77,7 @@ public class TezTaskCommunicatorImpl extends TaskCommunicator {
 
   protected final String tokenIdentifier;
   protected final Token<JobTokenIdentifier> sessionToken;
+  protected final Configuration conf;
   protected InetSocketAddress address;
 
   protected volatile Server server;
@@ -119,6 +120,12 @@ public class TezTaskCommunicatorImpl extends TaskCommunicator {
     this.taskUmbilical = new TezTaskUmbilicalProtocolImpl();
     this.tokenIdentifier = taskCommunicatorContext.getApplicationAttemptId().getApplicationId().toString();
     this.sessionToken = TokenCache.getSessionToken(taskCommunicatorContext.getCredentials());
+    try {
+      conf = TezUtils.createConfFromUserPayload(getContext().getInitialUserPayload());
+    } catch (IOException e) {
+      throw new TezUncheckedException(
+          "Unable to parse user payload for " + TezTaskCommunicatorImpl.class.getSimpleName(), e);
+    }
   }
 
   @Override
@@ -132,7 +139,6 @@ public class TezTaskCommunicatorImpl extends TaskCommunicator {
   }
 
   protected void startRpcServer() {
-    Configuration conf = getContext().getInitialConfiguration();
     try {
       JobTokenSecretManager jobTokenSecretManager =
           new JobTokenSecretManager();
@@ -171,6 +177,10 @@ public class TezTaskCommunicatorImpl extends TaskCommunicator {
     }
   }
 
+  protected Configuration getConf() {
+    return this.conf;
+  }
+
   private void refreshServiceAcls(Configuration configuration,
                                   PolicyProvider policyProvider) {
     this.server.refreshServiceAcl(configuration, policyProvider);

http://git-wip-us.apache.org/repos/asf/tez/blob/17feebc4/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 cba5c80..07d269d 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,6 +30,7 @@ import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 
 import org.apache.commons.lang.exception.ExceptionUtils;
+import org.apache.tez.common.TezUtils;
 import org.apache.tez.dag.api.TezConstants;
 import org.apache.tez.serviceplugins.api.ContainerLaunchRequest;
 import org.apache.tez.serviceplugins.api.ContainerLauncher;
@@ -224,7 +225,12 @@ public class ContainerLauncherImpl extends ContainerLauncher {
 
   public ContainerLauncherImpl(ContainerLauncherContext containerLauncherContext) {
     super(containerLauncherContext);
-    this.conf = new Configuration(containerLauncherContext.getInitialConfiguration());
+    try {
+      this.conf = TezUtils.createConfFromUserPayload(containerLauncherContext.getInitialUserPayload());
+    } catch (IOException e) {
+      throw new TezUncheckedException(
+          "Failed to parse user payload for " + ContainerLauncherImpl.class.getSimpleName(), e);
+    }
     conf.setInt(
         CommonConfigurationKeysPublic.IPC_CLIENT_CONNECTION_MAXIDLETIME_KEY,
         0);

http://git-wip-us.apache.org/repos/asf/tez/blob/17feebc4/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 594e6d3..2d56bfe 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.hadoop.yarn.event.EventHandler;
 import org.apache.tez.common.ReflectionUtils;
 import org.apache.tez.dag.api.NamedEntityDescriptor;
 import org.apache.tez.dag.api.TezConstants;
+import org.apache.tez.dag.api.UserPayload;
 import org.apache.tez.dag.app.ServicePluginLifecycleAbstractService;
 import org.apache.tez.serviceplugins.api.ContainerLaunchRequest;
 import org.apache.tez.serviceplugins.api.ContainerLauncher;
@@ -63,7 +64,7 @@ public class ContainerLauncherRouter extends AbstractService
   }
 
   // Accepting conf to setup final parameters, if required.
-  public ContainerLauncherRouter(Configuration conf, AppContext context,
+  public ContainerLauncherRouter(UserPayload defaultUserPayload, AppContext context,
                                  TaskAttemptListener taskAttemptListener,
                                  String workingDirectory,
                                  List<NamedEntityDescriptor> containerLauncherDescriptors,
@@ -74,10 +75,10 @@ public class ContainerLauncherRouter extends AbstractService
     if (containerLauncherDescriptors == null || containerLauncherDescriptors.isEmpty()) {
       if (isPureLocalMode) {
         containerLauncherDescriptors = Lists.newArrayList(new NamedEntityDescriptor(
-            TezConstants.getTezUberServicePluginName(), null));
+            TezConstants.getTezUberServicePluginName(), null).setUserPayload(defaultUserPayload));
       } else {
         containerLauncherDescriptors = Lists.newArrayList(new NamedEntityDescriptor(
-            TezConstants.getTezYarnServicePluginName(), null));
+            TezConstants.getTezYarnServicePluginName(), null).setUserPayload(defaultUserPayload));
       }
     }
     containerLauncherContexts = new ContainerLauncherContext[containerLauncherDescriptors.size()];
@@ -86,10 +87,20 @@ public class ContainerLauncherRouter extends AbstractService
 
 
     for (int i = 0; i < containerLauncherDescriptors.size(); i++) {
-      ContainerLauncherContext containerLauncherContext = new ContainerLauncherContextImpl(context, taskAttemptListener);
+      UserPayload userPayload;
+      if (containerLauncherDescriptors.get(i).getEntityName()
+          .equals(TezConstants.getTezYarnServicePluginName()) ||
+          containerLauncherDescriptors.get(i).getEntityName()
+              .equals(TezConstants.getTezUberServicePluginName())) {
+        userPayload = defaultUserPayload;
+      } else {
+        userPayload = containerLauncherDescriptors.get(i).getUserPayload();
+      }
+      ContainerLauncherContext containerLauncherContext =
+          new ContainerLauncherContextImpl(context, taskAttemptListener, userPayload);
       containerLauncherContexts[i] = containerLauncherContext;
       containerLaunchers[i] = createContainerLauncher(containerLauncherDescriptors.get(i), context,
-          containerLauncherContext, taskAttemptListener, workingDirectory, isPureLocalMode, conf);
+          containerLauncherContext, taskAttemptListener, workingDirectory, isPureLocalMode);
       containerLauncherServiceWrappers[i] = new ServicePluginLifecycleAbstractService(containerLaunchers[i]);
     }
   }
@@ -99,8 +110,7 @@ public class ContainerLauncherRouter extends AbstractService
                                                     ContainerLauncherContext containerLauncherContext,
                                                     TaskAttemptListener taskAttemptListener,
                                                     String workingDirectory,
-                                                    boolean isPureLocalMode,
-                                                    Configuration conf) throws
+                                                    boolean isPureLocalMode) throws
       UnknownHostException {
     if (containerLauncherDescriptor.getEntityName().equals(
         TezConstants.getTezYarnServicePluginName())) {

http://git-wip-us.apache.org/repos/asf/tez/blob/17feebc4/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 3975111..1d3e6df 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,6 +44,7 @@ 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.common.TezUtils;
 import org.apache.tez.serviceplugins.api.ContainerLaunchRequest;
 import org.apache.tez.serviceplugins.api.ContainerLauncher;
 import org.apache.tez.serviceplugins.api.ContainerLauncherContext;
@@ -129,7 +130,14 @@ public class LocalContainerLauncher extends ContainerLauncher {
         System.getenv(Environment.NM_HOST.name());
     executionContext = new ExecutionContextImpl(host);
 
-    numExecutors = getContext().getInitialConfiguration().getInt(TezConfiguration.TEZ_AM_INLINE_TASK_EXECUTION_MAX_TASKS,
+    Configuration conf;
+    try {
+      conf = TezUtils.createConfFromUserPayload(getContext().getInitialUserPayload());
+    } catch (IOException e) {
+      throw new TezUncheckedException(
+          "Failed to parse user payload for " + LocalContainerLauncher.class.getSimpleName(), e);
+    }
+    numExecutors = conf.getInt(TezConfiguration.TEZ_AM_INLINE_TASK_EXECUTION_MAX_TASKS,
         TezConfiguration.TEZ_AM_INLINE_TASK_EXECUTION_MAX_TASKS_DEFAULT);
     Preconditions.checkState(numExecutors >=1, "Must have at least 1 executor");
     ExecutorService rawExecutor = Executors.newFixedThreadPool(numExecutors,

http://git-wip-us.apache.org/repos/asf/tez/blob/17feebc4/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 476d00c..befde94 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
@@ -18,6 +18,7 @@
 
 package org.apache.tez.dag.app.rm;
 
+import java.io.IOException;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.PriorityBlockingQueue;
@@ -27,6 +28,7 @@ import java.util.LinkedHashMap;
 
 import com.google.common.primitives.Ints;
 
+import org.apache.tez.common.TezUtils;
 import org.apache.tez.serviceplugins.api.TaskScheduler;
 import org.apache.tez.serviceplugins.api.TaskSchedulerContext;
 import org.slf4j.Logger;
@@ -65,7 +67,13 @@ public class LocalTaskSchedulerService extends TaskScheduler {
     this.appTrackingUrl = taskSchedulerContext.getAppTrackingUrl();
     this.containerSignatureMatcher = taskSchedulerContext.getContainerSignatureMatcher();
     this.customContainerAppId = taskSchedulerContext.getCustomClusterIdentifier();
-    this.conf = taskSchedulerContext.getInitialConfiguration();
+    try {
+      this.conf = TezUtils.createConfFromUserPayload(taskSchedulerContext.getInitialUserPayload());
+    } catch (IOException e) {
+      throw new TezUncheckedException(
+          "Failed to deserialize payload for " + LocalTaskSchedulerService.class.getSimpleName(),
+          e);
+    }
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/tez/blob/17feebc4/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerContextImpl.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerContextImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerContextImpl.java
index 890870e..7f1d5a3 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerContextImpl.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerContextImpl.java
@@ -18,7 +18,6 @@ import java.nio.ByteBuffer;
 import java.util.List;
 import java.util.Map;
 
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.Container;
@@ -28,6 +27,7 @@ import org.apache.hadoop.yarn.api.records.NodeReport;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.tez.common.ContainerSignatureMatcher;
 import org.apache.tez.dag.api.TezUncheckedException;
+import org.apache.tez.dag.api.UserPayload;
 import org.apache.tez.dag.app.AppContext;
 import org.apache.tez.serviceplugins.api.TaskSchedulerContext;
 
@@ -40,12 +40,12 @@ public class TaskSchedulerContextImpl implements TaskSchedulerContext {
   private final long customClusterIdentifier;
   private final String appHostName;
   private final int clientPort;
-  private final Configuration conf;
+  private final UserPayload initialUserPayload;
 
   public TaskSchedulerContextImpl(TaskSchedulerEventHandler tseh, AppContext appContext,
                                   int schedulerId, String trackingUrl, long customClusterIdentifier,
                                   String appHostname, int clientPort,
-                                  Configuration conf) {
+                                  UserPayload initialUserPayload) {
     this.tseh = tseh;
     this.appContext = appContext;
     this.schedulerId = schedulerId;
@@ -53,7 +53,7 @@ public class TaskSchedulerContextImpl implements TaskSchedulerContext {
     this.customClusterIdentifier = customClusterIdentifier;
     this.appHostName = appHostname;
     this.clientPort = clientPort;
-    this.conf = conf;
+    this.initialUserPayload = initialUserPayload;
 
   }
 
@@ -110,8 +110,8 @@ public class TaskSchedulerContextImpl implements TaskSchedulerContext {
   }
 
   @Override
-  public Configuration getInitialConfiguration() {
-    return conf;
+  public UserPayload getInitialUserPayload() {
+    return initialUserPayload;
   }
 
 

http://git-wip-us.apache.org/repos/asf/tez/blob/17feebc4/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerContextImplWrapper.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerContextImplWrapper.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerContextImplWrapper.java
index e64ef43..9e4c8e0 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerContextImplWrapper.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerContextImplWrapper.java
@@ -27,7 +27,6 @@ import java.util.concurrent.Future;
 
 import com.google.common.annotations.VisibleForTesting;
 import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.Container;
@@ -37,6 +36,7 @@ import org.apache.hadoop.yarn.api.records.NodeReport;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.tez.common.ContainerSignatureMatcher;
 import org.apache.tez.dag.api.TezUncheckedException;
+import org.apache.tez.dag.api.UserPayload;
 import org.apache.tez.serviceplugins.api.TaskSchedulerContext;
 
 /**
@@ -132,8 +132,8 @@ class TaskSchedulerContextImplWrapper implements TaskSchedulerContext {
   // does not use locks.
 
   @Override
-  public Configuration getInitialConfiguration() {
-    return real.getInitialConfiguration();
+  public UserPayload getInitialUserPayload() {
+    return real.getInitialUserPayload();
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/tez/blob/17feebc4/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 67a088e..4c2e631 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
@@ -37,6 +37,7 @@ import com.google.common.util.concurrent.ThreadFactoryBuilder;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.tez.dag.api.NamedEntityDescriptor;
 import org.apache.tez.dag.api.TezConstants;
+import org.apache.tez.dag.api.UserPayload;
 import org.apache.tez.dag.app.ServicePluginLifecycleAbstractService;
 import org.apache.tez.serviceplugins.api.TaskScheduler;
 import org.apache.tez.serviceplugins.api.TaskSchedulerContext;
@@ -114,7 +115,7 @@ public class TaskSchedulerEventHandler extends AbstractService implements
   private AtomicBoolean shouldUnregisterFlag =
       new AtomicBoolean(false);
   private final WebUIService webUI;
-  private final String[] taskSchedulerClasses;
+  private final NamedEntityDescriptor[] taskSchedulerDescriptors;
   protected final TaskScheduler[]taskSchedulers;
   protected final ServicePluginLifecycleAbstractService []taskSchedulerServiceWrappers;
 
@@ -152,7 +153,8 @@ public class TaskSchedulerEventHandler extends AbstractService implements
   public TaskSchedulerEventHandler(AppContext appContext,
       DAGClientServer clientService, EventHandler eventHandler, 
       ContainerSignatureMatcher containerSignatureMatcher, WebUIService webUI,
-      List<NamedEntityDescriptor> schedulerDescriptors, boolean isPureLocalMode) {
+      List<NamedEntityDescriptor> schedulerDescriptors, UserPayload defaultPayload,
+                                   boolean isPureLocalMode) {
     super(TaskSchedulerEventHandler.class.getName());
     this.appContext = appContext;
     this.eventHandler = eventHandler;
@@ -168,39 +170,50 @@ public class TaskSchedulerEventHandler extends AbstractService implements
 
     // Override everything for pure local mode
     if (isPureLocalMode) {
-      this.taskSchedulerClasses = new String[] {TezConstants.getTezUberServicePluginName()};
+      this.taskSchedulerDescriptors = new NamedEntityDescriptor[]{
+          new NamedEntityDescriptor(TezConstants.getTezUberServicePluginName(), null)
+              .setUserPayload(defaultPayload)};
       this.yarnTaskSchedulerIndex = -1;
     } else {
       if (schedulerDescriptors == null || schedulerDescriptors.isEmpty()) {
-        this.taskSchedulerClasses = new String[] {TezConstants.getTezYarnServicePluginName()};
+        this.taskSchedulerDescriptors = new NamedEntityDescriptor[]{
+            new NamedEntityDescriptor(TezConstants.getTezYarnServicePluginName(), null)
+                .setUserPayload(defaultPayload)};
         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;
 
-        List<String> taskSchedulerClassList = new LinkedList<>();
+        List<NamedEntityDescriptor> schedulerDescriptorList = new LinkedList<>();
         for (int i = 0 ; i < schedulerDescriptors.size() ; i++) {
           if (schedulerDescriptors.get(i).getEntityName().equals(
               TezConstants.getTezYarnServicePluginName())) {
-            taskSchedulerClassList.add(schedulerDescriptors.get(i).getEntityName());
+            schedulerDescriptorList.add(
+                new NamedEntityDescriptor(schedulerDescriptors.get(i).getEntityName(), null)
+                    .setUserPayload(
+                        defaultPayload));
             foundYarnTaskSchedulerIndex = i;
           } else if (schedulerDescriptors.get(i).getEntityName().equals(
               TezConstants.getTezUberServicePluginName())) {
-            taskSchedulerClassList.add(schedulerDescriptors.get(i).getEntityName());
+            schedulerDescriptorList.add(
+                new NamedEntityDescriptor(schedulerDescriptors.get(i).getEntityName(), null)
+                    .setUserPayload(
+                        defaultPayload));
           } else {
-            taskSchedulerClassList.add(schedulerDescriptors.get(i).getClassName());
+            schedulerDescriptorList.add(schedulerDescriptors.get(i));
           }
         }
         if (foundYarnTaskSchedulerIndex == -1) {
-          taskSchedulerClassList.add(YarnTaskSchedulerService.class.getName());
-          foundYarnTaskSchedulerIndex = taskSchedulerClassList.size() -1;
+          schedulerDescriptorList.add(new NamedEntityDescriptor(TezConstants.getTezYarnServicePluginName(), null).setUserPayload(
+              defaultPayload));
+          foundYarnTaskSchedulerIndex = schedulerDescriptorList.size() -1;
         }
-        this.taskSchedulerClasses = taskSchedulerClassList.toArray(new String[taskSchedulerClassList.size()]);
+        this.taskSchedulerDescriptors = schedulerDescriptorList.toArray(new NamedEntityDescriptor[schedulerDescriptorList.size()]);
         this.yarnTaskSchedulerIndex = foundYarnTaskSchedulerIndex;
       }
     }
-    taskSchedulers = new TaskScheduler[this.taskSchedulerClasses.length];
-    taskSchedulerServiceWrappers = new ServicePluginLifecycleAbstractService[this.taskSchedulerClasses.length];
+    taskSchedulers = new TaskScheduler[this.taskSchedulerDescriptors.length];
+    taskSchedulerServiceWrappers = new ServicePluginLifecycleAbstractService[this.taskSchedulerDescriptors.length];
   }
 
   public Map<ApplicationAccessType, String> getApplicationAcls() {
@@ -417,23 +430,24 @@ public class TaskSchedulerEventHandler extends AbstractService implements
 
   private TaskScheduler createTaskScheduler(String host, int port, String trackingUrl,
                                                    AppContext appContext,
-                                                   String schedulerClassName,
+                                                   NamedEntityDescriptor taskSchedulerDescriptor,
                                                    long customAppIdIdentifier,
                                                    int schedulerId) {
     TaskSchedulerContext rawContext =
         new TaskSchedulerContextImpl(this, appContext, schedulerId, trackingUrl,
-            customAppIdIdentifier, host, port, getConfig());
+            customAppIdIdentifier, host, port, taskSchedulerDescriptor.getUserPayload());
     TaskSchedulerContext wrappedContext = new TaskSchedulerContextImplWrapper(rawContext, appCallbackExecutor);
-    if (schedulerClassName.equals(TezConstants.getTezYarnServicePluginName())) {
+    String schedulerName = taskSchedulerDescriptor.getEntityName();
+    if (schedulerName.equals(TezConstants.getTezYarnServicePluginName())) {
       LOG.info("Creating TaskScheduler: YarnTaskSchedulerService");
       return new YarnTaskSchedulerService(wrappedContext);
-    } else if (schedulerClassName.equals(TezConstants.getTezUberServicePluginName())) {
+    } else if (schedulerName.equals(TezConstants.getTezUberServicePluginName())) {
       LOG.info("Creating TaskScheduler: Local TaskScheduler");
       return new LocalTaskSchedulerService(wrappedContext);
     } else {
-      LOG.info("Creating custom TaskScheduler: " + schedulerClassName);
+      LOG.info("Creating custom TaskScheduler {}:{}", taskSchedulerDescriptor.getEntityName(), taskSchedulerDescriptor.getClassName());
       Class<? extends TaskScheduler> taskSchedulerClazz =
-          (Class<? extends TaskScheduler>) ReflectionUtils.getClazz(schedulerClassName);
+          (Class<? extends TaskScheduler>) ReflectionUtils.getClazz(taskSchedulerDescriptor.getClassName());
       try {
         Constructor<? extends TaskScheduler> ctor = taskSchedulerClazz
             .getConstructor(TaskSchedulerContext.class);
@@ -453,21 +467,20 @@ public class TaskSchedulerEventHandler extends AbstractService implements
 
   @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++) {
+    for (int i = 0; i < taskSchedulerDescriptors.length; i++) {
       long customAppIdIdentifier;
-      if (isPureLocalMode || taskSchedulerClasses[i].equals(
+      if (isPureLocalMode || taskSchedulerDescriptors[i].equals(
           TezConstants.getTezYarnServicePluginName())) { // Use the app identifier from the appId.
         customAppIdIdentifier = appContext.getApplicationID().getClusterTimestamp();
       } else {
         customAppIdIdentifier = SCHEDULER_APP_ID_BASE + (j++ * SCHEDULER_APP_ID_INCREMENT);
       }
-      LOG.info("ClusterIdentifier for TaskScheduler [" + i + ":" + taskSchedulerClasses[i] + "]=" +
+      LOG.info("ClusterIdentifier for TaskScheduler [" + i + ":" + taskSchedulerDescriptors[i].getEntityName() + "]=" +
           customAppIdIdentifier);
       taskSchedulers[i] = createTaskScheduler(host, port,
-          trackingUrl, appContext, taskSchedulerClasses[i], customAppIdIdentifier, i);
+          trackingUrl, appContext, taskSchedulerDescriptors[i], customAppIdIdentifier, i);
       taskSchedulerServiceWrappers[i] = new ServicePluginLifecycleAbstractService(taskSchedulers[i]);
     }
   }

http://git-wip-us.apache.org/repos/asf/tez/blob/17feebc4/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 1e76dc9..940c5b0 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
@@ -30,13 +30,11 @@ import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
 import java.util.concurrent.PriorityBlockingQueue;
-import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
 
+import org.apache.tez.common.TezUtils;
 import org.apache.tez.serviceplugins.api.TaskScheduler;
 import org.apache.tez.serviceplugins.api.TaskSchedulerContext;
 import org.apache.tez.serviceplugins.api.TaskSchedulerContext.AMState;
@@ -70,7 +68,6 @@ import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 import com.google.common.collect.Sets;
-import com.google.common.util.concurrent.ThreadFactoryBuilder;
 
 /* TODO not yet updating cluster nodes on every allocate response
  * from RMContainerRequestor
@@ -218,7 +215,13 @@ public class YarnTaskSchedulerService extends TaskScheduler
     this.appHostName = taskSchedulerContext.getAppHostName();
     this.appHostPort = taskSchedulerContext.getAppClientPort();
     this.appTrackingUrl = taskSchedulerContext.getAppTrackingUrl();
-    this.conf = taskSchedulerContext.getInitialConfiguration();
+    try {
+      this.conf = TezUtils.createConfFromUserPayload(taskSchedulerContext.getInitialUserPayload());
+    } catch (IOException e) {
+      throw new TezUncheckedException(
+          "Failed to deserialize payload for " + YarnTaskSchedulerService.class.getSimpleName(),
+          e);
+    }
   }
 
   @Private
@@ -231,7 +234,13 @@ public class YarnTaskSchedulerService extends TaskScheduler
     this.appHostName = taskSchedulerContext.getAppHostName();
     this.appHostPort = taskSchedulerContext.getAppClientPort();
     this.appTrackingUrl = taskSchedulerContext.getAppTrackingUrl();
-    this.conf = taskSchedulerContext.getInitialConfiguration();
+    try {
+      this.conf = TezUtils.createConfFromUserPayload(taskSchedulerContext.getInitialUserPayload());
+    } catch (IOException e) {
+      throw new TezUncheckedException(
+          "Failed to deserialize payload for " + YarnTaskSchedulerService.class.getSimpleName(),
+          e);
+    }
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/tez/blob/17feebc4/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 17feeaa..0723dbc 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
@@ -18,6 +18,7 @@
 
 package org.apache.tez.dag.app;
 
+import java.io.IOException;
 import java.lang.management.ManagementFactory;
 import java.lang.management.ThreadMXBean;
 import java.net.UnknownHostException;
@@ -34,7 +35,9 @@ import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicLong;
 
+import org.apache.tez.common.TezUtils;
 import org.apache.tez.dag.api.NamedEntityDescriptor;
+import org.apache.tez.dag.api.UserPayload;
 import org.apache.tez.serviceplugins.api.ContainerLaunchRequest;
 import org.apache.tez.serviceplugins.api.ContainerLauncher;
 import org.apache.tez.serviceplugins.api.ContainerLauncherContext;
@@ -488,7 +491,15 @@ public class MockDAGAppMaster extends DAGAppMaster {
     super(applicationAttemptId, containerId, nmHost, nmPort, nmHttpPort, clock, appSubmitTime,
         isSession, workingDirectory, localDirs, logDirs,  new TezApiVersionInfo().getVersion(), 1,
         credentials, jobUserName, null);
-    containerLauncherContext = new ContainerLauncherContextImpl(getContext(), getTaskAttemptListener());
+    Configuration conf = new Configuration(false);
+    UserPayload userPayload;
+    try {
+      userPayload = TezUtils.createUserPayloadFromConf(conf);
+    } catch (IOException e) {
+      throw new TezUncheckedException(e);
+    }
+    containerLauncherContext =
+        new ContainerLauncherContextImpl(getContext(), getTaskAttemptListener(), userPayload);
     containerLauncher = new MockContainerLauncher(launcherGoFlag, containerLauncherContext);
     shutdownHandler = new MockDAGAppMasterShutdownHandler();
     this.initFailFlag = initFailFlag;
@@ -500,7 +511,7 @@ public class MockDAGAppMaster extends DAGAppMaster {
 
   // use mock container launcher for tests
   @Override
-  protected ContainerLauncherRouter createContainerLauncherRouter(final Configuration conf,
+  protected ContainerLauncherRouter createContainerLauncherRouter(final UserPayload defaultUserPayload,
                                                                   List<NamedEntityDescriptor> containerLauncherDescirptors,
                                                                   boolean isLocal)
       throws UnknownHostException {

http://git-wip-us.apache.org/repos/asf/tez/blob/17feebc4/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 e45b0a2..1cb69a8 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
@@ -47,11 +47,14 @@ import org.apache.hadoop.yarn.event.Event;
 import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.tez.common.ContainerContext;
 import org.apache.tez.common.ContainerTask;
+import org.apache.tez.common.TezUtils;
 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.NamedEntityDescriptor;
 import org.apache.tez.dag.api.TezConfiguration;
+import org.apache.tez.dag.api.TezUncheckedException;
+import org.apache.tez.dag.api.UserPayload;
 import org.apache.tez.serviceplugins.api.ContainerEndReason;
 import org.apache.tez.serviceplugins.api.TaskAttemptEndReason;
 import org.apache.tez.dag.api.TaskHeartbeatRequest;
@@ -135,8 +138,15 @@ public class TestTaskAttemptListenerImplTezDag {
     doReturn(amContainer).when(amContainerMap).get(any(ContainerId.class));
     doReturn(container).when(amContainer).getContainer();
 
+    Configuration conf = new TezConfiguration();
+    UserPayload defaultPayload;
+    try {
+      defaultPayload = TezUtils.createUserPayloadFromConf(conf);
+    } catch (IOException e) {
+      throw new TezUncheckedException(e);
+    }
     taskAttemptListener = new TaskAttemptListenerImplForTest(appContext,
-        mock(TaskHeartbeatHandler.class), mock(ContainerHeartbeatHandler.class), null, null, false);
+        mock(TaskHeartbeatHandler.class), mock(ContainerHeartbeatHandler.class), null, defaultPayload, false);
     TezTaskCommunicatorImpl taskCommunicator = (TezTaskCommunicatorImpl)taskAttemptListener.getTaskCommunicator();
     TezTaskUmbilicalProtocol tezUmbilical = taskCommunicator.getUmbilical();
 
@@ -299,8 +309,14 @@ public class TestTaskAttemptListenerImplTezDag {
         new JobTokenSecretManager());
     sessionToken.setService(identifier.getJobId());
     TokenCache.setSessionToken(sessionToken, credentials);
+    UserPayload userPayload = null;
+    try {
+      userPayload = TezUtils.createUserPayloadFromConf(conf);
+    } catch (IOException e) {
+      throw new TezUncheckedException(e);
+    }
     taskAttemptListener = new TaskAttemptListenerImpTezDag(appContext,
-        mock(TaskHeartbeatHandler.class), mock(ContainerHeartbeatHandler.class), null, conf, false);
+        mock(TaskHeartbeatHandler.class), mock(ContainerHeartbeatHandler.class), null, userPayload, false);
     // no exception happen, should started properly
     taskAttemptListener.init(conf);
     taskAttemptListener.start();
@@ -319,8 +335,14 @@ public class TestTaskAttemptListenerImplTezDag {
       TokenCache.setSessionToken(sessionToken, credentials);
 
       conf.set(TezConfiguration.TEZ_AM_TASK_AM_PORT_RANGE, port + "-" + port);
+      UserPayload userPayload = null;
+      try {
+        userPayload = TezUtils.createUserPayloadFromConf(conf);
+      } catch (IOException e) {
+        throw new TezUncheckedException(e);
+      }
       taskAttemptListener = new TaskAttemptListenerImpTezDag(appContext,
-          mock(TaskHeartbeatHandler.class), mock(ContainerHeartbeatHandler.class), null, conf, false);
+          mock(TaskHeartbeatHandler.class), mock(ContainerHeartbeatHandler.class), null, userPayload, false);
       taskAttemptListener.init(conf);
       taskAttemptListener.start();
       int resultedPort = taskAttemptListener.getTaskCommunicator(0).getAddress().getPort();
@@ -377,9 +399,9 @@ public class TestTaskAttemptListenerImplTezDag {
                                           TaskHeartbeatHandler thh,
                                           ContainerHeartbeatHandler chh,
                                           List<NamedEntityDescriptor> taskCommDescriptors,
-                                          Configuration conf,
+                                          UserPayload userPayload,
                                           boolean isPureLocalMode) {
-      super(context, thh, chh, taskCommDescriptors, conf,
+      super(context, thh, chh, taskCommDescriptors, userPayload,
           isPureLocalMode);
     }
 

http://git-wip-us.apache.org/repos/asf/tez/blob/17feebc4/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
index 8d776fb..1c82bd8 100644
--- 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
@@ -22,9 +22,11 @@ import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.times;
 import static org.mockito.Mockito.verify;
 
+import java.io.IOException;
 import java.util.HashMap;
 import java.util.Map;
 
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
@@ -34,6 +36,10 @@ 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.common.TezUtils;
+import org.apache.tez.dag.api.TezConfiguration;
+import org.apache.tez.dag.api.TezUncheckedException;
+import org.apache.tez.dag.api.UserPayload;
 import org.apache.tez.serviceplugins.api.TaskAttemptEndReason;
 import org.apache.tez.dag.app.dag.DAG;
 import org.apache.tez.dag.app.dag.event.TaskAttemptEventAttemptFailed;
@@ -51,7 +57,7 @@ import org.mockito.ArgumentCaptor;
 public class TestTaskAttemptListenerImplTezDag2 {
 
   @Test(timeout = 5000)
-  public void testTaskAttemptFailedKilled() {
+  public void testTaskAttemptFailedKilled() throws IOException {
     ApplicationId appId = ApplicationId.newInstance(1000, 1);
     ApplicationAttemptId appAttemptId = ApplicationAttemptId.newInstance(appId, 1);
     Credentials credentials = new Credentials();
@@ -73,9 +79,11 @@ public class TestTaskAttemptListenerImplTezDag2 {
     doReturn(amContainer).when(amContainerMap).get(any(ContainerId.class));
     doReturn(container).when(amContainer).getContainer();
 
+    Configuration conf = new TezConfiguration();
+    UserPayload userPayload = TezUtils.createUserPayloadFromConf(conf);
     TaskAttemptListenerImpTezDag taskAttemptListener =
         new TaskAttemptListenerImpTezDag(appContext, mock(TaskHeartbeatHandler.class),
-            mock(ContainerHeartbeatHandler.class), null, null, false);
+            mock(ContainerHeartbeatHandler.class), null, userPayload, false);
 
     TaskSpec taskSpec1 = mock(TaskSpec.class);
     TezTaskAttemptID taskAttemptId1 = mock(TezTaskAttemptID.class);

http://git-wip-us.apache.org/repos/asf/tez/blob/17feebc4/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 88f6066..8e8224a 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
@@ -37,6 +37,7 @@ import java.util.Set;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.atomic.AtomicBoolean;
 
+import org.apache.tez.common.TezUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -144,7 +145,7 @@ public class TestContainerReuse {
     TaskSchedulerEventHandler taskSchedulerEventHandlerReal =
         new TaskSchedulerEventHandlerForTest(
           appContext, eventHandler, rmClient,
-          new AlwaysMatchesContainerMatcher());
+          new AlwaysMatchesContainerMatcher(), TezUtils.createUserPayloadFromConf(conf));
     TaskSchedulerEventHandler taskSchedulerEventHandler =
         spy(taskSchedulerEventHandlerReal);
     taskSchedulerEventHandler.init(conf);
@@ -279,7 +280,7 @@ public class TestContainerReuse {
 
     TaskSchedulerEventHandler taskSchedulerEventHandlerReal =
       new TaskSchedulerEventHandlerForTest(appContext, eventHandler, rmClient,
-        new AlwaysMatchesContainerMatcher());
+        new AlwaysMatchesContainerMatcher(), TezUtils.createUserPayloadFromConf(conf));
     TaskSchedulerEventHandler taskSchedulerEventHandler =
       spy(taskSchedulerEventHandlerReal);
     taskSchedulerEventHandler.init(conf);
@@ -378,7 +379,7 @@ public class TestContainerReuse {
     doReturn(dagID).when(appContext).getCurrentDAGID();
     doReturn(mock(ClusterInfo.class)).when(appContext).getClusterInfo();
 
-    TaskSchedulerEventHandler taskSchedulerEventHandlerReal = new TaskSchedulerEventHandlerForTest(appContext, eventHandler, rmClient, new AlwaysMatchesContainerMatcher());
+    TaskSchedulerEventHandler taskSchedulerEventHandlerReal = new TaskSchedulerEventHandlerForTest(appContext, eventHandler, rmClient, new AlwaysMatchesContainerMatcher(), TezUtils.createUserPayloadFromConf(tezConf));
     TaskSchedulerEventHandler taskSchedulerEventHandler = spy(taskSchedulerEventHandlerReal);
     taskSchedulerEventHandler.init(tezConf);
     taskSchedulerEventHandler.start();
@@ -514,7 +515,7 @@ public class TestContainerReuse {
 
     //Use ContainerContextMatcher here.  Otherwise it would not match the JVM options
     TaskSchedulerEventHandler taskSchedulerEventHandlerReal =
-        new TaskSchedulerEventHandlerForTest(appContext, eventHandler, rmClient, new ContainerContextMatcher());
+        new TaskSchedulerEventHandlerForTest(appContext, eventHandler, rmClient, new ContainerContextMatcher(), TezUtils.createUserPayloadFromConf(tezConf));
     TaskSchedulerEventHandler taskSchedulerEventHandler = spy(taskSchedulerEventHandlerReal);
     taskSchedulerEventHandler.init(tezConf);
     taskSchedulerEventHandler.start();
@@ -709,7 +710,7 @@ public class TestContainerReuse {
     TaskSchedulerEventHandler taskSchedulerEventHandlerReal =
         new TaskSchedulerEventHandlerForTest(
           appContext, eventHandler, rmClient,
-          new AlwaysMatchesContainerMatcher());
+          new AlwaysMatchesContainerMatcher(), TezUtils.createUserPayloadFromConf(tezConf));
     TaskSchedulerEventHandler taskSchedulerEventHandler =
         spy(taskSchedulerEventHandlerReal);
     taskSchedulerEventHandler.init(tezConf);
@@ -833,7 +834,7 @@ public class TestContainerReuse {
 
     TaskSchedulerEventHandler taskSchedulerEventHandlerReal =
       new TaskSchedulerEventHandlerForTest(appContext, eventHandler, rmClient,
-        new AlwaysMatchesContainerMatcher());
+        new AlwaysMatchesContainerMatcher(), TezUtils.createUserPayloadFromConf(tezConf));
     TaskSchedulerEventHandler taskSchedulerEventHandler =
       spy(taskSchedulerEventHandlerReal);
     taskSchedulerEventHandler.init(tezConf);
@@ -947,7 +948,7 @@ public class TestContainerReuse {
     doAnswer(dagIDAnswer).when(appContext).getCurrentDAGID();
     doReturn(mock(ClusterInfo.class)).when(appContext).getClusterInfo();
     
-    TaskSchedulerEventHandler taskSchedulerEventHandlerReal = new TaskSchedulerEventHandlerForTest(appContext, eventHandler, rmClient, new AlwaysMatchesContainerMatcher());
+    TaskSchedulerEventHandler taskSchedulerEventHandlerReal = new TaskSchedulerEventHandlerForTest(appContext, eventHandler, rmClient, new AlwaysMatchesContainerMatcher(), TezUtils.createUserPayloadFromConf(tezConf));
     TaskSchedulerEventHandler taskSchedulerEventHandler = spy(taskSchedulerEventHandlerReal);
     taskSchedulerEventHandler.init(tezConf);
     taskSchedulerEventHandler.start();
@@ -1105,7 +1106,7 @@ public class TestContainerReuse {
 
     TaskSchedulerEventHandler taskSchedulerEventHandlerReal =
         new TaskSchedulerEventHandlerForTest(appContext, eventHandler, rmClient,
-            new ContainerContextMatcher());
+            new ContainerContextMatcher(), TezUtils.createUserPayloadFromConf(tezConf));
     TaskSchedulerEventHandler taskSchedulerEventHandler = spy(taskSchedulerEventHandlerReal);
     taskSchedulerEventHandler.init(tezConf);
     taskSchedulerEventHandler.start();
@@ -1259,7 +1260,7 @@ public class TestContainerReuse {
 
     TaskSchedulerEventHandler taskSchedulerEventHandlerReal =
         new TaskSchedulerEventHandlerForTest(appContext, eventHandler, rmClient,
-            new AlwaysMatchesContainerMatcher());
+            new AlwaysMatchesContainerMatcher(), TezUtils.createUserPayloadFromConf(tezConf));
     TaskSchedulerEventHandler taskSchedulerEventHandler = spy(taskSchedulerEventHandlerReal);
     taskSchedulerEventHandler.init(tezConf);
     taskSchedulerEventHandler.start();

http://git-wip-us.apache.org/repos/asf/tez/blob/17feebc4/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 123a4d7..6af9815 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
@@ -145,7 +145,8 @@ public class TestTaskScheduler {
 
     scheduler.initialize();
     drainableAppCallback.drain();
-    verify(mockRMClient).init(conf);
+    // Verifying the validity of the configuration via the interval only instead of making sure
+    // it's the same instance.
     verify(mockRMClient).setHeartbeatInterval(interval);
 
     RegisterApplicationMasterResponse mockRegResponse =

http://git-wip-us.apache.org/repos/asf/tez/blob/17feebc4/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 f191175..f8aa1e2 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
@@ -49,9 +49,12 @@ 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.ContainerSignatureMatcher;
+import org.apache.tez.common.TezUtils;
 import org.apache.tez.dag.api.NamedEntityDescriptor;
 import org.apache.tez.dag.api.TaskLocationHint;
 import org.apache.tez.dag.api.TezConfiguration;
+import org.apache.tez.dag.api.TezUncheckedException;
+import org.apache.tez.dag.api.UserPayload;
 import org.apache.tez.dag.api.client.DAGClientServer;
 import org.apache.tez.dag.app.AppContext;
 import org.apache.tez.dag.app.ContainerContext;
@@ -92,8 +95,10 @@ public class TestTaskSchedulerEventHandler {
     
     public MockTaskSchedulerEventHandler(AppContext appContext,
         DAGClientServer clientService, EventHandler eventHandler,
-        ContainerSignatureMatcher containerSignatureMatcher, WebUIService webUI) {
-      super(appContext, clientService, eventHandler, containerSignatureMatcher, webUI, new LinkedList<NamedEntityDescriptor>(), false);
+        ContainerSignatureMatcher containerSignatureMatcher, WebUIService webUI,
+        UserPayload defaultPayload) {
+      super(appContext, clientService, eventHandler, containerSignatureMatcher, webUI,
+          new LinkedList<NamedEntityDescriptor>(), defaultPayload, false);
     }
 
     @Override
@@ -134,8 +139,15 @@ public class TestTaskSchedulerEventHandler {
     mockWebUIService = mock(WebUIService.class);
     when(mockAppContext.getAllContainers()).thenReturn(mockAMContainerMap);
     when(mockClientService.getBindAddress()).thenReturn(new InetSocketAddress(10000));
+    Configuration conf = new Configuration(false);
+    UserPayload userPayload;
+    try {
+      userPayload = TezUtils.createUserPayloadFromConf(conf);
+    } catch (IOException e) {
+      throw new TezUncheckedException(e);
+    }
     schedulerHandler = new MockTaskSchedulerEventHandler(
-        mockAppContext, mockClientService, mockEventHandler, mockSigMatcher, mockWebUIService);
+        mockAppContext, mockClientService, mockEventHandler, mockSigMatcher, mockWebUIService, userPayload);
   }
 
   @Test(timeout = 5000)

http://git-wip-us.apache.org/repos/asf/tez/blob/17feebc4/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 60d37e9..59ab00a 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
@@ -24,6 +24,7 @@ import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.spy;
 import static org.mockito.Mockito.when;
 
+import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.Collection;
 import java.util.LinkedList;
@@ -59,7 +60,10 @@ import org.apache.hadoop.yarn.client.api.impl.AMRMClientImpl;
 import org.apache.hadoop.yarn.event.Event;
 import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.tez.common.ContainerSignatureMatcher;
+import org.apache.tez.common.TezUtils;
 import org.apache.tez.dag.api.NamedEntityDescriptor;
+import org.apache.tez.dag.api.TezUncheckedException;
+import org.apache.tez.dag.api.UserPayload;
 import org.apache.tez.dag.app.AppContext;
 import org.apache.tez.dag.app.ServicePluginLifecycleAbstractService;
 import org.apache.tez.dag.app.rm.YarnTaskSchedulerService.CookieContainerRequest;
@@ -125,22 +129,26 @@ class TestTaskSchedulerHelpers {
 
     private TezAMRMClientAsync<CookieContainerRequest> amrmClientAsync;
     private ContainerSignatureMatcher containerSignatureMatcher;
+    private UserPayload defaultPayload;
 
     @SuppressWarnings("rawtypes")
     public TaskSchedulerEventHandlerForTest(AppContext appContext,
         EventHandler eventHandler,
         TezAMRMClientAsync<CookieContainerRequest> amrmClientAsync,
-        ContainerSignatureMatcher containerSignatureMatcher) {
-      super(appContext, null, eventHandler, containerSignatureMatcher, null, new LinkedList<NamedEntityDescriptor>(), false);
+        ContainerSignatureMatcher containerSignatureMatcher,
+        UserPayload defaultPayload) {
+      super(appContext, null, eventHandler, containerSignatureMatcher, null,
+          new LinkedList<NamedEntityDescriptor>(), defaultPayload, false);
       this.amrmClientAsync = amrmClientAsync;
       this.containerSignatureMatcher = containerSignatureMatcher;
+      this.defaultPayload = defaultPayload;
     }
 
     @Override
     public void instantiateScheduelrs(String host, int port, String trackingUrl, AppContext appContext) {
       TaskSchedulerContext taskSchedulerContext =
           new TaskSchedulerContextImpl(this, appContext, 0, trackingUrl, 1000, host, port,
-              getConfig());
+              defaultPayload);
       TaskSchedulerContextImplWrapper wrapper =
           new TaskSchedulerContextImplWrapper(taskSchedulerContext,
               new CountingExecutorService(appCallbackExecutor));
@@ -287,8 +295,8 @@ class TestTaskSchedulerHelpers {
     // Not incrementing invocations for methods which to not obtain locks,
     // and do not go via the executor service.
     @Override
-    public Configuration getInitialConfiguration() {
-      return real.getInitialConfiguration();
+    public UserPayload getInitialUserPayload() {
+      return real.getInitialUserPayload();
     }
 
     @Override
@@ -523,7 +531,13 @@ class TestTaskSchedulerHelpers {
     when(mockContext.getAppTrackingUrl()).thenReturn(appUrl);
 
     when(mockContext.getAMState()).thenReturn(TaskSchedulerContext.AMState.RUNNING_APP);
-    when(mockContext.getInitialConfiguration()).thenReturn(conf);
+    UserPayload userPayload;
+    try {
+      userPayload = TezUtils.createUserPayloadFromConf(conf);
+    } catch (IOException e) {
+      throw new TezUncheckedException(e);
+    }
+    when(mockContext.getInitialUserPayload()).thenReturn(userPayload);
     when(mockContext.isSession()).thenReturn(isSession);
     if (containerSignatureMatcher != null) {
       when(mockContext.getContainerSignatureMatcher())

http://git-wip-us.apache.org/repos/asf/tez/blob/17feebc4/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 0002b42..f31a07b 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
@@ -19,10 +19,12 @@ import java.net.InetSocketAddress;
 
 import com.google.common.base.Preconditions;
 import com.google.protobuf.ByteString;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.util.StringUtils;
 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.tez.common.TezUtils;
 import org.apache.tez.serviceplugins.api.ContainerLaunchRequest;
 import org.apache.tez.serviceplugins.api.ContainerLauncher;
 import org.apache.tez.serviceplugins.api.ContainerLauncherContext;
@@ -49,17 +51,22 @@ public class TezTestServiceContainerLauncher extends ContainerLauncher {
   private final int servicePort;
   private final TezTestServiceCommunicator communicator;
   private final ApplicationAttemptId appAttemptId;
-  //  private final TaskAttemptListener tal;
+  private final Configuration conf;
 
 
   // Configuration passed in here to set up final parameters
   public TezTestServiceContainerLauncher(ContainerLauncherContext containerLauncherContext) {
     super(containerLauncherContext);
-    int numThreads = getContext().getInitialConfiguration().getInt(
+    try {
+      conf = TezUtils.createConfFromUserPayload(getContext().getInitialUserPayload());
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+    int numThreads = conf.getInt(
         TezTestServiceConfConstants.TEZ_TEST_SERVICE_AM_COMMUNICATOR_NUM_THREADS,
         TezTestServiceConfConstants.TEZ_TEST_SERVICE_AM_COMMUNICATOR_NUM_THREADS_DEFAULT);
 
-    this.servicePort = getContext().getInitialConfiguration().getInt(
+    this.servicePort = conf.getInt(
         TezTestServiceConfConstants.TEZ_TEST_SERVICE_RPC_PORT, -1);
     Preconditions.checkArgument(servicePort > 0,
         TezTestServiceConfConstants.TEZ_TEST_SERVICE_RPC_PORT + " must be set");
@@ -70,7 +77,7 @@ public class TezTestServiceContainerLauncher extends ContainerLauncher {
 
   @Override
   public void start() {
-    communicator.init(getContext().getInitialConfiguration());
+    communicator.init(conf);
     communicator.start();
   }
 

http://git-wip-us.apache.org/repos/asf/tez/blob/17feebc4/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 7d209bc..0d87995 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,6 +14,7 @@
 
 package org.apache.tez.dag.app.rm;
 
+import java.io.IOException;
 import java.util.Arrays;
 import java.util.LinkedList;
 import java.util.List;
@@ -32,6 +33,8 @@ 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.common.TezUtils;
+import org.apache.tez.dag.api.TezUncheckedException;
 import org.apache.tez.serviceplugins.api.TaskAttemptEndReason;
 import org.apache.tez.service.TezTestServiceConfConstants;
 import org.apache.tez.serviceplugins.api.TaskScheduler;
@@ -74,7 +77,12 @@ public class TezTestServiceTaskSchedulerService extends TaskScheduler {
     this.containerFactory = new ContainerFactory(taskSchedulerContext.getApplicationAttemptId(),
         taskSchedulerContext.getCustomClusterIdentifier());
 
-    Configuration conf = taskSchedulerContext.getInitialConfiguration();
+    Configuration conf = null;
+    try {
+      conf = TezUtils.createConfFromUserPayload(taskSchedulerContext.getInitialUserPayload());
+    } catch (IOException e) {
+      throw new TezUncheckedException(e);
+    }
     this.memoryPerInstance = conf
         .getInt(TezTestServiceConfConstants.TEZ_TEST_SERVICE_MEMORY_PER_INSTANCE_MB, -1);
     Preconditions.checkArgument(memoryPerInstance > 0,

http://git-wip-us.apache.org/repos/asf/tez/blob/17feebc4/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 078ea79..ef8f9e4 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
@@ -76,7 +76,7 @@ public class TezTestServiceTaskCommunicatorImpl extends TezTaskCommunicatorImpl
   @Override
   public void initialize() throws Exception {
     super.initialize();
-    this.communicator.init(getContext().getInitialConfiguration());
+    this.communicator.init(getConf());
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/tez/blob/17feebc4/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 07dd363..2c52ae3 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
@@ -24,10 +24,12 @@ 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.common.TezUtils;
 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.TezException;
+import org.apache.tez.dag.api.UserPayload;
 import org.apache.tez.dag.api.Vertex;
 import org.apache.tez.dag.api.Vertex.VertexExecutionContext;
 import org.apache.tez.dag.api.client.DAGClient;
@@ -136,17 +138,22 @@ public class TestExternalTezServices {
     confForJobs.set(TezConfiguration.TEZ_AM_STAGING_DIR, stagingDirPath.toString());
     confForJobs.setBoolean(TezRuntimeConfiguration.TEZ_RUNTIME_OPTIMIZE_LOCAL_FETCH, false);
 
+    UserPayload userPayload = TezUtils.createUserPayloadFromConf(confForJobs);
+
     TaskSchedulerDescriptor[] taskSchedulerDescriptors = new TaskSchedulerDescriptor[]{
         TaskSchedulerDescriptor
-            .create(EXT_PUSH_ENTITY_NAME, TezTestServiceTaskSchedulerService.class.getName())};
+            .create(EXT_PUSH_ENTITY_NAME, TezTestServiceTaskSchedulerService.class.getName())
+            .setUserPayload(userPayload)};
 
     ContainerLauncherDescriptor[] containerLauncherDescriptors = new ContainerLauncherDescriptor[]{
         ContainerLauncherDescriptor
-            .create(EXT_PUSH_ENTITY_NAME, TezTestServiceNoOpContainerLauncher.class.getName())};
+            .create(EXT_PUSH_ENTITY_NAME, TezTestServiceNoOpContainerLauncher.class.getName())
+            .setUserPayload(userPayload)};
 
     TaskCommunicatorDescriptor[] taskCommunicatorDescriptors = new TaskCommunicatorDescriptor[]{
         TaskCommunicatorDescriptor
-            .create(EXT_PUSH_ENTITY_NAME, TezTestServiceTaskCommunicatorImpl.class.getName())};
+            .create(EXT_PUSH_ENTITY_NAME, TezTestServiceTaskCommunicatorImpl.class.getName())
+            .setUserPayload(userPayload)};
 
     ServicePluginsDescriptor servicePluginsDescriptor = ServicePluginsDescriptor.create(true, true,
         taskSchedulerDescriptors, containerLauncherDescriptors, taskCommunicatorDescriptors);


[06/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/2fc431d9
Tree: http://git-wip-us.apache.org/repos/asf/tez/tree/2fc431d9
Diff: http://git-wip-us.apache.org/repos/asf/tez/diff/2fc431d9

Branch: refs/heads/TEZ-2003
Commit: 2fc431d90919f3840fde1bf2786c21da9983bd01
Parents: 619aaf3
Author: Siddharth Seth <ss...@apache.org>
Authored: Wed May 6 00:39:46 2015 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Fri Aug 14 13:46:43 2015 -0700

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


http://git-wip-us.apache.org/repos/asf/tez/blob/2fc431d9/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/2fc431d9/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 84e5e0d..8263b3f 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,6 +20,9 @@ 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.text.SimpleDateFormat;
 import java.util.ArrayList;
@@ -41,6 +44,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;
@@ -174,6 +178,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;
@@ -420,6 +427,7 @@ public class LogicalIOProcessorRuntimeTask extends RuntimeTask {
           taskSpec.getTaskAttemptID());
       initializedInputs.put(edgeName, input);
       LOG.info("Initialized Input with src edge: " + edgeName);
+      initializedInputs.put(edgeName, input);
       return null;
     }
   }
@@ -469,6 +477,7 @@ public class LogicalIOProcessorRuntimeTask extends RuntimeTask {
           outputContext.getDestinationVertexName(), taskSpec.getTaskAttemptID());
       initializedOutputs.put(edgeName, output);
       LOG.info("Initialized Output with dest edge: " + edgeName);
+      initializedOutputs.put(edgeName, output);
       return null;
     }
   }
@@ -694,6 +703,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() {
@@ -713,6 +729,7 @@ public class LogicalIOProcessorRuntimeTask extends RuntimeTask {
             if (!isTaskDone()) {
               LOG.warn("Event Router thread interrupted. Returning.");
             }
+            Thread.currentThread().interrupt();
             return;
           }
         }
@@ -724,6 +741,12 @@ public class LogicalIOProcessorRuntimeTask extends RuntimeTask {
     eventRouterThread.start();
   }
 
+  private void maybeResetInterruptStatus() {
+    if (!Thread.currentThread().isInterrupted()) {
+      Thread.currentThread().interrupt();
+    }
+  }
+
   private void closeContexts() throws IOException {
     closeContext(inputContextMap);
     closeContext(outputContextMap);
@@ -763,6 +786,18 @@ public class LogicalIOProcessorRuntimeTask extends RuntimeTask {
     }
 
     // Close the unclosed IPO
+    /**
+     * 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());
@@ -773,10 +808,16 @@ public class LogicalIOProcessorRuntimeTask extends RuntimeTask {
       try {
         processorClosed = true;
         processor.close();
-        LOG.info("Closed processor for vertex={}, index={}",
+        LOG.info("Closed processor for vertex={}, index={}, interruptedStatus={}",
             processor
                 .getContext().getTaskVertexName(),
-            processor.getContext().getTaskVertexIndex());
+            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(
             "Ignoring Exception when closing processor(cleanup). Exception class={}, message={}",
@@ -792,13 +833,19 @@ public class LogicalIOProcessorRuntimeTask extends RuntimeTask {
       inputIterator.remove();
       try {
         ((InputFrameworkInterface)entry.getValue()).close();
+        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(
             "Ignoring exception when closing input {}(cleanup). Exception class={}, message={}",
             srcVertexName, e.getClass().getName(), e.getMessage());
       } finally {
-        LOG.info("Close input for vertex={}, sourceVertex={}", processor
-            .getContext().getTaskVertexName(), srcVertexName);
+        LOG.info("Close input for vertex={}, sourceVertex={}, interruptedStatus={}", processor
+            .getContext().getTaskVertexName(), srcVertexName, Thread.currentThread().isInterrupted());
       }
     }
 
@@ -810,16 +857,26 @@ public class LogicalIOProcessorRuntimeTask extends RuntimeTask {
       outputIterator.remove();
       try {
         ((OutputFrameworkInterface) entry.getValue()).close();
+        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(
             "Ignoring exception when closing output {}(cleanup). Exception class={}, message={}",
             destVertexName, e.getClass().getName(), e.getMessage());
       } finally {
-        LOG.info("Close input for vertex={}, sourceVertex={}", processor
-            .getContext().getTaskVertexName(), destVertexName);
+        LOG.info("Close input for vertex={}, sourceVertex={}, interruptedStatus={}", processor
+            .getContext().getTaskVertexName(), destVertexName, Thread.currentThread().isInterrupted());
       }
     }
 
+    if (LOG.isDebugEnabled()) {
+      printThreads();
+    }
+
     try {
       closeContexts();
       // Cleanup references which may be held by misbehaved tasks.
@@ -867,6 +924,20 @@ public class LogicalIOProcessorRuntimeTask extends RuntimeTask {
     inputReadyTracker = null;
     objectRegistry = null;
   }
+
+
+  /**
+   * 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/2fc431d9/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 17d7053..cdfb46a 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
@@ -76,6 +76,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);
@@ -163,4 +167,5 @@ public abstract class RuntimeTask {
     taskDone.set(true);
   }
 
+  public abstract void abortTask() throws Exception;
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/2fc431d9/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) {


[39/50] [abbrv] tez git commit: TEZ-2703. TEZ-2003 build fails (zjffdu)

Posted by ss...@apache.org.
TEZ-2703. TEZ-2003 build fails (zjffdu)


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

Branch: refs/heads/TEZ-2003
Commit: a960c646d361914dbeb36c896318a7eab78c924f
Parents: efea3a5
Author: Jeff Zhang <zj...@apache.org>
Authored: Mon Aug 10 13:38:19 2015 +0800
Committer: Siddharth Seth <ss...@apache.org>
Committed: Fri Aug 14 13:47:10 2015 -0700

----------------------------------------------------------------------
 tez-tools/analyzers/job-analyzer/pom.xml | 2 +-
 tez-tools/analyzers/pom.xml              | 2 +-
 2 files changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/a960c646/tez-tools/analyzers/job-analyzer/pom.xml
----------------------------------------------------------------------
diff --git a/tez-tools/analyzers/job-analyzer/pom.xml b/tez-tools/analyzers/job-analyzer/pom.xml
index 36b12fe..40da1f3 100644
--- a/tez-tools/analyzers/job-analyzer/pom.xml
+++ b/tez-tools/analyzers/job-analyzer/pom.xml
@@ -20,7 +20,7 @@
   <parent>
     <groupId>org.apache.tez</groupId>
     <artifactId>tez-perf-analyzer</artifactId>
-    <version>0.8.0-SNAPSHOT</version>
+    <version>0.8.0-TEZ-2003-SNAPSHOT</version>
   </parent>
   <artifactId>tez-job-analyzer</artifactId>
 

http://git-wip-us.apache.org/repos/asf/tez/blob/a960c646/tez-tools/analyzers/pom.xml
----------------------------------------------------------------------
diff --git a/tez-tools/analyzers/pom.xml b/tez-tools/analyzers/pom.xml
index 97ce541..ee2c163 100644
--- a/tez-tools/analyzers/pom.xml
+++ b/tez-tools/analyzers/pom.xml
@@ -20,7 +20,7 @@
   <parent>
     <groupId>org.apache.tez</groupId>
     <artifactId>tez-tools</artifactId>
-    <version>0.8.0-SNAPSHOT</version>
+    <version>0.8.0-TEZ-2003-SNAPSHOT</version>
   </parent>
   <artifactId>tez-perf-analyzer</artifactId>
   <packaging>pom</packaging>


[28/50] [abbrv] tez git commit: TEZ-2004. Define basic interface for pluggable ContainerLaunchers. (sseth)

Posted by ss...@apache.org.
http://git-wip-us.apache.org/repos/asf/tez/blob/af1cc723/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 07dfcd6..25fd13e 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,7 +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;
+import org.apache.tez.serviceplugins.api.TaskAttemptEndReason;
 
 public abstract class TaskSchedulerService extends AbstractService{
 

http://git-wip-us.apache.org/repos/asf/tez/blob/af1cc723/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 6f897e1..d4cf317 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,7 +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.serviceplugins.api.TaskAttemptEndReason;
 import org.apache.tez.dag.api.TezConfiguration;
 import org.apache.tez.dag.api.TezUncheckedException;
 import org.apache.tez.dag.app.AppContext;

http://git-wip-us.apache.org/repos/asf/tez/blob/af1cc723/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerEventCompleted.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerEventCompleted.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerEventCompleted.java
index 8ef2a83..cecb019 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerEventCompleted.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerEventCompleted.java
@@ -20,7 +20,7 @@ package org.apache.tez.dag.app.rm.container;
 
 import org.apache.hadoop.yarn.api.records.ContainerExitStatus;
 import org.apache.hadoop.yarn.api.records.ContainerId;
-import org.apache.tez.dag.api.ContainerEndReason;
+import org.apache.tez.serviceplugins.api.ContainerEndReason;
 import org.apache.tez.dag.records.TaskAttemptTerminationCause;
 
 public class AMContainerEventCompleted extends AMContainerEvent {

http://git-wip-us.apache.org/repos/asf/tez/blob/af1cc723/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 7446734..5cff766 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
@@ -28,8 +28,8 @@ import java.util.concurrent.locks.ReentrantReadWriteLock.ReadLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock.WriteLock;
 
 import org.apache.tez.common.TezUtilsInternal;
-import org.apache.tez.dag.api.ContainerEndReason;
-import org.apache.tez.dag.api.TaskAttemptEndReason;
+import org.apache.tez.serviceplugins.api.ContainerEndReason;
+import org.apache.tez.serviceplugins.api.TaskAttemptEndReason;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.security.Credentials;
@@ -38,7 +38,6 @@ 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;
@@ -1079,12 +1078,12 @@ public class AMContainerImpl implements AMContainer {
   }
 
   protected void sendStartRequestToNM(ContainerLaunchContext clc) {
-    sendEvent(new NMCommunicatorLaunchRequestEvent(clc, container, launcherId, taskCommId));
+    sendEvent(new NMCommunicatorLaunchRequestEvent(clc, container, launcherId, schedulerId, taskCommId));
   }
 
   protected void sendStopRequestToNM() {
     sendEvent(new NMCommunicatorStopRequestEvent(containerId,
-        container.getNodeId(), container.getContainerToken(), launcherId));
+        container.getNodeId(), container.getContainerToken(), launcherId, schedulerId, taskCommId));
   }
 
   protected void unregisterAttemptFromListener(TezTaskAttemptID attemptId, TaskAttemptEndReason endReason) {

http://git-wip-us.apache.org/repos/asf/tez/blob/af1cc723/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 0f35bba..3c3c6a7 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,11 +34,14 @@ import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicLong;
 
+import org.apache.tez.serviceplugins.api.ContainerLaunchRequest;
+import org.apache.tez.serviceplugins.api.ContainerLauncher;
+import org.apache.tez.serviceplugins.api.ContainerLauncherContext;
+import org.apache.tez.serviceplugins.api.ContainerStopRequest;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 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;
 import org.apache.hadoop.yarn.api.records.ContainerExitStatus;
 import org.apache.hadoop.yarn.api.records.ContainerId;
@@ -53,23 +56,14 @@ 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.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;
-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.HistoryEventHandler;
 import org.apache.tez.dag.records.TezTaskAttemptID;
 import org.apache.tez.dag.records.TezTaskID;
-import org.apache.tez.runtime.api.events.CompositeDataMovementEvent;
-import org.apache.tez.runtime.api.events.DataMovementEvent;
 import org.apache.tez.runtime.api.events.TaskAttemptCompletedEvent;
 import org.apache.tez.runtime.api.events.TaskStatusUpdateEvent;
 import org.apache.tez.runtime.api.impl.EventMetaData;
-import org.apache.tez.runtime.api.impl.OutputSpec;
 import org.apache.tez.runtime.api.impl.TaskSpec;
 import org.apache.tez.runtime.api.impl.TaskStatistics;
 import org.apache.tez.runtime.api.impl.TezEvent;
@@ -89,6 +83,7 @@ import com.google.common.util.concurrent.ThreadFactoryBuilder;
 public class MockDAGAppMaster extends DAGAppMaster {
   
   private static final Logger LOG = LoggerFactory.getLogger(MockDAGAppMaster.class);
+  ContainerLauncherContext containerLauncherContext;
   MockContainerLauncher containerLauncher;
   boolean initFailFlag;
   boolean startFailFlag;
@@ -121,7 +116,7 @@ public class MockDAGAppMaster extends DAGAppMaster {
   // Upon, launch of a container is simulates the container asking for tasks
   // Upon receiving a task it simulates completion of the tasks
   // It can be used to preempt the container for a given task
-  public class MockContainerLauncher extends AbstractService implements ContainerLauncher, Runnable {
+  public class MockContainerLauncher extends ContainerLauncher implements Runnable {
 
     BlockingQueue<NMCommunicatorEvent> eventQueue = new LinkedBlockingQueue<NMCommunicatorEvent>();
     Thread eventHandlingThread;
@@ -141,12 +136,14 @@ public class MockDAGAppMaster extends DAGAppMaster {
     Map<TezTaskID, Integer> preemptedTasks = Maps.newConcurrentMap();
     
     Map<TezTaskAttemptID, Integer> tasksWithStatusUpdates = Maps.newConcurrentMap();
-    
-    public MockContainerLauncher(AtomicBoolean goFlag) {
-      super("MockContainerLauncher");
+
+    public MockContainerLauncher(AtomicBoolean goFlag,
+                                 ContainerLauncherContext containerLauncherContext) {
+      super("MockContainerLauncher", containerLauncherContext);
       this.goFlag = goFlag;
     }
 
+
     public class ContainerData {
       ContainerId cId;
       TezTaskAttemptID taId;
@@ -211,20 +208,18 @@ public class MockDAGAppMaster extends DAGAppMaster {
         executorService.shutdownNow();
       }
     }
-    
+
+
     @Override
-    public void handle(NMCommunicatorEvent event) {
-      switch (event.getType()) {
-      case CONTAINER_LAUNCH_REQUEST:
-        launch((NMCommunicatorLaunchRequestEvent) event);
-        break;
-      case CONTAINER_STOP_REQUEST:
-        stop((NMCommunicatorStopRequestEvent)event);
-        break;
-      }
+    public void launchContainer(ContainerLaunchRequest launchRequest) {
+      launch(launchRequest);
     }
-    
-    
+
+    @Override
+    public void stopContainer(ContainerStopRequest stopRequest) {
+      stop(stopRequest);
+    }
+
     void waitToGo() {
       if (goFlag == null) {
         return;
@@ -266,20 +261,19 @@ public class MockDAGAppMaster extends DAGAppMaster {
       tasksWithStatusUpdates.put(tId, numUpdates);
     }
     
-    void stop(NMCommunicatorStopRequestEvent event) {
+    void stop(ContainerStopRequest event) {
       // remove from simulated container list
       containers.remove(event.getContainerId());
-      getContext().getEventHandler().handle(
-          new AMContainerEvent(event.getContainerId(), AMContainerEventType.C_NM_STOP_SENT));
+      getContext().containerStopRequested(event.getContainerId());
     }
 
-    void launch(NMCommunicatorLaunchRequestEvent event) {
+    void launch(ContainerLaunchRequest event) {
       // launch container by putting it in simulated container list
       ContainerData cData = new ContainerData(event.getContainerId(),
           event.getContainerLaunchContext());
       containers.put(event.getContainerId(), cData);
       containersToProcess.add(cData);
-      getContext().getEventHandler().handle(new AMContainerEventLaunched(event.getContainerId()));      
+      getContext().containerLaunched(event.getContainerId());
     }
     
     public void waitTillContainersLaunched() throws InterruptedException {
@@ -289,7 +283,7 @@ public class MockDAGAppMaster extends DAGAppMaster {
     }
     
     void incrementTime(long inc) {
-      Clock clock = getContext().getClock();
+      Clock clock = MockDAGAppMaster.this.getContext().getClock();
       if (clock instanceof MockClock) {
         ((MockClock) clock).incrementTime(inc);
       }
@@ -493,7 +487,8 @@ public class MockDAGAppMaster extends DAGAppMaster {
     super(applicationAttemptId, containerId, nmHost, nmPort, nmHttpPort, clock, appSubmitTime,
         isSession, workingDirectory, localDirs, logDirs,  new TezApiVersionInfo().getVersion(), 1,
         credentials, jobUserName);
-    containerLauncher = new MockContainerLauncher(launcherGoFlag);
+    containerLauncherContext = new ContainerLauncherContextImpl(getContext(), getTaskAttemptListener());
+    containerLauncher = new MockContainerLauncher(launcherGoFlag, containerLauncherContext);
     shutdownHandler = new MockDAGAppMasterShutdownHandler();
     this.initFailFlag = initFailFlag;
     this.startFailFlag = startFailFlag;
@@ -508,7 +503,7 @@ public class MockDAGAppMaster extends DAGAppMaster {
                                                                   String[] containerLaunchers,
                                                                   boolean isLocal)
       throws UnknownHostException {
-    return new ContainerLauncherRouter(containerLauncher);
+    return new ContainerLauncherRouter(containerLauncher, getContext());
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/tez/blob/af1cc723/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 7f0362d..df643e4 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
@@ -51,8 +51,8 @@ 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.TezConfiguration;
-import org.apache.tez.dag.api.ContainerEndReason;
-import org.apache.tez.dag.api.TaskAttemptEndReason;
+import org.apache.tez.serviceplugins.api.ContainerEndReason;
+import org.apache.tez.serviceplugins.api.TaskAttemptEndReason;
 import org.apache.tez.dag.api.TaskHeartbeatRequest;
 import org.apache.tez.dag.api.TaskHeartbeatResponse;
 import org.apache.tez.dag.api.TezException;

http://git-wip-us.apache.org/repos/asf/tez/blob/af1cc723/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
index 934543f..8d776fb 100644
--- 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
@@ -34,7 +34,7 @@ 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.serviceplugins.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;

http://git-wip-us.apache.org/repos/asf/tez/blob/af1cc723/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 62edac9..e37ab4a 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
@@ -53,12 +53,11 @@ 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.conf.YarnConfiguration;
-import org.apache.hadoop.yarn.util.RackResolver;
 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.serviceplugins.api.TaskAttemptEndReason;
 import org.apache.tez.dag.api.TezConfiguration;
 import org.apache.tez.dag.api.TaskLocationHint;
 import org.apache.tez.dag.api.oldrecords.TaskAttemptState;
@@ -94,7 +93,6 @@ 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;
 

http://git-wip-us.apache.org/repos/asf/tez/blob/af1cc723/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 0a642bb..b555c62 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,7 +27,6 @@ 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;

http://git-wip-us.apache.org/repos/asf/tez/blob/af1cc723/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 322eabc..f9952d8 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,8 +63,8 @@ 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.ContainerEndReason;
-import org.apache.tez.dag.api.TaskAttemptEndReason;
+import org.apache.tez.serviceplugins.api.ContainerEndReason;
+import org.apache.tez.serviceplugins.api.TaskAttemptEndReason;
 import org.apache.tez.dag.api.TaskCommunicator;
 import org.apache.tez.dag.app.AppContext;
 import org.apache.tez.dag.app.ContainerHeartbeatHandler;

http://git-wip-us.apache.org/repos/asf/tez/blob/af1cc723/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 9d22196..dbf5054 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,34 +14,30 @@
 
 package org.apache.tez.dag.app.launcher;
 
+import java.io.IOException;
 import java.net.InetSocketAddress;
 
 import com.google.common.base.Preconditions;
 import com.google.protobuf.ByteString;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.service.AbstractService;
+import org.apache.hadoop.util.StringUtils;
 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;
-import org.apache.tez.dag.app.TaskAttemptListener;
+import org.apache.tez.serviceplugins.api.ContainerLaunchRequest;
+import org.apache.tez.serviceplugins.api.ContainerLauncher;
+import org.apache.tez.serviceplugins.api.ContainerLauncherContext;
+import org.apache.tez.serviceplugins.api.ContainerStopRequest;
+import org.apache.tez.dag.api.TezConstants;
 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;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-public class TezTestServiceContainerLauncher extends AbstractService implements ContainerLauncher {
+// TODO TEZ-2003 look for all LOG.*(DBG and LOG.*(DEBUG messages
+
+public class TezTestServiceContainerLauncher extends ContainerLauncher {
 
   // TODO Support interruptability of tasks which haven't yet been launched.
 
@@ -49,40 +45,32 @@ public class TezTestServiceContainerLauncher extends AbstractService implements
 
   static final Logger LOG = LoggerFactory.getLogger(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;
   private final ApplicationAttemptId appAttemptId;
+  //  private final TaskAttemptListener tal;
 
 
   // 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,
+  public TezTestServiceContainerLauncher(ContainerLauncherContext containerLauncherContext) {
+    super(TezTestServiceContainerLauncher.class.getName(), containerLauncherContext);
+    int numThreads = getContext().getInitialConfiguration().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);
+    this.servicePort = getContext().getInitialConfiguration().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.appAttemptId = appContext.getApplicationAttemptId();
-    this.tal = tal;
-  }
-
-  @Override
-  public void serviceInit(Configuration conf) {
-    communicator.init(conf);
+    this.tokenIdentifier = getContext().getApplicationAttemptId().getApplicationId().toString();
+    this.appAttemptId = getContext().getApplicationAttemptId();
   }
 
   @Override
   public void serviceStart() {
+    communicator.init(getContext().getInitialConfiguration());
     communicator.start();
   }
 
@@ -92,51 +80,56 @@ public class TezTestServiceContainerLauncher extends AbstractService implements
   }
 
   @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;
+  public void launchContainer(final ContainerLaunchRequest launchRequest) {
+    RunContainerRequestProto runRequest = null;
+    try {
+      runRequest = constructRunContainerRequest(launchRequest);
+    } catch (IOException e) {
+      getContext().containerLaunchFailed(launchRequest.getContainerId(),
+          "Failed to construct launch request, " + StringUtils.stringifyException(e));
+      return;
     }
+    communicator.runContainer(runRequest, launchRequest.getNodeId().getHost(),
+        launchRequest.getNodeId().getPort(),
+        new TezTestServiceCommunicator.ExecuteRequestCallback<TezTestServiceProtocolProtos.RunContainerResponseProto>() {
+          @Override
+          public void setResponse(TezTestServiceProtocolProtos.RunContainerResponseProto response) {
+            LOG.info(
+                "Container: " + launchRequest.getContainerId() + " launch succeeded on host: " +
+                    launchRequest.getNodeId());
+            getContext().containerLaunched(launchRequest.getContainerId());
+          }
+
+          @Override
+          public void indicateError(Throwable t) {
+            LOG.error(
+                "Failed to launch container: " + launchRequest.getContainerId() + " on host: " +
+                    launchRequest.getNodeId(), t);
+            sendContainerLaunchFailedMsg(launchRequest.getContainerId(), t);
+          }
+        });
   }
 
-  private RunContainerRequestProto constructRunContainerRequest(NMCommunicatorLaunchRequestEvent event) {
+  @Override
+  public void stopContainer(ContainerStopRequest stopRequest) {
+    LOG.info("DEBUG: Ignoring STOP_REQUEST for event: " + stopRequest);
+    // that the container is actually done (normally received from RM)
+    // TODO Sending this out for an un-launched container is invalid
+    getContext().containerStopRequested(stopRequest.getContainerId());
+  }
+
+  private RunContainerRequestProto constructRunContainerRequest(ContainerLaunchRequest launchRequest) throws
+      IOException {
     RunContainerRequestProto.Builder builder = RunContainerRequestProto.newBuilder();
-    InetSocketAddress address = tal.getTaskCommunicator(event.getTaskCommId()).getAddress();
+    Preconditions.checkArgument(launchRequest.getTaskCommunicatorName().equals(TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT));
+    InetSocketAddress address = (InetSocketAddress) getContext().getTaskCommunicatorMetaInfo(launchRequest.getTaskCommunicatorName());
     builder.setAmHost(address.getHostName()).setAmPort(address.getPort());
     builder.setAppAttemptNumber(appAttemptId.getAttemptId());
     builder.setApplicationIdString(appAttemptId.getApplicationId().toString());
     builder.setTokenIdentifier(tokenIdentifier);
-    builder.setContainerIdString(event.getContainer().getId().toString());
+    builder.setContainerIdString(launchRequest.getContainerId().toString());
     builder.setCredentialsBinary(
-        ByteString.copyFrom(event.getContainerLaunchContext().getTokens()));
+        ByteString.copyFrom(launchRequest.getContainerLaunchContext().getTokens()));
     // TODO Avoid reading this from the environment
     builder.setUser(System.getenv(ApplicationConstants.Environment.USER.name()));
     return builder.build();
@@ -144,6 +137,8 @@ public class TezTestServiceContainerLauncher extends AbstractService implements
 
   @SuppressWarnings("unchecked")
   void sendContainerLaunchFailedMsg(ContainerId containerId, Throwable t) {
-    context.getEventHandler().handle(new AMContainerEventLaunchFailed(containerId, t == null ? "" : t.getMessage()));
+    getContext().containerLaunchFailed(containerId, t == null ? "" : t.getMessage());
   }
+
+
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/af1cc723/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 977d0d3..d3743e1 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,53 +14,32 @@
 
 package org.apache.tez.dag.app.launcher;
 
-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;
+import org.apache.tez.serviceplugins.api.ContainerLaunchRequest;
+import org.apache.tez.serviceplugins.api.ContainerLauncher;
+import org.apache.tez.serviceplugins.api.ContainerLauncherContext;
+import org.apache.tez.serviceplugins.api.ContainerStopRequest;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-public class TezTestServiceNoOpContainerLauncher extends AbstractService implements ContainerLauncher {
+public class TezTestServiceNoOpContainerLauncher extends ContainerLauncher {
 
   static final Logger LOG = LoggerFactory.getLogger(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();
+  public TezTestServiceNoOpContainerLauncher(ContainerLauncherContext containerLauncherContext) {
+    super(TezTestServiceNoOpContainerLauncher.class.getName(), containerLauncherContext);
   }
 
   @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;
-    }
+  public void launchContainer(ContainerLaunchRequest launchRequest) {
+    LOG.info("No-op launch for container {} succeeded on host: {}", launchRequest.getContainerId(),
+        launchRequest.getNodeId());
+    getContext().containerLaunched(launchRequest.getContainerId());
+  }
 
+  @Override
+  public void stopContainer(ContainerStopRequest stopRequest) {
+    LOG.info("DEBUG: Ignoring STOP_REQUEST {}", stopRequest);
+    getContext().containerStopRequested(stopRequest.getContainerId());
   }
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/af1cc723/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 073cb50..506e991 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
@@ -36,7 +36,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.serviceplugins.api.TaskAttemptEndReason;
 import org.apache.tez.dag.app.AppContext;
 import org.apache.tez.service.TezTestServiceConfConstants;
 import org.slf4j.Logger;

http://git-wip-us.apache.org/repos/asf/tez/blob/af1cc723/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 98673a6..444498e 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
@@ -30,8 +30,8 @@ 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.ContainerEndReason;
-import org.apache.tez.dag.api.TaskAttemptEndReason;
+import org.apache.tez.serviceplugins.api.ContainerEndReason;
+import org.apache.tez.serviceplugins.api.TaskAttemptEndReason;
 import org.apache.tez.dag.api.TaskCommunicatorContext;
 import org.apache.tez.dag.app.TezTaskCommunicatorImpl;
 import org.apache.tez.dag.app.TezTestServiceCommunicator;


[20/50] [abbrv] tez git commit: TEZ-2443. TaskRunner2 should call abort, NPEs while cleaning up tasks. (sseth)

Posted by ss...@apache.org.
TEZ-2443. TaskRunner2 should call abort, NPEs while cleaning up tasks. (sseth)


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

Branch: refs/heads/TEZ-2003
Commit: ec4a5c3d328079f08f0762d06a3b195183dce73b
Parents: 73f7316
Author: Siddharth Seth <ss...@apache.org>
Authored: Tue May 12 14:27:42 2015 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Fri Aug 14 13:46:44 2015 -0700

----------------------------------------------------------------------
 TEZ-2003-CHANGES.txt                                   |  1 +
 .../java/org/apache/tez/dag/api/TaskCommunicator.java  |  4 ++++
 .../tez/runtime/LogicalIOProcessorRuntimeTask.java     | 11 ++++++-----
 .../main/java/org/apache/tez/runtime/RuntimeTask.java  |  2 +-
 .../apache/tez/runtime/task/TaskRunner2Callable.java   | 13 +++++++------
 .../org/apache/tez/runtime/task/TezTaskRunner2.java    | 10 ++++++----
 6 files changed, 25 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/ec4a5c3d/TEZ-2003-CHANGES.txt
----------------------------------------------------------------------
diff --git a/TEZ-2003-CHANGES.txt b/TEZ-2003-CHANGES.txt
index 5d2e40a..ed72d6b 100644
--- a/TEZ-2003-CHANGES.txt
+++ b/TEZ-2003-CHANGES.txt
@@ -25,5 +25,6 @@ ALL CHANGES:
   TEZ-2433. Fixes after rebase 05/08
   TEZ-2438. tez-tools version in the branch is incorrect.
   TEZ-2434. Allow tasks to be killed in the Runtime.
+  TEZ-2443. TaskRunner2 should call abort, NPEs while cleaning up tasks.
 
 INCOMPATIBLE CHANGES:

http://git-wip-us.apache.org/repos/asf/tez/blob/ec4a5c3d/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 cadca0c..2651013 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
@@ -48,6 +48,10 @@ public abstract class TaskCommunicator extends AbstractService {
   // TODO TEZ-2003. Are additional APIs required to mark a container as completed ? - for completeness.
 
   // TODO TEZ-2003 Remove reference to TaskAttemptID
+  // TODO TEZ-2003 This needs some information about why the attempt is being unregistered.
+  // e.g. preempted in which case the task may need to be informed. Alternately as a result of
+  // a failed task.
+  // In case of preemption - a killTask API is likely a better bet than trying to overload this method.
   public abstract void unregisterRunningTaskAttempt(TezTaskAttemptID taskAttemptID);
 
   // TODO TEZ-2003 This doesn't necessarily belong here. A server may not start within the AM.

http://git-wip-us.apache.org/repos/asf/tez/blob/ec4a5c3d/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 8263b3f..de08e56 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
@@ -704,7 +704,7 @@ public class LogicalIOProcessorRuntimeTask extends RuntimeTask {
   }
 
   @Override
-  public synchronized void abortTask() throws Exception {
+  public synchronized void abortTask() {
     if (processor != null) {
       processor.abort();
     }
@@ -803,6 +803,7 @@ public class LogicalIOProcessorRuntimeTask extends RuntimeTask {
       LOG.debug("Num of inputs to be closed={}", initializedInputs.size());
       LOG.debug("Num of outputs to be closed={}", initializedOutputs.size());
     }
+
     // Close processor
     if (!processorClosed && processor != null) {
       try {
@@ -820,8 +821,8 @@ public class LogicalIOProcessorRuntimeTask extends RuntimeTask {
         Thread.currentThread().interrupt();
       } catch (Throwable e) {
         LOG.warn(
-            "Ignoring Exception when closing processor(cleanup). Exception class={}, message={}",
-                e.getClass().getName(), e.getMessage());
+            "Ignoring Exception when closing processor(cleanup). Exception class={}, message={}" +
+                e.getClass().getName(), e.getMessage(), e);
       }
     }
 
@@ -842,7 +843,7 @@ public class LogicalIOProcessorRuntimeTask extends RuntimeTask {
       } catch (Throwable e) {
         LOG.warn(
             "Ignoring exception when closing input {}(cleanup). Exception class={}, message={}",
-            srcVertexName, e.getClass().getName(), e.getMessage());
+            srcVertexName, e.getClass().getName(), e.getMessage(), e);
       } finally {
         LOG.info("Close input for vertex={}, sourceVertex={}, interruptedStatus={}", processor
             .getContext().getTaskVertexName(), srcVertexName, Thread.currentThread().isInterrupted());
@@ -866,7 +867,7 @@ public class LogicalIOProcessorRuntimeTask extends RuntimeTask {
       } catch (Throwable e) {
         LOG.warn(
             "Ignoring exception when closing output {}(cleanup). Exception class={}, message={}",
-            destVertexName, e.getClass().getName(), e.getMessage());
+            destVertexName, e.getClass().getName(), e.getMessage(), e);
       } finally {
         LOG.info("Close input for vertex={}, sourceVertex={}, interruptedStatus={}", processor
             .getContext().getTaskVertexName(), destVertexName, Thread.currentThread().isInterrupted());

http://git-wip-us.apache.org/repos/asf/tez/blob/ec4a5c3d/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 cdfb46a..33c0113 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
@@ -167,5 +167,5 @@ public abstract class RuntimeTask {
     taskDone.set(true);
   }
 
-  public abstract void abortTask() throws Exception;
+  public abstract void abortTask();
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/ec4a5c3d/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TaskRunner2Callable.java
----------------------------------------------------------------------
diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TaskRunner2Callable.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TaskRunner2Callable.java
index 7315bbd..ab77635 100644
--- a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TaskRunner2Callable.java
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TaskRunner2Callable.java
@@ -63,26 +63,26 @@ public class TaskRunner2Callable extends CallableWithNdc<TaskRunner2Callable.Tas
           if (stopRequested.get() || Thread.currentThread().isInterrupted()) {
             return new TaskRunner2CallableResult(null);
           }
-          LOG.info("Initializing task" + ", taskAttemptId=" + task.getTaskAttemptID());
+          LOG.info("Initializing task" + ", taskAttemptId={}", task.getTaskAttemptID());
           task.initialize();
 
           if (!stopRequested.get() && !Thread.currentThread().isInterrupted()) {
-            LOG.info("Running task, taskAttemptId=" + task.getTaskAttemptID());
+            LOG.info("Running task, taskAttemptId={}", task.getTaskAttemptID());
             task.run();
           } else {
-            LOG.info("Stopped before running the processor.");
+            LOG.info("Stopped before running the processor taskAttemptId={}", task.getTaskAttemptID());
             return new TaskRunner2CallableResult(null);
           }
 
           if (!stopRequested.get() && !Thread.currentThread().isInterrupted()) {
-            LOG.info("Closing task, taskAttemptId=" + task.getTaskAttemptID());
+            LOG.info("Closing task, taskAttemptId={}", task.getTaskAttemptID());
             task.close();
             task.setFrameworkCounters();
           } else {
-            LOG.info("Stopped before closing the processor");
+            LOG.info("Stopped before closing the processor, taskAttemptId={}", task.getTaskAttemptID());
             return new TaskRunner2CallableResult(null);
           }
-          LOG.info("Task completed, taskAttemptId=" + task.getTaskAttemptID() + ", askedToStop=" + stopRequested.get());
+          LOG.info("Task completed, taskAttemptId={}, askedToStop={}", task.getTaskAttemptID(), stopRequested.get());
 
 
           return new TaskRunner2CallableResult(null);
@@ -115,6 +115,7 @@ public class TaskRunner2Callable extends CallableWithNdc<TaskRunner2Callable.Tas
   public void interruptTask() {
     // Ensure the task is only interrupted once.
     if (!stopRequested.getAndSet(true)) {
+      task.abortTask();
       if (ownThread != null) {
         ownThread.interrupt();
       }

http://git-wip-us.apache.org/repos/asf/tez/blob/ec4a5c3d/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezTaskRunner2.java
----------------------------------------------------------------------
diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezTaskRunner2.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezTaskRunner2.java
index 73e5c76..ffbc6e8 100644
--- a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezTaskRunner2.java
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezTaskRunner2.java
@@ -250,10 +250,12 @@ public class TezTaskRunner2 {
   public void killTask() {
     synchronized (this) {
       if (isRunningState()) {
-        trySettingEndReason(EndReason.KILL_REQUESTED);
-        if (taskRunnerCallable != null) {
-          taskKillStartTime = System.currentTimeMillis();
-          taskRunnerCallable.interruptTask();
+        if (trySettingEndReason(EndReason.KILL_REQUESTED)) {
+          killTaskRequested.set(true);
+          if (taskRunnerCallable != null) {
+            taskKillStartTime = System.currentTimeMillis();
+            taskRunnerCallable.interruptTask();
+          }
         }
       }
     }


[33/50] [abbrv] tez git commit: TEZ-2651. Pluggable services should not extend AbstractService. (sseth)

Posted by ss...@apache.org.
TEZ-2651. Pluggable services should not extend AbstractService. (sseth)


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

Branch: refs/heads/TEZ-2003
Commit: fc8a4ce598d52496c19ec2508e4ebed7ef22eb63
Parents: 82c24ac
Author: Siddharth Seth <ss...@apache.org>
Authored: Tue Jul 28 14:55:40 2015 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Fri Aug 14 13:46:45 2015 -0700

----------------------------------------------------------------------
 TEZ-2003-CHANGES.txt                            |  1 +
 .../serviceplugins/api/ContainerLauncher.java   | 18 ++++++++++--
 .../apache/tez/dag/api/TaskCommunicator.java    | 30 +++++++++++++++++---
 .../tez/dag/api/TaskCommunicatorContext.java    |  5 ++++
 .../org/apache/tez/dag/app/DAGAppMaster.java    |  4 +--
 .../dag/app/TaskAttemptListenerImpTezDag.java   | 16 ++++++-----
 .../dag/app/TaskCommunicatorContextImpl.java    |  9 ++++++
 .../tez/dag/app/TezTaskCommunicatorImpl.java    | 24 ++++++----------
 .../dag/app/launcher/ContainerLauncherImpl.java |  6 ++--
 .../app/launcher/ContainerLauncherRouter.java   | 12 ++++++--
 .../app/launcher/LocalContainerLauncher.java    |  6 ++--
 .../apache/tez/dag/app/MockDAGAppMaster.java    |  6 ++--
 .../app/TestTaskAttemptListenerImplTezDag.java  |  8 +++---
 .../TezTestServiceContainerLauncher.java        |  6 ++--
 .../TezTestServiceNoOpContainerLauncher.java    |  2 +-
 .../TezTestServiceTaskCommunicatorImpl.java     | 29 +++++++++----------
 16 files changed, 116 insertions(+), 66 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/fc8a4ce5/TEZ-2003-CHANGES.txt
----------------------------------------------------------------------
diff --git a/TEZ-2003-CHANGES.txt b/TEZ-2003-CHANGES.txt
index a51669d..e57f76f 100644
--- a/TEZ-2003-CHANGES.txt
+++ b/TEZ-2003-CHANGES.txt
@@ -36,5 +36,6 @@ ALL CHANGES:
   TEZ-2124. Change Node tracking to work per external container source.
   TEZ-2004. Define basic interface for pluggable ContainerLaunchers.
   TEZ-2005. Define basic interface for pluggable TaskScheduler.
+  TEZ-2651. Pluggable services should not extend AbstractService.
 
 INCOMPATIBLE CHANGES:

http://git-wip-us.apache.org/repos/asf/tez/blob/fc8a4ce5/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ContainerLauncher.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ContainerLauncher.java b/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ContainerLauncher.java
index 218edb6..8337dcb 100644
--- a/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ContainerLauncher.java
+++ b/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ContainerLauncher.java
@@ -17,6 +17,7 @@ package org.apache.tez.serviceplugins.api;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.service.AbstractService;
+import org.apache.tez.common.ServicePluginLifecycle;
 
 /**
  * Plugin to allow custom container launchers to be written to launch containers on different types
@@ -25,18 +26,29 @@ import org.apache.hadoop.service.AbstractService;
 
 @InterfaceAudience.Public
 @InterfaceStability.Unstable
-public abstract class ContainerLauncher extends AbstractService {
+public abstract class ContainerLauncher implements ServicePluginLifecycle {
 
   private final ContainerLauncherContext containerLauncherContext;
 
   // TODO TEZ-2003 Simplify this by moving away from AbstractService. Potentially Guava AbstractService.
   // A serviceInit(Configuration) is not likely to be very useful, and will expose unnecessary internal
   // configuration to the services if populated with the AM Configuration
-  public ContainerLauncher(String name, ContainerLauncherContext containerLauncherContext) {
-    super(name);
+  public ContainerLauncher(ContainerLauncherContext containerLauncherContext) {
     this.containerLauncherContext = containerLauncherContext;
   }
 
+  @Override
+  public void initialize() throws Exception {
+  }
+
+  @Override
+  public void start() throws Exception {
+  }
+
+  @Override
+  public void shutdown() throws Exception {
+  }
+
   public final ContainerLauncherContext getContext() {
     return this.containerLauncherContext;
   }

http://git-wip-us.apache.org/repos/asf/tez/blob/fc8a4ce5/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 05e437c..f221414 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
@@ -18,9 +18,9 @@ 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.common.ServicePluginLifecycle;
 import org.apache.tez.dag.api.event.VertexStateUpdate;
 import org.apache.tez.serviceplugins.api.ContainerEndReason;
 import org.apache.tez.serviceplugins.api.TaskAttemptEndReason;
@@ -28,11 +28,33 @@ 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);
+public abstract class TaskCommunicator implements ServicePluginLifecycle {
+
+  private final TaskCommunicatorContext taskCommunicatorContext;
+
+  public TaskCommunicator(TaskCommunicatorContext taskCommunicatorContext) {
+    this.taskCommunicatorContext = taskCommunicatorContext;
+  }
+
+  public TaskCommunicatorContext getContext() {
+    return taskCommunicatorContext;
+  }
+
+  @Override
+  public void initialize() throws Exception {
   }
 
+  @Override
+  public void start() throws Exception {
+  }
+
+  @Override
+  public void shutdown() throws Exception {
+  }
+
+  // TODO Post TEZ-2003 Move this into the API module. Moving this requires abstractions for
+  // TaskSpec and related classes. (assuming that's efficient for execution)
+
   // 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);

http://git-wip-us.apache.org/repos/asf/tez/blob/fc8a4ce5/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 b6e63f7..ab32ec1 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
@@ -18,6 +18,7 @@ import javax.annotation.Nullable;
 import java.io.IOException;
 import java.util.Set;
 
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
@@ -35,6 +36,9 @@ public interface TaskCommunicatorContext {
 
   // TODO TEZ-2003 Maybe add book-keeping as a helper library, instead of each impl tracking container to task etc.
 
+  // TODO TEZ-2003 To be replaced by getInitialPayload
+  Configuration getInitialConfiguration();
+
   ApplicationAttemptId getApplicationAttemptId();
   Credentials getCredentials();
 
@@ -42,6 +46,7 @@ public interface TaskCommunicatorContext {
   boolean canCommit(TezTaskAttemptID taskAttemptId) throws IOException;
 
   // TODO TEZ-2003 Split the heartbeat API to a liveness check and a status update
+  // KKK Rename this API
   TaskHeartbeatResponse heartbeat(TaskHeartbeatRequest request) throws IOException, TezException;
 
   boolean isKnownContainer(ContainerId containerId);

http://git-wip-us.apache.org/repos/asf/tez/blob/fc8a4ce5/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 ef27ddf..f3914d8 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
@@ -1047,8 +1047,8 @@ public class DAGAppMaster extends AbstractService {
                                                           String[] taskCommunicatorClasses,
                                                           boolean isLocal) {
     TaskAttemptListener lis =
-        new TaskAttemptListenerImpTezDag(context, thh, chh, jobTokenSecretManager,
-            taskCommunicatorClasses, isLocal);
+        new TaskAttemptListenerImpTezDag(context, thh, chh,
+            taskCommunicatorClasses, amConf, isLocal);
     return lis;
   }
 

http://git-wip-us.apache.org/repos/asf/tez/blob/fc8a4ce5/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 47b63dd..599c208 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
@@ -28,6 +28,7 @@ import java.util.concurrent.ConcurrentMap;
 
 import com.google.common.annotations.VisibleForTesting;
 import org.apache.commons.collections4.ListUtils;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.tez.serviceplugins.api.ContainerEndReason;
 import org.apache.tez.dag.app.dag.event.TaskAttemptEvent;
 import org.apache.tez.dag.app.dag.event.TaskAttemptEventStatusUpdate;
@@ -61,7 +62,6 @@ import org.apache.tez.dag.app.rm.container.AMContainerTask;
 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;
 
 
 @SuppressWarnings("unchecked")
@@ -75,6 +75,7 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
   private final AppContext context;
   private final TaskCommunicator[] taskCommunicators;
   private final TaskCommunicatorContext[] taskCommunicatorContexts;
+  protected final ServicePluginLifecycleAbstractService []taskCommunicatorServiceWrappers;
 
   protected final TaskHeartbeatHandler taskHeartbeatHandler;
   protected final ContainerHeartbeatHandler containerHeartbeatHandler;
@@ -99,9 +100,8 @@ 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,
                                       String [] taskCommunicatorClassIdentifiers,
+                                      Configuration conf,
                                       boolean isPureLocalMode) {
     super(TaskAttemptListenerImpTezDag.class.getName());
     this.context = context;
@@ -118,9 +118,11 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
     }
     this.taskCommunicators = new TaskCommunicator[taskCommunicatorClassIdentifiers.length];
     this.taskCommunicatorContexts = new TaskCommunicatorContext[taskCommunicatorClassIdentifiers.length];
+    this.taskCommunicatorServiceWrappers = new ServicePluginLifecycleAbstractService[taskCommunicatorClassIdentifiers.length];
     for (int i = 0 ; i < taskCommunicatorClassIdentifiers.length ; i++) {
-      taskCommunicatorContexts[i] = new TaskCommunicatorContextImpl(context, this, i);
+      taskCommunicatorContexts[i] = new TaskCommunicatorContextImpl(context, this, conf, i);
       taskCommunicators[i] = createTaskCommunicator(taskCommunicatorClassIdentifiers[i], i);
+      taskCommunicatorServiceWrappers[i] = new ServicePluginLifecycleAbstractService(taskCommunicators[i]);
     }
     // TODO TEZ-2118 Start using taskCommunicator indices properly
   }
@@ -129,15 +131,15 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
   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();
+      taskCommunicatorServiceWrappers[i].init(getConfig());
+      taskCommunicatorServiceWrappers[i].start();
     }
   }
 
   @Override
   public void serviceStop() {
     for (int i = 0 ; i < taskCommunicators.length ; i++) {
-      taskCommunicators[i].stop();
+      taskCommunicatorServiceWrappers[i].stop();
     }
   }
 

http://git-wip-us.apache.org/repos/asf/tez/blob/fc8a4ce5/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 50e006d..035db93 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
@@ -23,6 +23,7 @@ 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.conf.Configuration;
 import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
@@ -48,14 +49,17 @@ public class TaskCommunicatorContextImpl implements TaskCommunicatorContext, Ver
   private final int taskCommunicatorIndex;
   private final ReentrantReadWriteLock.ReadLock dagChangedReadLock;
   private final ReentrantReadWriteLock.WriteLock dagChangedWriteLock;
+  private final Configuration conf;
 
   private DAG dag;
 
   public TaskCommunicatorContextImpl(AppContext appContext,
                                      TaskAttemptListenerImpTezDag taskAttemptListener,
+                                     Configuration conf,
                                      int taskCommunicatorIndex) {
     this.context = appContext;
     this.taskAttemptListener = taskAttemptListener;
+    this.conf = conf;
     this.taskCommunicatorIndex = taskCommunicatorIndex;
 
     ReentrantReadWriteLock dagChangedLock = new ReentrantReadWriteLock();
@@ -64,6 +68,11 @@ public class TaskCommunicatorContextImpl implements TaskCommunicatorContext, Ver
   }
 
   @Override
+  public Configuration getInitialConfiguration() {
+    return conf;
+  }
+
+  @Override
   public ApplicationAttemptId getApplicationAttemptId() {
     return context.getApplicationAttemptId();
   }

http://git-wip-us.apache.org/repos/asf/tez/blob/fc8a4ce5/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 0374022..93b5b43 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
@@ -67,7 +67,6 @@ public class TezTaskCommunicatorImpl extends TaskCommunicator {
   private static final ContainerTask TASK_FOR_INVALID_JVM = new ContainerTask(
       null, true, null, null, false);
 
-  private final TaskCommunicatorContext taskCommunicatorContext;
   private final TezTaskUmbilicalProtocol taskUmbilical;
 
   protected final ConcurrentMap<ContainerId, ContainerInfo> registeredContainers =
@@ -116,25 +115,24 @@ public class TezTaskCommunicatorImpl extends TaskCommunicator {
    * Construct the service.
    */
   public TezTaskCommunicatorImpl(TaskCommunicatorContext taskCommunicatorContext) {
-    super(TezTaskCommunicatorImpl.class.getName());
-    this.taskCommunicatorContext = taskCommunicatorContext;
+    super(taskCommunicatorContext);
     this.taskUmbilical = new TezTaskUmbilicalProtocolImpl();
-    this.tokenIdentifier = this.taskCommunicatorContext.getApplicationAttemptId().getApplicationId().toString();
+    this.tokenIdentifier = taskCommunicatorContext.getApplicationAttemptId().getApplicationId().toString();
     this.sessionToken = TokenCache.getSessionToken(taskCommunicatorContext.getCredentials());
   }
 
   @Override
-  public void serviceStart() {
+  public void start() {
     startRpcServer();
   }
 
   @Override
-  public void serviceStop() {
+  public void shutdown() {
     stopRpcServer();
   }
 
   protected void startRpcServer() {
-    Configuration conf = getConfig();
+    Configuration conf = getContext().getInitialConfiguration();
     try {
       JobTokenSecretManager jobTokenSecretManager =
           new JobTokenSecretManager();
@@ -281,10 +279,6 @@ public class TezTaskCommunicatorImpl extends TaskCommunicator {
     return sessionToken;
   }
 
-  protected TaskCommunicatorContext getTaskCommunicatorContext() {
-    return taskCommunicatorContext;
-  }
-
   public TezTaskUmbilicalProtocol getUmbilical() {
     return this.taskUmbilical;
   }
@@ -305,7 +299,7 @@ public class TezTaskCommunicatorImpl extends TaskCommunicator {
         }
         task = getContainerTask(containerId);
         if (task != null && !task.shouldDie()) {
-          taskCommunicatorContext
+          getContext()
               .taskStartedRemotely(task.getTaskSpec().getTaskAttemptID(), containerId);
         }
       }
@@ -317,7 +311,7 @@ public class TezTaskCommunicatorImpl extends TaskCommunicator {
 
     @Override
     public boolean canCommit(TezTaskAttemptID taskAttemptId) throws IOException {
-      return taskCommunicatorContext.canCommit(taskAttemptId);
+      return getContext().canCommit(taskAttemptId);
     }
 
     @Override
@@ -370,7 +364,7 @@ public class TezTaskCommunicatorImpl extends TaskCommunicator {
         TaskHeartbeatRequest tRequest = new TaskHeartbeatRequest(request.getContainerIdentifier(),
             request.getCurrentTaskAttemptID(), request.getEvents(), request.getStartIndex(),
             request.getPreRoutedStartIndex(), request.getMaxEvents());
-        tResponse = taskCommunicatorContext.heartbeat(tRequest);
+        tResponse = getContext().heartbeat(tRequest);
       }
       TezHeartbeatResponse response = new TezHeartbeatResponse();
       response.setLastRequestId(requestId);
@@ -402,7 +396,7 @@ public class TezTaskCommunicatorImpl extends TaskCommunicator {
     ContainerInfo containerInfo = registeredContainers.get(containerId);
     ContainerTask task = null;
     if (containerInfo == null) {
-      if (taskCommunicatorContext.isKnownContainer(containerId)) {
+      if (getContext().isKnownContainer(containerId)) {
         LOG.info("Container with id: " + containerId
             + " is valid, but no longer registered, and will be killed");
       } else {

http://git-wip-us.apache.org/repos/asf/tez/blob/fc8a4ce5/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 fe0178c..34c7bc0 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
@@ -223,7 +223,7 @@ public class ContainerLauncherImpl extends ContainerLauncher {
   }
 
   public ContainerLauncherImpl(ContainerLauncherContext containerLauncherContext) {
-    super(ContainerLauncherImpl.class.getName(), containerLauncherContext);
+    super(containerLauncherContext);
     this.conf = new Configuration(containerLauncherContext.getInitialConfiguration());
     conf.setInt(
         CommonConfigurationKeysPublic.IPC_CLIENT_CONNECTION_MAXIDLETIME_KEY,
@@ -235,7 +235,7 @@ public class ContainerLauncherImpl extends ContainerLauncher {
   }
 
   @Override
-  public void serviceStart() {
+  public void start() {
     // pass a copy of config to ContainerManagementProtocolProxy until YARN-3497 is fixed
     cmProxy =
         new ContainerManagementProtocolProxy(conf);
@@ -307,7 +307,7 @@ public class ContainerLauncherImpl extends ContainerLauncher {
   }
 
   @Override
-  public void serviceStop() {
+  public void shutdown() {
     if(!serviceStopped.compareAndSet(false, true)) {
       LOG.info("Ignoring multiple stops");
       return;

http://git-wip-us.apache.org/repos/asf/tez/blob/fc8a4ce5/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 9f741cf..7c6a6a4 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
@@ -23,6 +23,7 @@ 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.app.ServicePluginLifecycleAbstractService;
 import org.apache.tez.serviceplugins.api.ContainerLaunchRequest;
 import org.apache.tez.serviceplugins.api.ContainerLauncher;
 import org.apache.tez.serviceplugins.api.ContainerLauncherContext;
@@ -45,6 +46,7 @@ public class ContainerLauncherRouter extends AbstractService
 
   private final ContainerLauncher containerLaunchers[];
   private final ContainerLauncherContext containerLauncherContexts[];
+  protected final ServicePluginLifecycleAbstractService[] containerLauncherServiceWrappers;
   private final AppContext appContext;
 
   @VisibleForTesting
@@ -53,6 +55,8 @@ public class ContainerLauncherRouter extends AbstractService
     this.appContext = context;
     containerLaunchers = new ContainerLauncher[] {containerLauncher};
     containerLauncherContexts = new ContainerLauncherContext[] {containerLauncher.getContext()};
+    containerLauncherServiceWrappers = new ServicePluginLifecycleAbstractService[]{
+        new ServicePluginLifecycleAbstractService(containerLauncher)};
   }
 
   // Accepting conf to setup final parameters, if required.
@@ -75,6 +79,7 @@ public class ContainerLauncherRouter extends AbstractService
     }
     containerLauncherContexts = new ContainerLauncherContext[containerLauncherClassIdentifiers.length];
     containerLaunchers = new ContainerLauncher[containerLauncherClassIdentifiers.length];
+    containerLauncherServiceWrappers = new ServicePluginLifecycleAbstractService[containerLauncherClassIdentifiers.length];
 
 
     for (int i = 0; i < containerLauncherClassIdentifiers.length; i++) {
@@ -82,6 +87,7 @@ public class ContainerLauncherRouter extends AbstractService
       containerLauncherContexts[i] = containerLauncherContext;
       containerLaunchers[i] = createContainerLauncher(containerLauncherClassIdentifiers[i], context,
           containerLauncherContext, taskAttemptListener, workingDirectory, isPureLocalMode, conf);
+      containerLauncherServiceWrappers[i] = new ServicePluginLifecycleAbstractService(containerLaunchers[i]);
     }
   }
 
@@ -130,21 +136,21 @@ public class ContainerLauncherRouter extends AbstractService
   @Override
   public void serviceInit(Configuration conf) {
     for (int i = 0 ; i < containerLaunchers.length ; i++) {
-      ((AbstractService) containerLaunchers[i]).init(conf);
+      containerLauncherServiceWrappers[i].init(conf);
     }
   }
 
   @Override
   public void serviceStart() {
     for (int i = 0 ; i < containerLaunchers.length ; i++) {
-      ((AbstractService) containerLaunchers[i]).start();
+      containerLauncherServiceWrappers[i].start();
     }
   }
 
   @Override
   public void serviceStop() {
     for (int i = 0 ; i < containerLaunchers.length ; i++) {
-      ((AbstractService) containerLaunchers[i]).stop();
+      containerLauncherServiceWrappers[i].stop();
     }
   }
 

http://git-wip-us.apache.org/repos/asf/tez/blob/fc8a4ce5/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 a1b8e29..3975111 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
@@ -111,7 +111,7 @@ public class LocalContainerLauncher extends ContainerLauncher {
     // starts up. It's not possible to set these up via a static payload.
     // Will need some kind of mechanism to dynamically crate payloads / bind to parameters
     // after the AM starts up.
-    super(LocalContainerLauncher.class.getName(), containerLauncherContext);
+    super(containerLauncherContext);
     this.context = context;
     this.tal = taskAttemptListener;
     this.workingDirectory = workingDirectory;
@@ -139,14 +139,14 @@ public class LocalContainerLauncher extends ContainerLauncher {
   }
 
   @Override
-  public void serviceStart() throws Exception {
+  public void start() throws Exception {
     eventHandlingThread =
         new Thread(new TezSubTaskRunner(), "LocalContainerLauncher-SubTaskRunner");
     eventHandlingThread.start();
   }
 
   @Override
-  public void serviceStop() throws Exception {
+  public void shutdown() throws Exception {
     if (!serviceStopped.compareAndSet(false, true)) {
       LOG.info("Service Already stopped. Ignoring additional stop");
       return;

http://git-wip-us.apache.org/repos/asf/tez/blob/fc8a4ce5/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 3c3c6a7..21ae5f7 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
@@ -139,7 +139,7 @@ public class MockDAGAppMaster extends DAGAppMaster {
 
     public MockContainerLauncher(AtomicBoolean goFlag,
                                  ContainerLauncherContext containerLauncherContext) {
-      super("MockContainerLauncher", containerLauncherContext);
+      super(containerLauncherContext);
       this.goFlag = goFlag;
     }
 
@@ -182,7 +182,7 @@ public class MockDAGAppMaster extends DAGAppMaster {
     }
     
     @Override
-    public void serviceStart() throws Exception {
+    public void start() throws Exception {
       taListener = (TaskAttemptListenerImpTezDag) getTaskAttemptListener();
       taskCommunicator = (TezTaskCommunicatorImpl) taListener.getTaskCommunicator(0);
       eventHandlingThread = new Thread(this);
@@ -199,7 +199,7 @@ public class MockDAGAppMaster extends DAGAppMaster {
     }
 
     @Override
-    public void serviceStop() throws Exception {
+    public void shutdown() throws Exception {
       if (eventHandlingThread != null) {
         eventHandlingThread.interrupt();
         eventHandlingThread.join(2000l);

http://git-wip-us.apache.org/repos/asf/tez/blob/fc8a4ce5/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 df643e4..41a7373 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
@@ -299,7 +299,7 @@ public class TestTaskAttemptListenerImplTezDag {
     sessionToken.setService(identifier.getJobId());
     TokenCache.setSessionToken(sessionToken, credentials);
     taskAttemptListener = new TaskAttemptListenerImpTezDag(appContext,
-        mock(TaskHeartbeatHandler.class), mock(ContainerHeartbeatHandler.class), null, null, false);
+        mock(TaskHeartbeatHandler.class), mock(ContainerHeartbeatHandler.class), null, conf, false);
     // no exception happen, should started properly
     taskAttemptListener.init(conf);
     taskAttemptListener.start();
@@ -319,7 +319,7 @@ public class TestTaskAttemptListenerImplTezDag {
 
       conf.set(TezConfiguration.TEZ_AM_TASK_AM_PORT_RANGE, port + "-" + port);
       taskAttemptListener = new TaskAttemptListenerImpTezDag(appContext,
-          mock(TaskHeartbeatHandler.class), mock(ContainerHeartbeatHandler.class), null, null, false);
+          mock(TaskHeartbeatHandler.class), mock(ContainerHeartbeatHandler.class), null, conf, false);
       taskAttemptListener.init(conf);
       taskAttemptListener.start();
       int resultedPort = taskAttemptListener.getTaskCommunicator(0).getAddress().getPort();
@@ -375,10 +375,10 @@ public class TestTaskAttemptListenerImplTezDag {
     public TaskAttemptListenerImplForTest(AppContext context,
                                           TaskHeartbeatHandler thh,
                                           ContainerHeartbeatHandler chh,
-                                          JobTokenSecretManager jobTokenSecretManager,
                                           String[] taskCommunicatorClassIdentifiers,
+                                          Configuration conf,
                                           boolean isPureLocalMode) {
-      super(context, thh, chh, jobTokenSecretManager, taskCommunicatorClassIdentifiers,
+      super(context, thh, chh, taskCommunicatorClassIdentifiers, conf,
           isPureLocalMode);
     }
 

http://git-wip-us.apache.org/repos/asf/tez/blob/fc8a4ce5/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 dbf5054..85f9415 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
@@ -54,7 +54,7 @@ public class TezTestServiceContainerLauncher extends ContainerLauncher {
 
   // Configuration passed in here to set up final parameters
   public TezTestServiceContainerLauncher(ContainerLauncherContext containerLauncherContext) {
-    super(TezTestServiceContainerLauncher.class.getName(), containerLauncherContext);
+    super(containerLauncherContext);
     int numThreads = getContext().getInitialConfiguration().getInt(
         TezTestServiceConfConstants.TEZ_TEST_SERVICE_AM_COMMUNICATOR_NUM_THREADS,
         TezTestServiceConfConstants.TEZ_TEST_SERVICE_AM_COMMUNICATOR_NUM_THREADS_DEFAULT);
@@ -69,13 +69,13 @@ public class TezTestServiceContainerLauncher extends ContainerLauncher {
   }
 
   @Override
-  public void serviceStart() {
+  public void start() {
     communicator.init(getContext().getInitialConfiguration());
     communicator.start();
   }
 
   @Override
-  public void serviceStop() {
+  public void shutdown() {
     communicator.stop();
   }
 

http://git-wip-us.apache.org/repos/asf/tez/blob/fc8a4ce5/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 d3743e1..7b42296 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
@@ -27,7 +27,7 @@ public class TezTestServiceNoOpContainerLauncher extends ContainerLauncher {
 
 
   public TezTestServiceNoOpContainerLauncher(ContainerLauncherContext containerLauncherContext) {
-    super(TezTestServiceNoOpContainerLauncher.class.getName(), containerLauncherContext);
+    super(containerLauncherContext);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/tez/blob/fc8a4ce5/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 444498e..078ea79 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,7 +23,6 @@ import java.util.concurrent.RejectedExecutionException;
 
 import com.google.protobuf.ByteString;
 import com.google.protobuf.ServiceException;
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.DataOutputBuffer;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.security.Credentials;
@@ -75,20 +74,20 @@ public class TezTestServiceTaskCommunicatorImpl extends TezTaskCommunicatorImpl
   }
 
   @Override
-  public void serviceInit(Configuration conf) throws Exception {
-    super.serviceInit(conf);
-    this.communicator.init(conf);
+  public void initialize() throws Exception {
+    super.initialize();
+    this.communicator.init(getContext().getInitialConfiguration());
   }
 
   @Override
-  public void serviceStart() {
-    super.serviceStart();
+  public void start() {
+    super.start();
     this.communicator.start();
   }
 
   @Override
-  public void serviceStop() {
-    super.serviceStop();
+  public void shutdown() {
+    super.shutdown();
     this.communicator.stop();
   }
 
@@ -132,7 +131,7 @@ public class TezTestServiceTaskCommunicatorImpl extends TezTaskCommunicatorImpl
     }
     // 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()
+    getContext()
         .taskStartedRemotely(taskSpec.getTaskAttemptID(), containerId);
     communicator.submitWork(requestProto, host, port,
         new TezTestServiceCommunicator.ExecuteRequestCallback<SubmitWorkResponseProto>() {
@@ -154,19 +153,19 @@ public class TezTestServiceTaskCommunicatorImpl extends TezTaskCommunicatorImpl
               RemoteException re = (RemoteException) t;
               String message = re.toString();
               if (message.contains(RejectedExecutionException.class.getName())) {
-                getTaskCommunicatorContext().taskKilled(taskSpec.getTaskAttemptID(),
+                getContext().taskKilled(taskSpec.getTaskAttemptID(),
                     TaskAttemptEndReason.SERVICE_BUSY, "Service Busy");
               } else {
-                getTaskCommunicatorContext()
+                getContext()
                     .taskFailed(taskSpec.getTaskAttemptID(), TaskAttemptEndReason.OTHER,
                         t.toString());
               }
             } else {
               if (t instanceof IOException) {
-                getTaskCommunicatorContext().taskKilled(taskSpec.getTaskAttemptID(),
+                getContext().taskKilled(taskSpec.getTaskAttemptID(),
                     TaskAttemptEndReason.COMMUNICATION_ERROR, "Communication Error");
               } else {
-                getTaskCommunicatorContext()
+                getContext()
                     .taskFailed(taskSpec.getTaskAttemptID(), TaskAttemptEndReason.OTHER,
                         t.getMessage());
               }
@@ -191,11 +190,11 @@ public class TezTestServiceTaskCommunicatorImpl extends TezTaskCommunicatorImpl
     builder.setAmPort(getAddress().getPort());
     Credentials taskCredentials = new Credentials();
     // Credentials can change across DAGs. Ideally construct only once per DAG.
-    taskCredentials.addAll(getTaskCommunicatorContext().getCredentials());
+    taskCredentials.addAll(getContext().getCredentials());
 
     ByteBuffer credentialsBinary = credentialMap.get(taskSpec.getDAGName());
     if (credentialsBinary == null) {
-      credentialsBinary = serializeCredentials(getTaskCommunicatorContext().getCredentials());
+      credentialsBinary = serializeCredentials(getContext().getCredentials());
       credentialMap.putIfAbsent(taskSpec.getDAGName(), credentialsBinary.duplicate());
     } else {
       credentialsBinary = credentialsBinary.duplicate();


[31/50] [abbrv] tez git commit: TEZ-2005. Define basic interface for pluggable TaskScheduler. (sseth)

Posted by ss...@apache.org.
http://git-wip-us.apache.org/repos/asf/tez/blob/82c24ac0/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 d4cf317..1e76dc9 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
@@ -37,6 +37,10 @@ import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
 
+import org.apache.tez.serviceplugins.api.TaskScheduler;
+import org.apache.tez.serviceplugins.api.TaskSchedulerContext;
+import org.apache.tez.serviceplugins.api.TaskSchedulerContext.AMState;
+import org.apache.tez.serviceplugins.api.TaskSchedulerContext.AppFinalStatus;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.commons.math3.random.RandomDataGenerator;
@@ -59,10 +63,7 @@ import org.apache.hadoop.yarn.util.resource.Resources;
 import org.apache.tez.serviceplugins.api.TaskAttemptEndReason;
 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.DAGAppMasterState;
-import org.apache.tez.dag.app.rm.TaskSchedulerService.TaskSchedulerAppCallback.AppFinalStatus;
-import org.apache.tez.dag.app.rm.container.ContainerSignatureMatcher;
+import org.apache.tez.common.ContainerSignatureMatcher;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
@@ -80,17 +81,14 @@ import com.google.common.util.concurrent.ThreadFactoryBuilder;
       eventHandler.handle(new AMNodeEventNodeCountUpdated(clusterNmCount));
     }
  */
-public class YarnTaskSchedulerService extends TaskSchedulerService
+public class YarnTaskSchedulerService extends TaskScheduler
                              implements AMRMClientAsync.CallbackHandler {
   private static final Logger LOG = LoggerFactory.getLogger(YarnTaskSchedulerService.class);
 
 
 
   final TezAMRMClientAsync<CookieContainerRequest> amRmClient;
-  final TaskSchedulerAppCallback realAppClient;
-  final TaskSchedulerAppCallback appClientDelegate;
   final ContainerSignatureMatcher containerSignatureMatcher;
-  ExecutorService appCallbackExecutor;
 
   // Container Re-Use configuration
   private boolean shouldReuseContainers;
@@ -131,7 +129,6 @@ public class YarnTaskSchedulerService extends TaskSchedulerService
   final String appHostName;
   final int appHostPort;
   final String appTrackingUrl;
-  final AppContext appContext;
   private AtomicBoolean hasUnregistered = new AtomicBoolean(false);
 
   AtomicBoolean isStopped = new AtomicBoolean(false);
@@ -150,6 +147,7 @@ public class YarnTaskSchedulerService extends TaskSchedulerService
   Set<ContainerId> sessionMinHeldContainers = Sets.newHashSet();
   
   RandomDataGenerator random = new RandomDataGenerator();
+  private final Configuration conf;
 
   @VisibleForTesting
   protected AtomicBoolean shouldUnregister = new AtomicBoolean(false);
@@ -213,51 +211,29 @@ public class YarnTaskSchedulerService extends TaskSchedulerService
     }
   }
 
-  public YarnTaskSchedulerService(TaskSchedulerAppCallback appClient,
-                        ContainerSignatureMatcher containerSignatureMatcher,
-                        String appHostName,
-                        int appHostPort,
-                        String appTrackingUrl,
-                        AppContext appContext) {
-    super(YarnTaskSchedulerService.class.getName());
-    this.realAppClient = appClient;
-    this.appCallbackExecutor = createAppCallbackExecutorService();
-    this.containerSignatureMatcher = containerSignatureMatcher;
-    this.appClientDelegate = createAppCallbackDelegate(appClient);
+  public YarnTaskSchedulerService(TaskSchedulerContext taskSchedulerContext) {
+    super(taskSchedulerContext);
+    this.containerSignatureMatcher = taskSchedulerContext.getContainerSignatureMatcher();
     this.amRmClient = TezAMRMClientAsync.createAMRMClientAsync(1000, this);
-    this.appHostName = appHostName;
-    this.appHostPort = appHostPort;
-    this.appTrackingUrl = appTrackingUrl;
-    this.appContext = appContext;
+    this.appHostName = taskSchedulerContext.getAppHostName();
+    this.appHostPort = taskSchedulerContext.getAppClientPort();
+    this.appTrackingUrl = taskSchedulerContext.getAppTrackingUrl();
+    this.conf = taskSchedulerContext.getInitialConfiguration();
   }
 
   @Private
   @VisibleForTesting
-  YarnTaskSchedulerService(TaskSchedulerAppCallback appClient,
-      ContainerSignatureMatcher containerSignatureMatcher,
-      String appHostName,
-      int appHostPort,
-      String appTrackingUrl,
-      TezAMRMClientAsync<CookieContainerRequest> client,
-      AppContext appContext) {
-    super(YarnTaskSchedulerService.class.getName());
-    this.realAppClient = appClient;
-    this.appCallbackExecutor = createAppCallbackExecutorService();
-    this.containerSignatureMatcher = containerSignatureMatcher;
-    this.appClientDelegate = createAppCallbackDelegate(appClient);
+  YarnTaskSchedulerService(TaskSchedulerContext taskSchedulerContext,
+      TezAMRMClientAsync<CookieContainerRequest> client) {
+    super(taskSchedulerContext);
+    this.containerSignatureMatcher = taskSchedulerContext.getContainerSignatureMatcher();
     this.amRmClient = client;
-    this.appHostName = appHostName;
-    this.appHostPort = appHostPort;
-    this.appTrackingUrl = appTrackingUrl;
-    this.appContext = appContext;
+    this.appHostName = taskSchedulerContext.getAppHostName();
+    this.appHostPort = taskSchedulerContext.getAppClientPort();
+    this.appTrackingUrl = taskSchedulerContext.getAppTrackingUrl();
+    this.conf = taskSchedulerContext.getInitialConfiguration();
   }
 
-  @VisibleForTesting
-  ExecutorService createAppCallbackExecutorService() {
-    return Executors.newSingleThreadExecutor(new ThreadFactoryBuilder()
-        .setNameFormat("TaskSchedulerAppCaller #%d").setDaemon(true).build());
-  }
-  
   @Override
   public Resource getAvailableResources() {
     return amRmClient.getAvailableResources();
@@ -269,12 +245,6 @@ public class YarnTaskSchedulerService extends TaskSchedulerService
     return amRmClient.getClusterNodeCount();
   }
 
-  TaskSchedulerAppCallback createAppCallbackDelegate(
-      TaskSchedulerAppCallback realAppClient) {
-    return new TaskSchedulerAppCallbackWrapper(realAppClient,
-        appCallbackExecutor);
-  }
-
   @Override
   public void setShouldUnregister() {
     this.shouldUnregister.set(true);
@@ -287,8 +257,9 @@ public class YarnTaskSchedulerService extends TaskSchedulerService
 
   // AbstractService methods
   @Override
-  public synchronized void serviceInit(Configuration conf) {
+  public synchronized void initialize() {
 
+    // TODO Post TEZ-2003. Make all of these final fields.
     amRmClient.init(conf);
     int heartbeatIntervalMax = conf.getInt(
         TezConfiguration.TEZ_AM_RM_HEARTBEAT_INTERVAL_MS_MAX,
@@ -361,7 +332,7 @@ public class YarnTaskSchedulerService extends TaskSchedulerService
   }
 
   @Override
-  public void serviceStart() {
+  public void start() {
     try {
       RegisterApplicationMasterResponse response;
       synchronized (this) {
@@ -371,7 +342,7 @@ public class YarnTaskSchedulerService extends TaskSchedulerService
                                                         appTrackingUrl);
       }
       // upcall to app outside locks
-      appClientDelegate.setApplicationRegistrationData(
+      getContext().setApplicationRegistrationData(
           response.getMaximumResourceCapability(),
           response.getApplicationACLs(),
           response.getClientToAMTokenMasterKey());
@@ -387,7 +358,7 @@ public class YarnTaskSchedulerService extends TaskSchedulerService
   }
 
   @Override
-  public void serviceStop() throws InterruptedException {
+  public void shutdown() throws InterruptedException {
     // upcall to app outside of locks
     try {
       delayedContainerManager.shutdown();
@@ -396,7 +367,7 @@ public class YarnTaskSchedulerService extends TaskSchedulerService
       synchronized (this) {
         isStopped.set(true);
         if (shouldUnregister.get()) {
-          AppFinalStatus status = appClientDelegate.getFinalAppStatus();
+          AppFinalStatus status = getContext().getFinalAppStatus();
           LOG.info("Unregistering application from RM"
               + ", exitStatus=" + status.exitStatus
               + ", exitMessage=" + status.exitMessage
@@ -413,8 +384,6 @@ public class YarnTaskSchedulerService extends TaskSchedulerService
       // operation and at the same time the callback operation might be trying
       // to get our lock.
       amRmClient.stop();
-      appCallbackExecutor.shutdown();
-      appCallbackExecutor.awaitTermination(1000l, TimeUnit.MILLISECONDS);
     } catch (YarnException e) {
       LOG.error("Yarn Exception while unregistering ", e);
       throw new TezUncheckedException(e);
@@ -478,7 +447,7 @@ public class YarnTaskSchedulerService extends TaskSchedulerService
 
     // upcall to app must be outside locks
     for (Entry<Object, ContainerStatus> entry : appContainerStatus.entrySet()) {
-      appClientDelegate.containerCompleted(entry.getKey(), entry.getValue());
+      getContext().containerCompleted(entry.getKey(), entry.getValue());
     }
   }
 
@@ -528,7 +497,7 @@ public class YarnTaskSchedulerService extends TaskSchedulerService
   private synchronized Map<CookieContainerRequest, Container>
       assignNewlyAllocatedContainers(Iterable<Container> containers) {
 
-    boolean amInCompletionState = appContext.isAMInCompletionState();
+    boolean amInCompletionState = (getContext().getAMState() == AMState.COMPLETED);
     Map<CookieContainerRequest, Container> assignedContainers =
         new HashMap<CookieContainerRequest, Container>();
 
@@ -550,7 +519,7 @@ public class YarnTaskSchedulerService extends TaskSchedulerService
   private synchronized Map<CookieContainerRequest, Container>
       tryAssignReUsedContainers(Iterable<Container> containers) {
 
-    boolean amInCompletionState = appContext.isAMInCompletionState();
+    boolean amInCompletionState = (getContext().getAMState() == AMState.COMPLETED);
     Map<CookieContainerRequest, Container> assignedContainers =
       new HashMap<CookieContainerRequest, Container>();
 
@@ -590,7 +559,7 @@ public class YarnTaskSchedulerService extends TaskSchedulerService
   private synchronized Map<CookieContainerRequest, Container>
       assignDelayedContainer(HeldContainer heldContainer) {
 
-    DAGAppMasterState state = appContext.getAMState();
+    AMState state = getContext().getAMState();
 
     boolean isNew = heldContainer.isNew();
     if (LOG.isDebugEnabled()) {
@@ -606,13 +575,13 @@ public class YarnTaskSchedulerService extends TaskSchedulerService
         + ", isNew=" + isNew);
     }
 
-    if (state.equals(DAGAppMasterState.IDLE) || taskRequests.isEmpty()) {
+    if (state.equals(AMState.IDLE) || taskRequests.isEmpty()) {
       // reset locality level on held container
       // if sessionDelay defined, push back into delayed queue if not already
       // done so
 
       // Compute min held containers.
-      if (appContext.isSession() && sessionNumMinHeldContainers > 0 &&
+      if (getContext().isSession() && sessionNumMinHeldContainers > 0 &&
           sessionMinHeldContainers.isEmpty()) {
         // session mode and need to hold onto containers and not done so already
         determineMinHeldContainers();
@@ -626,7 +595,7 @@ public class YarnTaskSchedulerService extends TaskSchedulerService
           && idleContainerTimeoutMin != -1)) {
         // container idle timeout has expired or is a new unused container. 
         // new container is possibly a spurious race condition allocation.
-        if (appContext.isSession()
+        if (getContext().isSession()
             && sessionMinHeldContainers.contains(heldContainer.getContainer().getId())) {
           // There are no outstanding requests. So its safe to hold new containers.
           // We may have received more containers than necessary and some are unused
@@ -667,7 +636,7 @@ public class YarnTaskSchedulerService extends TaskSchedulerService
             heldContainer.getContainer(), currentTime
                 + localitySchedulingDelay);        
       }
-    } else if (state.equals(DAGAppMasterState.RUNNING)) {
+    } else if (state.equals(AMState.RUNNING_APP)) {
       // clear min held containers since we need to allocate to tasks
       if (!sessionMinHeldContainers.isEmpty()) {
         // update the expire time of min held containers so that they are
@@ -806,12 +775,12 @@ public class YarnTaskSchedulerService extends TaskSchedulerService
             // Are there any pending requests at any priority?
             // release if there are tasks or this is not a session
             if (safeToRelease && 
-                (!taskRequests.isEmpty() || !appContext.isSession())) {
+                (!taskRequests.isEmpty() || !getContext().isSession())) {
               LOG.info("Releasing held container as either there are pending but "
                 + " unmatched requests or this is not a session"
                 + ", containerId=" + heldContainer.container.getId()
                 + ", pendingTasks=" + taskRequests.size()
-                + ", isSession=" + appContext.isSession()
+                + ", isSession=" + getContext().isSession()
                 + ". isNew=" + isNew);
               releaseUnassignedContainers(
                 Lists.newArrayList(heldContainer.container));
@@ -862,7 +831,7 @@ public class YarnTaskSchedulerService extends TaskSchedulerService
       return;
     }
     // upcall to app must be outside locks
-    appClientDelegate.appShutdownRequested();
+    getContext().appShutdownRequested();
   }
 
   @Override
@@ -872,7 +841,7 @@ public class YarnTaskSchedulerService extends TaskSchedulerService
     }
     // ignore bad nodes for now
     // upcall to app must be outside locks
-    appClientDelegate.nodesUpdated(updatedNodes);
+    getContext().nodesUpdated(updatedNodes);
   }
 
   @Override
@@ -894,7 +863,7 @@ public class YarnTaskSchedulerService extends TaskSchedulerService
     numHeartbeats++;
     preemptIfNeeded();
 
-    return appClientDelegate.getProgress();
+    return getContext().getProgress();
   }
 
   @Override
@@ -902,7 +871,7 @@ public class YarnTaskSchedulerService extends TaskSchedulerService
     if (isStopped.get()) {
       return;
     }
-    appClientDelegate.onError(t);
+    getContext().onError(t);
   }
 
   @Override
@@ -1289,7 +1258,7 @@ public class YarnTaskSchedulerService extends TaskSchedulerService
         ContainerId cId = preemptedContainers[i];
         if (cId != null) {
           LOG.info("Preempting container: " + cId + " currently allocated to a task.");
-          appClientDelegate.preemptContainer(cId);
+          getContext().preemptContainer(cId);
         }
       }
     }
@@ -1422,7 +1391,7 @@ public class YarnTaskSchedulerService extends TaskSchedulerService
     Object assignedTask = containerAssignments.remove(containerId);
     if (assignedTask != null) {
       // A task was assigned to this container at some point. Inform the app.
-      appClientDelegate.containerBeingReleased(containerId);
+      getContext().containerBeingReleased(containerId);
     }
     HeldContainer delayedContainer = heldContainers.remove(containerId);
     if (delayedContainer != null) {
@@ -1626,7 +1595,7 @@ public class YarnTaskSchedulerService extends TaskSchedulerService
 
   private void informAppAboutAssignment(CookieContainerRequest assigned,
       Container container) {
-    appClientDelegate.taskAllocated(getTask(assigned),
+    getContext().taskAllocated(getTask(assigned),
         assigned.getCookie().getAppCookie(), container);
   }
 

http://git-wip-us.apache.org/repos/asf/tez/blob/82c24ac0/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 5cff766..aeacf84 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
@@ -29,6 +29,7 @@ import java.util.concurrent.locks.ReentrantReadWriteLock.WriteLock;
 
 import org.apache.tez.common.TezUtilsInternal;
 import org.apache.tez.serviceplugins.api.ContainerEndReason;
+import org.apache.tez.common.ContainerSignatureMatcher;
 import org.apache.tez.serviceplugins.api.TaskAttemptEndReason;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;

http://git-wip-us.apache.org/repos/asf/tez/blob/82c24ac0/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 938096d..fcb9eaf 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
@@ -22,6 +22,7 @@ import java.util.Collection;
 import java.util.concurrent.ConcurrentHashMap;
 
 import org.apache.tez.dag.app.dag.DAG;
+import org.apache.tez.common.ContainerSignatureMatcher;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.service.AbstractService;

http://git-wip-us.apache.org/repos/asf/tez/blob/82c24ac0/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/ContainerContextMatcher.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/ContainerContextMatcher.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/ContainerContextMatcher.java
index 211c537..436f098 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/ContainerContextMatcher.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/ContainerContextMatcher.java
@@ -26,6 +26,7 @@ import org.apache.hadoop.yarn.api.records.LocalResource;
 import org.apache.tez.dag.app.ContainerContext;
 
 import com.google.common.base.Preconditions;
+import org.apache.tez.common.ContainerSignatureMatcher;
 
 public class ContainerContextMatcher implements ContainerSignatureMatcher {
 

http://git-wip-us.apache.org/repos/asf/tez/blob/82c24ac0/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/ContainerSignatureMatcher.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/ContainerSignatureMatcher.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/ContainerSignatureMatcher.java
deleted file mode 100644
index 0f9c2d6..0000000
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/ContainerSignatureMatcher.java
+++ /dev/null
@@ -1,60 +0,0 @@
-/* Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF 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.container;
-
-import java.util.Map;
-
-import org.apache.hadoop.yarn.api.records.LocalResource;
-
-public interface ContainerSignatureMatcher {
-  /**
-   * Checks the compatibility between the specified container signatures.
-   *
-   * @return true if the first signature is a super set of the second
-   *         signature.
-   */
-  public boolean isSuperSet(Object cs1, Object cs2);
-  
-  /**
-   * Checks if the container signatures match exactly
-   * @return true if exact match
-   */
-  public boolean isExactMatch(Object cs1, Object cs2);
-  
-  /**
-   * Gets additional resources specified in lr2, which are not present for lr1
-   * 
-   * @param lr1
-   * @param lr2
-   * @return additional resources specified in lr2, which are not present for lr1
-   */
-  public Map<String, LocalResource> getAdditionalResources(Map<String, LocalResource> lr1,
-      Map<String, LocalResource> lr2);
-
-
-  /**
-   * Do a union of 2 signatures
-   * Pre-condition. This function should only be invoked iff cs1 is compatible with cs2.
-   * i.e. isSuperSet should not return false.
-   * @param cs1 Signature 1 Original signature
-   * @param cs2 Signature 2 New signature
-   * @return Union of 2 signatures
-   */
-  public Object union(Object cs1, Object cs2);
-
-}

http://git-wip-us.apache.org/repos/asf/tez/blob/82c24ac0/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 e37ab4a..88f6066 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,7 +22,6 @@ 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;
@@ -46,7 +45,6 @@ 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.FinalApplicationStatus;
 import org.apache.hadoop.yarn.api.records.LocalResource;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.Priority;
@@ -68,16 +66,14 @@ import org.apache.tez.dag.app.ContainerHeartbeatHandler;
 import org.apache.tez.dag.app.DAGAppMasterState;
 import org.apache.tez.dag.app.TaskAttemptListener;
 import org.apache.tez.dag.app.dag.TaskAttempt;
-import org.apache.tez.dag.app.rm.TaskSchedulerService.TaskSchedulerAppCallback;
-import org.apache.tez.dag.app.rm.TaskSchedulerService.TaskSchedulerAppCallback.AppFinalStatus;
 import org.apache.tez.dag.app.rm.YarnTaskSchedulerService.CookieContainerRequest;
 import org.apache.tez.dag.app.rm.TestTaskSchedulerHelpers.AMRMClientAsyncForTest;
 import org.apache.tez.dag.app.rm.TestTaskSchedulerHelpers.AMRMClientForTest;
 import org.apache.tez.dag.app.rm.TestTaskSchedulerHelpers.AlwaysMatchesContainerMatcher;
 import org.apache.tez.dag.app.rm.TestTaskSchedulerHelpers.CapturingEventHandler;
-import org.apache.tez.dag.app.rm.TestTaskSchedulerHelpers.TaskSchedulerAppCallbackDrainable;
+import org.apache.tez.dag.app.rm.TestTaskSchedulerHelpers.TaskSchedulerContextDrainable;
 import org.apache.tez.dag.app.rm.TestTaskSchedulerHelpers.TaskSchedulerEventHandlerForTest;
-import org.apache.tez.dag.app.rm.TestTaskSchedulerHelpers.TaskSchedulerWithDrainableAppCallback;
+import org.apache.tez.dag.app.rm.TestTaskSchedulerHelpers.TaskSchedulerWithDrainableContext;
 import org.apache.tez.dag.app.rm.container.AMContainerEventAssignTA;
 import org.apache.tez.dag.app.rm.container.AMContainerEventStopRequest;
 import org.apache.tez.dag.app.rm.container.AMContainerMap;
@@ -116,14 +112,13 @@ public class TestContainerReuse {
     conf.setBoolean(
       TezConfiguration.TEZ_AM_CONTAINER_REUSE_ENABLED, true);
     conf.setBoolean(
-      TezConfiguration.TEZ_AM_CONTAINER_REUSE_RACK_FALLBACK_ENABLED, false);
+        TezConfiguration.TEZ_AM_CONTAINER_REUSE_RACK_FALLBACK_ENABLED, false);
     conf.setBoolean(
-      TezConfiguration.TEZ_AM_CONTAINER_REUSE_NON_LOCAL_FALLBACK_ENABLED, false);
+        TezConfiguration.TEZ_AM_CONTAINER_REUSE_NON_LOCAL_FALLBACK_ENABLED, false);
     conf.setLong(
       TezConfiguration.TEZ_AM_CONTAINER_REUSE_LOCALITY_DELAY_ALLOCATION_MILLIS, 3000l);
     conf.setLong(TezConfiguration.TEZ_AM_CONTAINER_IDLE_RELEASE_TIMEOUT_MIN_MILLIS, 0);
     conf.setLong(TezConfiguration.TEZ_AM_CONTAINER_IDLE_RELEASE_TIMEOUT_MAX_MILLIS, 0);
-    TaskSchedulerAppCallback mockApp = mock(TaskSchedulerAppCallback.class);
 
     CapturingEventHandler eventHandler = new CapturingEventHandler();
     TezDAGID dagID = TezDAGID.getInstance("0", 0, 0);
@@ -132,12 +127,6 @@ public class TestContainerReuse {
     AMRMClient<CookieContainerRequest> rmClientCore = new AMRMClientForTest();
     TezAMRMClientAsync<CookieContainerRequest> rmClient =
       spy(new AMRMClientAsyncForTest(rmClientCore, 100));
-    String appUrl = "url";
-    String appMsg = "success";
-    AppFinalStatus finalStatus =
-        new AppFinalStatus(FinalApplicationStatus.SUCCEEDED, appMsg, appUrl);
-
-    doReturn(finalStatus).when(mockApp).getFinalAppStatus();
 
     AppContext appContext = mock(AppContext.class);
     doReturn(conf).when(appContext).getAMConf();
@@ -161,11 +150,11 @@ public class TestContainerReuse {
     taskSchedulerEventHandler.init(conf);
     taskSchedulerEventHandler.start();
 
-    TaskSchedulerWithDrainableAppCallback taskScheduler =
-      (TaskSchedulerWithDrainableAppCallback)
+    TaskSchedulerWithDrainableContext taskScheduler =
+      (TaskSchedulerWithDrainableContext)
         ((TaskSchedulerEventHandlerForTest) taskSchedulerEventHandler)
         .getSpyTaskScheduler();
-    TaskSchedulerAppCallbackDrainable drainableAppCallback =
+    TaskSchedulerContextDrainable drainableAppCallback =
       taskScheduler.getDrainableAppCallback();
 
     AtomicBoolean drainNotifier = new AtomicBoolean(false);
@@ -251,8 +240,7 @@ public class TestContainerReuse {
       }
     }
     assertTrue("containerHost2 was not released", exception == null);
-    taskScheduler.stop();
-    taskScheduler.close();
+    taskScheduler.shutdown();
     taskSchedulerEventHandler.close();
   }
 
@@ -267,7 +255,6 @@ public class TestContainerReuse {
     conf.setLong(TezConfiguration.TEZ_AM_CONTAINER_REUSE_LOCALITY_DELAY_ALLOCATION_MILLIS, 1000l);
     conf.setLong(TezConfiguration.TEZ_AM_CONTAINER_IDLE_RELEASE_TIMEOUT_MIN_MILLIS, 0);
     conf.setLong(TezConfiguration.TEZ_AM_CONTAINER_IDLE_RELEASE_TIMEOUT_MAX_MILLIS, 0);
-    TaskSchedulerAppCallback mockApp = mock(TaskSchedulerAppCallback.class);
 
     CapturingEventHandler eventHandler = new CapturingEventHandler();
     TezDAGID dagID = TezDAGID.getInstance("0", 0, 0);
@@ -276,12 +263,6 @@ public class TestContainerReuse {
     AMRMClient<CookieContainerRequest> rmClientCore = new AMRMClientForTest();
     TezAMRMClientAsync<CookieContainerRequest> rmClient =
       spy(new AMRMClientAsyncForTest(rmClientCore, 100));
-    String appUrl = "url";
-    String appMsg = "success";
-    AppFinalStatus finalStatus =
-        new AppFinalStatus(FinalApplicationStatus.SUCCEEDED, appMsg, appUrl);
-
-    doReturn(finalStatus).when(mockApp).getFinalAppStatus();
 
     AppContext appContext = mock(AppContext.class);
     doReturn(new Configuration(false)).when(appContext).getAMConf();
@@ -304,11 +285,11 @@ public class TestContainerReuse {
     taskSchedulerEventHandler.init(conf);
     taskSchedulerEventHandler.start();
 
-    TaskSchedulerWithDrainableAppCallback taskScheduler =
-      (TaskSchedulerWithDrainableAppCallback)
+    TaskSchedulerWithDrainableContext taskScheduler =
+      (TaskSchedulerWithDrainableContext)
         ((TaskSchedulerEventHandlerForTest) taskSchedulerEventHandler)
           .getSpyTaskScheduler();
-    TaskSchedulerAppCallbackDrainable drainableAppCallback =
+    TaskSchedulerContextDrainable drainableAppCallback =
       taskScheduler.getDrainableAppCallback();
     
     AtomicBoolean drainNotifier = new AtomicBoolean(false);
@@ -366,8 +347,7 @@ public class TestContainerReuse {
       eq(containerHost2.getId()));
     eventHandler.verifyInvocation(AMContainerEventStopRequest.class);
 
-    taskScheduler.stop();
-    taskScheduler.close();
+    taskScheduler.shutdown();
     taskSchedulerEventHandler.close();
   }
 
@@ -380,19 +360,12 @@ public class TestContainerReuse {
     tezConf.setLong(TezConfiguration.TEZ_AM_CONTAINER_REUSE_LOCALITY_DELAY_ALLOCATION_MILLIS, 0);
     tezConf.setLong(TezConfiguration.TEZ_AM_CONTAINER_IDLE_RELEASE_TIMEOUT_MIN_MILLIS, 0);
     tezConf.setLong(TezConfiguration.TEZ_AM_CONTAINER_IDLE_RELEASE_TIMEOUT_MAX_MILLIS, 0);
-    TaskSchedulerAppCallback mockApp = mock(TaskSchedulerAppCallback.class);
 
     CapturingEventHandler eventHandler = new CapturingEventHandler();
     TezDAGID dagID = TezDAGID.getInstance("0", 0, 0);
 
     AMRMClient<CookieContainerRequest> rmClientCore = new AMRMClientForTest();
     TezAMRMClientAsync<CookieContainerRequest> rmClient = spy(new AMRMClientAsyncForTest(rmClientCore, 100));
-    String appUrl = "url";
-    String appMsg = "success";
-    AppFinalStatus finalStatus =
-        new AppFinalStatus(FinalApplicationStatus.SUCCEEDED, appMsg, appUrl);
-
-    doReturn(finalStatus).when(mockApp).getFinalAppStatus();
 
     AppContext appContext = mock(AppContext.class);
     doReturn(new Configuration(false)).when(appContext).getAMConf();
@@ -410,9 +383,9 @@ public class TestContainerReuse {
     taskSchedulerEventHandler.init(tezConf);
     taskSchedulerEventHandler.start();
 
-    TaskSchedulerWithDrainableAppCallback taskScheduler = (TaskSchedulerWithDrainableAppCallback) ((TaskSchedulerEventHandlerForTest) taskSchedulerEventHandler)
+    TaskSchedulerWithDrainableContext taskScheduler = (TaskSchedulerWithDrainableContext) ((TaskSchedulerEventHandlerForTest) taskSchedulerEventHandler)
         .getSpyTaskScheduler();
-    TaskSchedulerAppCallbackDrainable drainableAppCallback = taskScheduler.getDrainableAppCallback();
+    TaskSchedulerContextDrainable drainableAppCallback = taskScheduler.getDrainableAppCallback();
     AtomicBoolean drainNotifier = new AtomicBoolean(false);
     taskScheduler.delayedContainerManager.drainedDelayedContainersForTest = drainNotifier;
 
@@ -504,7 +477,7 @@ public class TestContainerReuse {
     eventHandler.verifyInvocation(AMContainerEventStopRequest.class);
     eventHandler.reset();
 
-    taskScheduler.close();
+    taskScheduler.shutdown();
     taskSchedulerEventHandler.close();
   }
 
@@ -522,19 +495,11 @@ public class TestContainerReuse {
     tezConf.set(TezConfiguration.TEZ_TASK_SPECIFIC_LAUNCH_CMD_OPTS, "dir=/tmp/__VERTEX_NAME__/__TASK_INDEX__");
     TaskSpecificLaunchCmdOption taskSpecificLaunchCmdOption =  new TaskSpecificLaunchCmdOption(tezConf);
 
-    TaskSchedulerAppCallback mockApp = mock(TaskSchedulerAppCallback.class);
-
     CapturingEventHandler eventHandler = new CapturingEventHandler();
     TezDAGID dagID = TezDAGID.getInstance("0", 0, 0);
 
     AMRMClient<CookieContainerRequest> rmClientCore = new AMRMClientForTest();
     TezAMRMClientAsync<CookieContainerRequest> rmClient = spy(new AMRMClientAsyncForTest(rmClientCore, 100));
-    String appUrl = "url";
-    String appMsg = "success";
-    AppFinalStatus finalStatus =
-        new AppFinalStatus(FinalApplicationStatus.SUCCEEDED, appMsg, appUrl);
-
-    doReturn(finalStatus).when(mockApp).getFinalAppStatus();
 
     AppContext appContext = mock(AppContext.class);
     doReturn(new Configuration(false)).when(appContext).getAMConf();
@@ -554,10 +519,10 @@ public class TestContainerReuse {
     taskSchedulerEventHandler.init(tezConf);
     taskSchedulerEventHandler.start();
 
-    TaskSchedulerWithDrainableAppCallback taskScheduler =
-        (TaskSchedulerWithDrainableAppCallback) ((TaskSchedulerEventHandlerForTest) taskSchedulerEventHandler)
+    TaskSchedulerWithDrainableContext taskScheduler =
+        (TaskSchedulerWithDrainableContext) ((TaskSchedulerEventHandlerForTest) taskSchedulerEventHandler)
           .getSpyTaskScheduler();
-    TaskSchedulerAppCallbackDrainable drainableAppCallback = taskScheduler.getDrainableAppCallback();
+    TaskSchedulerContextDrainable drainableAppCallback = taskScheduler.getDrainableAppCallback();
     AtomicBoolean drainNotifier = new AtomicBoolean(false);
     taskScheduler.delayedContainerManager.drainedDelayedContainersForTest = drainNotifier;
 
@@ -705,7 +670,7 @@ public class TestContainerReuse {
     verify(taskSchedulerEventHandler).taskAllocated(eq(0), eq(ta16), any(Object.class), eq(container3));
     eventHandler.reset();
 
-    taskScheduler.close();
+    taskScheduler.shutdown();
     taskSchedulerEventHandler.close();
   }
 
@@ -721,20 +686,12 @@ public class TestContainerReuse {
     tezConf.setLong(TezConfiguration.TEZ_AM_CONTAINER_IDLE_RELEASE_TIMEOUT_MIN_MILLIS, 1000l);
     tezConf.setLong(TezConfiguration.TEZ_AM_CONTAINER_IDLE_RELEASE_TIMEOUT_MAX_MILLIS, 1000l);
 
-    TaskSchedulerAppCallback mockApp = mock(TaskSchedulerAppCallback.class);
-
     CapturingEventHandler eventHandler = new CapturingEventHandler();
     TezDAGID dagID = TezDAGID.getInstance("0", 0, 0);
 
     AMRMClient<CookieContainerRequest> rmClientCore = new AMRMClientForTest();
     TezAMRMClientAsync<CookieContainerRequest> rmClient =
         spy(new AMRMClientAsyncForTest(rmClientCore, 100));
-    String appUrl = "url";
-    String appMsg = "success";
-    AppFinalStatus finalStatus =
-        new AppFinalStatus(FinalApplicationStatus.SUCCEEDED, appMsg, appUrl);
-
-    doReturn(finalStatus).when(mockApp).getFinalAppStatus();
 
     AppContext appContext = mock(AppContext.class);
     doReturn(new Configuration(false)).when(appContext).getAMConf();
@@ -758,11 +715,11 @@ public class TestContainerReuse {
     taskSchedulerEventHandler.init(tezConf);
     taskSchedulerEventHandler.start();
 
-    TaskSchedulerWithDrainableAppCallback taskScheduler =
-      (TaskSchedulerWithDrainableAppCallback)
+    TaskSchedulerWithDrainableContext taskScheduler =
+      (TaskSchedulerWithDrainableContext)
         ((TaskSchedulerEventHandlerForTest) taskSchedulerEventHandler)
         .getSpyTaskScheduler();
-    TaskSchedulerAppCallbackDrainable drainableAppCallback =
+    TaskSchedulerContextDrainable drainableAppCallback =
       taskScheduler.getDrainableAppCallback();
     AtomicBoolean drainNotifier = new AtomicBoolean(false);
     taskScheduler.delayedContainerManager.drainedDelayedContainersForTest = drainNotifier;
@@ -836,7 +793,7 @@ public class TestContainerReuse {
     verify(rmClient).releaseAssignedContainer(eq(container1.getId()));
     eventHandler.verifyInvocation(AMContainerEventStopRequest.class);
 
-    taskScheduler.close();
+    taskScheduler.shutdown();
     taskSchedulerEventHandler.close();
   }
 
@@ -853,20 +810,12 @@ public class TestContainerReuse {
     tezConf.setInt(
         TezConfiguration.TEZ_AM_SESSION_MIN_HELD_CONTAINERS, 1);
 
-    TaskSchedulerAppCallback mockApp = mock(TaskSchedulerAppCallback.class);
-
     CapturingEventHandler eventHandler = new CapturingEventHandler();
     TezDAGID dagID = TezDAGID.getInstance("0", 0, 0);
 
     AMRMClient<CookieContainerRequest> rmClientCore = new AMRMClientForTest();
     TezAMRMClientAsync<CookieContainerRequest> rmClient =
       spy(new AMRMClientAsyncForTest(rmClientCore, 100));
-    String appUrl = "url";
-    String appMsg = "success";
-    AppFinalStatus finalStatus =
-        new AppFinalStatus(FinalApplicationStatus.SUCCEEDED, appMsg, appUrl);
-
-    doReturn(finalStatus).when(mockApp).getFinalAppStatus();
 
     AppContext appContext = mock(AppContext.class);
     doReturn(new Configuration(false)).when(appContext).getAMConf();
@@ -890,11 +839,11 @@ public class TestContainerReuse {
     taskSchedulerEventHandler.init(tezConf);
     taskSchedulerEventHandler.start();
 
-    TaskSchedulerWithDrainableAppCallback taskScheduler =
-      (TaskSchedulerWithDrainableAppCallback)
+    TaskSchedulerWithDrainableContext taskScheduler =
+      (TaskSchedulerWithDrainableContext)
         ((TaskSchedulerEventHandlerForTest) taskSchedulerEventHandler)
           .getSpyTaskScheduler();
-    TaskSchedulerAppCallbackDrainable drainableAppCallback = taskScheduler.getDrainableAppCallback();
+    TaskSchedulerContextDrainable drainableAppCallback = taskScheduler.getDrainableAppCallback();
 
     AtomicBoolean drainNotifier = new AtomicBoolean(false);
     taskScheduler.delayedContainerManager.drainedDelayedContainersForTest = drainNotifier;
@@ -965,7 +914,7 @@ public class TestContainerReuse {
     // container should not get released due to min held containers
     verify(rmClient, times(0)).releaseAssignedContainer(eq(container1.getId()));
 
-    taskScheduler.close();
+    taskScheduler.shutdown();
     taskSchedulerEventHandler.close();
   }
   
@@ -979,19 +928,11 @@ public class TestContainerReuse {
     tezConf.setLong(TezConfiguration.TEZ_AM_CONTAINER_REUSE_LOCALITY_DELAY_ALLOCATION_MILLIS, 0);
     tezConf.setLong(TezConfiguration.TEZ_AM_CONTAINER_IDLE_RELEASE_TIMEOUT_MIN_MILLIS, -1);
 
-    TaskSchedulerAppCallback mockApp = mock(TaskSchedulerAppCallback.class);
-
     CapturingEventHandler eventHandler = new CapturingEventHandler();
     TezDAGID dagID1 = TezDAGID.getInstance("0", 1, 0);
 
     AMRMClient<CookieContainerRequest> rmClientCore = new AMRMClientForTest();
     TezAMRMClientAsync<CookieContainerRequest> rmClient = spy(new AMRMClientAsyncForTest(rmClientCore, 100));
-    String appUrl = "url";
-    String appMsg = "success";
-    AppFinalStatus finalStatus =
-        new AppFinalStatus(FinalApplicationStatus.SUCCEEDED, appMsg, appUrl);
-
-    doReturn(finalStatus).when(mockApp).getFinalAppStatus();
 
     AppContext appContext = mock(AppContext.class);
     doReturn(new Configuration(false)).when(appContext).getAMConf();
@@ -1011,9 +952,9 @@ public class TestContainerReuse {
     taskSchedulerEventHandler.init(tezConf);
     taskSchedulerEventHandler.start();
 
-    TaskSchedulerWithDrainableAppCallback taskScheduler = (TaskSchedulerWithDrainableAppCallback) ((TaskSchedulerEventHandlerForTest) taskSchedulerEventHandler)
+    TaskSchedulerWithDrainableContext taskScheduler = (TaskSchedulerWithDrainableContext) ((TaskSchedulerEventHandlerForTest) taskSchedulerEventHandler)
         .getSpyTaskScheduler();
-    TaskSchedulerAppCallbackDrainable drainableAppCallback = taskScheduler.getDrainableAppCallback();
+    TaskSchedulerContextDrainable drainableAppCallback = taskScheduler.getDrainableAppCallback();
     AtomicBoolean drainNotifier = new AtomicBoolean(false);
     taskScheduler.delayedContainerManager.drainedDelayedContainersForTest = drainNotifier;
 
@@ -1129,7 +1070,7 @@ public class TestContainerReuse {
     assertEquals(2, assignEvent.getRemoteTaskLocalResources().size());
     eventHandler.reset();
 
-    taskScheduler.close();
+    taskScheduler.shutdown();
     taskSchedulerEventHandler.close();
   }
 
@@ -1143,19 +1084,11 @@ public class TestContainerReuse {
     tezConf.setLong(TezConfiguration.TEZ_AM_CONTAINER_REUSE_LOCALITY_DELAY_ALLOCATION_MILLIS, 0);
     tezConf.setLong(TezConfiguration.TEZ_AM_CONTAINER_IDLE_RELEASE_TIMEOUT_MIN_MILLIS, -1);
 
-    TaskSchedulerAppCallback mockApp = mock(TaskSchedulerAppCallback.class);
-
     CapturingEventHandler eventHandler = new CapturingEventHandler();
     TezDAGID dagID1 = TezDAGID.getInstance("0", 1, 0);
 
     AMRMClient<CookieContainerRequest> rmClientCore = new AMRMClientForTest();
     TezAMRMClientAsync<CookieContainerRequest> rmClient = spy(new AMRMClientAsyncForTest(rmClientCore, 100));
-    String appUrl = "url";
-    String appMsg = "success";
-    AppFinalStatus finalStatus =
-        new AppFinalStatus(FinalApplicationStatus.SUCCEEDED, appMsg, appUrl);
-
-    doReturn(finalStatus).when(mockApp).getFinalAppStatus();
 
     AppContext appContext = mock(AppContext.class);
     doReturn(new Configuration(false)).when(appContext).getAMConf();
@@ -1177,9 +1110,9 @@ public class TestContainerReuse {
     taskSchedulerEventHandler.init(tezConf);
     taskSchedulerEventHandler.start();
 
-    TaskSchedulerWithDrainableAppCallback taskScheduler = (TaskSchedulerWithDrainableAppCallback) ((TaskSchedulerEventHandlerForTest) taskSchedulerEventHandler)
+    TaskSchedulerWithDrainableContext taskScheduler = (TaskSchedulerWithDrainableContext) ((TaskSchedulerEventHandlerForTest) taskSchedulerEventHandler)
         .getSpyTaskScheduler();
-    TaskSchedulerAppCallbackDrainable drainableAppCallback = taskScheduler.getDrainableAppCallback();
+    TaskSchedulerContextDrainable drainableAppCallback = taskScheduler.getDrainableAppCallback();
     AtomicBoolean drainNotifier = new AtomicBoolean(false);
     taskScheduler.delayedContainerManager.drainedDelayedContainersForTest = drainNotifier;
 
@@ -1291,7 +1224,7 @@ public class TestContainerReuse {
     verify(taskSchedulerEventHandler).taskAllocated(eq(0), eq(ta211), any(Object.class), eq(container2));
     eventHandler.reset();
 
-    taskScheduler.close();
+    taskScheduler.shutdown();
     taskSchedulerEventHandler.close();
   }
 
@@ -1305,19 +1238,12 @@ public class TestContainerReuse {
     tezConf.setLong(TezConfiguration.TEZ_AM_CONTAINER_REUSE_LOCALITY_DELAY_ALLOCATION_MILLIS, 0);
     tezConf.setLong(TezConfiguration.TEZ_AM_CONTAINER_IDLE_RELEASE_TIMEOUT_MIN_MILLIS, 0);
     tezConf.setLong(TezConfiguration.TEZ_AM_CONTAINER_IDLE_RELEASE_TIMEOUT_MAX_MILLIS, 0);
-    TaskSchedulerAppCallback mockApp = mock(TaskSchedulerAppCallback.class);
 
     CapturingEventHandler eventHandler = new CapturingEventHandler();
     TezDAGID dagID = TezDAGID.getInstance("0", 0, 0);
 
     AMRMClient<CookieContainerRequest> rmClientCore = new AMRMClientForTest();
     TezAMRMClientAsync<CookieContainerRequest> rmClient = spy(new AMRMClientAsyncForTest(rmClientCore, 100));
-    String appUrl = "url";
-    String appMsg = "success";
-    AppFinalStatus finalStatus =
-        new AppFinalStatus(FinalApplicationStatus.SUCCEEDED, appMsg, appUrl);
-
-    doReturn(finalStatus).when(mockApp).getFinalAppStatus();
 
     AppContext appContext = mock(AppContext.class);
     doReturn(new Configuration(false)).when(appContext).getAMConf();
@@ -1326,7 +1252,7 @@ public class TestContainerReuse {
     AMNodeTracker amNodeTracker = new AMNodeTracker(eventHandler, appContext);
     doReturn(amContainerMap).when(appContext).getAllContainers();
     doReturn(amNodeTracker).when(appContext).getNodeTracker();
-    doReturn(DAGAppMasterState.RUNNING).when(appContext).getAMState();
+    doReturn(DAGAppMasterState.SUCCEEDED).when(appContext).getAMState();
     doReturn(true).when(appContext).isAMInCompletionState();
     doReturn(dagID).when(appContext).getCurrentDAGID();
     doReturn(mock(ClusterInfo.class)).when(appContext).getClusterInfo();
@@ -1338,10 +1264,10 @@ public class TestContainerReuse {
     taskSchedulerEventHandler.init(tezConf);
     taskSchedulerEventHandler.start();
 
-    TaskSchedulerWithDrainableAppCallback taskScheduler = (TaskSchedulerWithDrainableAppCallback)
+    TaskSchedulerWithDrainableContext taskScheduler = (TaskSchedulerWithDrainableContext)
         ((TaskSchedulerEventHandlerForTest) taskSchedulerEventHandler)
         .getSpyTaskScheduler();
-    TaskSchedulerAppCallbackDrainable drainableAppCallback = taskScheduler.getDrainableAppCallback();
+    TaskSchedulerContextDrainable drainableAppCallback = taskScheduler.getDrainableAppCallback();
     AtomicBoolean drainNotifier = new AtomicBoolean(false);
     taskScheduler.delayedContainerManager.drainedDelayedContainersForTest = drainNotifier;
 
@@ -1369,7 +1295,7 @@ public class TestContainerReuse {
     drainableAppCallback.drain();
     verify(taskSchedulerEventHandler, times(0)).taskAllocated(eq(0), eq(ta11),
         any(Object.class), eq(container1));
-    taskScheduler.close();
+    taskScheduler.shutdown();
     taskSchedulerEventHandler.close();
   }
 

http://git-wip-us.apache.org/repos/asf/tez/blob/82c24ac0/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 12390b2..2ada2f1 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
@@ -18,12 +18,12 @@
 
 package org.apache.tez.dag.app.rm;
 
-import static org.mockito.Mockito.doReturn;
-import static org.mockito.Mockito.mock;
 import java.util.HashMap;
 import java.util.LinkedHashMap;
 import java.util.concurrent.PriorityBlockingQueue;
 
+import org.apache.hadoop.conf.Configuration;
+import org.apache.tez.serviceplugins.api.TaskSchedulerContext;
 import org.junit.Assert;
 import org.junit.Test;
 
@@ -33,24 +33,12 @@ import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.Priority;
 
 import org.apache.tez.dag.api.TezConfiguration;
-import org.apache.tez.dag.app.AppContext;
 import org.apache.tez.dag.app.rm.LocalTaskSchedulerService.AsyncDelegateRequestHandler;
 import org.apache.tez.dag.app.rm.LocalTaskSchedulerService.LocalContainerFactory;
 import org.apache.tez.dag.app.rm.LocalTaskSchedulerService.TaskRequest;
-import org.apache.tez.dag.app.rm.TaskSchedulerService.TaskSchedulerAppCallback;
 
 public class TestLocalTaskScheduler {
 
-  public AppContext createMockAppContext() {
-
-    ApplicationId appId = ApplicationId.newInstance(2000, 1);
-    ApplicationAttemptId appAttemptId = ApplicationAttemptId.newInstance(appId, 1);
-
-    AppContext appContext = mock(AppContext.class);
-    doReturn(appAttemptId).when(appContext).getApplicationAttemptId();
-
-    return appContext;
-  }
 
   @Test(timeout = 5000)
   public void maxTasksAllocationsCannotBeExceeded() {
@@ -59,17 +47,24 @@ public class TestLocalTaskScheduler {
     TezConfiguration tezConf = new TezConfiguration();
     tezConf.setInt(TezConfiguration.TEZ_AM_INLINE_TASK_EXECUTION_MAX_TASKS, MAX_TASKS);
 
-    LocalContainerFactory containerFactory = new LocalContainerFactory(createMockAppContext(), 1000);
+    ApplicationId appId = ApplicationId.newInstance(2000, 1);
+    ApplicationAttemptId appAttemptId = ApplicationAttemptId.newInstance(appId, 1);
+
+    TaskSchedulerContext
+        mockContext = TestTaskSchedulerHelpers.setupMockTaskSchedulerContext("", 0, "", true,
+        appAttemptId, 1000l, null, new Configuration());
+
+    LocalContainerFactory containerFactory = new LocalContainerFactory(appAttemptId, 1000);
+
     HashMap<Object, Container> taskAllocations = new LinkedHashMap<Object, Container>();
     PriorityBlockingQueue<TaskRequest> taskRequestQueue = new PriorityBlockingQueue<TaskRequest>();
-    TaskSchedulerAppCallback appClientDelegate = mock(TaskSchedulerAppCallback.class);
 
     // Object under test
     AsyncDelegateRequestHandler requestHandler =
       new AsyncDelegateRequestHandler(taskRequestQueue,
           containerFactory,
           taskAllocations,
-          appClientDelegate,
+          mockContext,
           tezConf);
 
     // Allocate up to max tasks

http://git-wip-us.apache.org/repos/asf/tez/blob/82c24ac0/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 b555c62..c637f5f 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,11 +27,9 @@ 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.app.AppContext;
 import org.apache.tez.dag.app.dag.Task;
-import org.apache.tez.dag.app.rm.TaskSchedulerService.TaskSchedulerAppCallback;
 import org.apache.tez.dag.app.rm.TestLocalTaskSchedulerService.MockLocalTaskSchedulerSerivce.MockAsyncDelegateRequestHandler;
-import org.apache.tez.dag.app.rm.container.ContainerSignatureMatcher;
+import org.apache.tez.serviceplugins.api.TaskSchedulerContext;
 import org.junit.Assert;
 import org.junit.Test;
 
@@ -82,14 +80,15 @@ public class TestLocalTaskSchedulerService {
    * Normal flow of TaskAttempt
    */
   @Test(timeout = 5000)
-  public void testDeallocationBeforeAllocation() {
-    AppContext appContext = mock(AppContext.class);
+  public void testDeallocationBeforeAllocation() throws InterruptedException {
     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, "", appContext);
-    taskSchedulerService.init(new Configuration());
+
+    TaskSchedulerContext mockContext = TestTaskSchedulerHelpers
+        .setupMockTaskSchedulerContext("", 0, "", false, appAttemptId, 10000l, null, new Configuration());
+
+    MockLocalTaskSchedulerSerivce taskSchedulerService = new MockLocalTaskSchedulerSerivce(mockContext);
+    taskSchedulerService.initialize();
     taskSchedulerService.start();
 
     Task task = mock(Task.class);
@@ -103,21 +102,24 @@ public class TestLocalTaskSchedulerService {
     assertEquals(1, requestHandler.deallocateCount);
     // The corresponding AllocateTaskRequest will be removed, so won't been processed.
     assertEquals(0, requestHandler.allocateCount);
-    taskSchedulerService.stop();
+    taskSchedulerService.shutdown();
   }
 
   /**
    * TaskAttempt Killed from START_WAIT
    */
   @Test(timeout = 5000)
-  public void testDeallocationAfterAllocation() {
-    AppContext appContext = mock(AppContext.class);
+  public void testDeallocationAfterAllocation() throws InterruptedException {
     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, "", appContext);
-    taskSchedulerService.init(new Configuration());
+
+    TaskSchedulerContext mockContext = TestTaskSchedulerHelpers
+        .setupMockTaskSchedulerContext("", 0, "", false, appAttemptId, 10000l, null, new Configuration());
+
+    MockLocalTaskSchedulerSerivce taskSchedulerService =
+        new MockLocalTaskSchedulerSerivce(mockContext);
+
+    taskSchedulerService.initialize();
     taskSchedulerService.start();
 
     Task task = mock(Task.class);
@@ -130,33 +132,29 @@ public class TestLocalTaskSchedulerService {
     requestHandler.drainRequest(2);
     assertEquals(1, requestHandler.deallocateCount);
     assertEquals(1, requestHandler.allocateCount);
-    taskSchedulerService.stop();
+    taskSchedulerService.shutdown();
   }
 
   static class MockLocalTaskSchedulerSerivce extends LocalTaskSchedulerService {
 
     private MockAsyncDelegateRequestHandler requestHandler;
 
-    public MockLocalTaskSchedulerSerivce(TaskSchedulerAppCallback appClient,
-        ContainerSignatureMatcher containerSignatureMatcher,
-        String appHostName, int appHostPort, String appTrackingUrl,
-        AppContext appContext) {
-      super(appClient, containerSignatureMatcher, appHostName, appHostPort,
-          appTrackingUrl, 10000l, appContext);
+    public MockLocalTaskSchedulerSerivce(TaskSchedulerContext appClient) {
+      super(appClient);
     }
 
     @Override
     public AsyncDelegateRequestHandler createRequestHandler(Configuration conf) {
       requestHandler = new MockAsyncDelegateRequestHandler(taskRequestQueue,
-          new LocalContainerFactory(appContext, customContainerAppId),
+          new LocalContainerFactory(getContext().getApplicationAttemptId(), customContainerAppId),
           taskAllocations,
-          appClientDelegate,
+          getContext(),
           conf);
       return requestHandler;
     }
 
     @Override
-    public void serviceStart() {
+    public void start() {
       // don't start RequestHandler thread, control it in unit test
     }
 
@@ -178,7 +176,7 @@ public class TestLocalTaskSchedulerService {
           BlockingQueue<TaskRequest> taskRequestQueue,
           LocalContainerFactory localContainerFactory,
           HashMap<Object, Container> taskAllocations,
-          TaskSchedulerAppCallback appClientDelegate, Configuration conf) {
+          TaskSchedulerContext appClientDelegate, Configuration conf) {
         super(taskRequestQueue, localContainerFactory, taskAllocations,
             appClientDelegate, conf);
       }

http://git-wip-us.apache.org/repos/asf/tez/blob/82c24ac0/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 807e772..123a4d7 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
@@ -18,6 +18,8 @@
 
 package org.apache.tez.dag.app.rm;
 
+import static org.apache.tez.dag.app.rm.TestTaskSchedulerHelpers.createCountingExecutingService;
+import static org.apache.tez.dag.app.rm.TestTaskSchedulerHelpers.setupMockTaskSchedulerContext;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
@@ -42,8 +44,11 @@ import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Set;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
 import java.util.concurrent.atomic.AtomicBoolean;
 
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse;
 import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
@@ -59,23 +64,21 @@ import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
-import org.apache.hadoop.yarn.factories.RecordFactory;
-import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.util.RackResolver;
+import org.apache.tez.common.ContainerSignatureMatcher;
 import org.apache.tez.common.MockDNSToSwitchMapping;
 import org.apache.tez.dag.api.TezConfiguration;
-import org.apache.tez.dag.app.AppContext;
-import org.apache.tez.dag.app.DAGAppMasterState;
 import org.apache.tez.dag.app.rm.YarnTaskSchedulerService.CookieContainerRequest;
 import org.apache.tez.dag.app.rm.YarnTaskSchedulerService.HeldContainer;
-import org.apache.tez.dag.app.rm.TaskSchedulerService.TaskSchedulerAppCallback;
-import org.apache.tez.dag.app.rm.TaskSchedulerService.TaskSchedulerAppCallback.AppFinalStatus;
-import org.apache.tez.dag.app.rm.TestTaskSchedulerHelpers.TaskSchedulerAppCallbackDrainable;
-import org.apache.tez.dag.app.rm.TestTaskSchedulerHelpers.TaskSchedulerWithDrainableAppCallback;
+import org.apache.tez.dag.app.rm.TestTaskSchedulerHelpers.TaskSchedulerContextDrainable;
+import org.apache.tez.dag.app.rm.TestTaskSchedulerHelpers.TaskSchedulerWithDrainableContext;
 import org.apache.tez.dag.app.rm.TestTaskSchedulerHelpers.AlwaysMatchesContainerMatcher;
 import org.apache.tez.dag.app.rm.TestTaskSchedulerHelpers.PreemptionMatcher;
-import org.apache.tez.dag.app.rm.container.ContainerSignatureMatcher;
+import org.apache.tez.serviceplugins.api.TaskSchedulerContext;
+import org.apache.tez.serviceplugins.api.TaskSchedulerContext.AppFinalStatus;
+import org.junit.After;
 import org.junit.Assert;
+import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
 import org.mockito.ArgumentCaptor;
@@ -88,23 +91,39 @@ import com.google.common.collect.Sets;
 
 public class TestTaskScheduler {
 
-  RecordFactory recordFactory =
-      RecordFactoryProvider.getRecordFactory(null);
-
   static ContainerSignatureMatcher containerSignatureMatcher = new AlwaysMatchesContainerMatcher();
+  private ExecutorService contextCallbackExecutor;
 
   @BeforeClass
   public static void beforeClass() {
     MockDNSToSwitchMapping.initializeMockRackResolver();
   }
 
+  @Before
+  public void preTest() {
+    contextCallbackExecutor = Executors.newSingleThreadExecutor(
+        new ThreadFactoryBuilder().setNameFormat("TaskSchedulerAppCallbackExecutor #%d")
+            .setDaemon(true)
+            .build());
+  }
+
+  @After
+  public void postTest() {
+    contextCallbackExecutor.shutdownNow();
+  }
+
+  private TaskSchedulerContextDrainable createDrainableContext(
+      TaskSchedulerContext taskSchedulerContext) {
+    TaskSchedulerContextImplWrapper wrapper =
+        new TaskSchedulerContextImplWrapper(taskSchedulerContext,
+            createCountingExecutingService(contextCallbackExecutor));
+    return new TaskSchedulerContextDrainable(wrapper);
+  }
+
   @SuppressWarnings({ "unchecked" })
   @Test(timeout=10000)
   public void testTaskSchedulerNoReuse() throws Exception {
     RackResolver.init(new YarnConfiguration());
-    TaskSchedulerAppCallback mockApp = mock(TaskSchedulerAppCallback.class);
-    AppContext mockAppContext = mock(AppContext.class);
-    when(mockAppContext.getAMState()).thenReturn(DAGAppMasterState.RUNNING);
 
     TezAMRMClientAsync<CookieContainerRequest> mockRMClient =
                                                   mock(TezAMRMClientAsync.class);
@@ -112,18 +131,19 @@ public class TestTaskScheduler {
     String appHost = "host";
     int appPort = 0;
     String appUrl = "url";
-    TaskSchedulerWithDrainableAppCallback scheduler =
-      new TaskSchedulerWithDrainableAppCallback(
-        mockApp, new AlwaysMatchesContainerMatcher(), appHost, appPort,
-        appUrl, mockRMClient, mockAppContext);
-    TaskSchedulerAppCallbackDrainable drainableAppCallback = scheduler
-        .getDrainableAppCallback();
 
     Configuration conf = new Configuration();
     conf.setBoolean(TezConfiguration.TEZ_AM_CONTAINER_REUSE_ENABLED, false);
     int interval = 100;
     conf.setInt(TezConfiguration.TEZ_AM_RM_HEARTBEAT_INTERVAL_MS_MAX, interval);
-    scheduler.init(conf);
+
+    TaskSchedulerContext mockApp = setupMockTaskSchedulerContext(appHost, appPort, appUrl, conf);
+    TaskSchedulerContextDrainable drainableAppCallback = createDrainableContext(mockApp);
+
+    TaskSchedulerWithDrainableContext scheduler =
+        new TaskSchedulerWithDrainableContext(drainableAppCallback, mockRMClient);
+
+    scheduler.initialize();
     drainableAppCallback.drain();
     verify(mockRMClient).init(conf);
     verify(mockRMClient).setHeartbeatInterval(interval);
@@ -495,22 +515,18 @@ public class TestTaskScheduler {
     AppFinalStatus finalStatus =
         new AppFinalStatus(FinalApplicationStatus.SUCCEEDED, appMsg, appUrl);
     when(mockApp.getFinalAppStatus()).thenReturn(finalStatus);
-    scheduler.stop();
+    scheduler.shutdown();
     drainableAppCallback.drain();
     verify(mockRMClient).
                   unregisterApplicationMaster(FinalApplicationStatus.SUCCEEDED,
-                                              appMsg, appUrl);
+                      appMsg, appUrl);
     verify(mockRMClient).stop();
-    scheduler.close();
   }
   
   @SuppressWarnings({ "unchecked" })
   @Test(timeout=10000)
   public void testTaskSchedulerWithReuse() throws Exception {
     RackResolver.init(new YarnConfiguration());
-    TaskSchedulerAppCallback mockApp = mock(TaskSchedulerAppCallback.class);
-    AppContext mockAppContext = mock(AppContext.class);
-    when(mockAppContext.getAMState()).thenReturn(DAGAppMasterState.RUNNING);
 
     TezAMRMClientAsync<CookieContainerRequest> mockRMClient =
                                                   mock(TezAMRMClientAsync.class);
@@ -518,12 +534,6 @@ public class TestTaskScheduler {
     String appHost = "host";
     int appPort = 0;
     String appUrl = "url";
-    TaskSchedulerWithDrainableAppCallback scheduler =
-      new TaskSchedulerWithDrainableAppCallback(
-        mockApp, new AlwaysMatchesContainerMatcher(), appHost, appPort,
-        appUrl, mockRMClient, mockAppContext);
-    final TaskSchedulerAppCallbackDrainable drainableAppCallback = scheduler
-        .getDrainableAppCallback();
 
     Configuration conf = new Configuration();
     // to match all in the same pass
@@ -531,7 +541,15 @@ public class TestTaskScheduler {
     // to release immediately after deallocate
     conf.setLong(TezConfiguration.TEZ_AM_CONTAINER_IDLE_RELEASE_TIMEOUT_MIN_MILLIS, 0);
     conf.setLong(TezConfiguration.TEZ_AM_CONTAINER_IDLE_RELEASE_TIMEOUT_MAX_MILLIS, 0);
-    scheduler.init(conf);
+
+    TaskSchedulerContext mockApp = setupMockTaskSchedulerContext(appHost, appPort, appUrl, conf);
+    final TaskSchedulerContextDrainable drainableAppCallback = createDrainableContext(mockApp);
+
+    TaskSchedulerWithDrainableContext scheduler =
+        new TaskSchedulerWithDrainableContext(drainableAppCallback, mockRMClient);
+
+
+    scheduler.initialize();
     drainableAppCallback.drain();
 
     RegisterApplicationMasterResponse mockRegResponse =
@@ -992,23 +1010,18 @@ public class TestTaskScheduler {
     AppFinalStatus finalStatus =
         new AppFinalStatus(FinalApplicationStatus.SUCCEEDED, appMsg, appUrl);
     when(mockApp.getFinalAppStatus()).thenReturn(finalStatus);
-    scheduler.stop();
+    scheduler.shutdown();
     drainableAppCallback.drain();
     verify(mockRMClient).
                   unregisterApplicationMaster(FinalApplicationStatus.SUCCEEDED,
-                                              appMsg, appUrl);
+                      appMsg, appUrl);
     verify(mockRMClient).stop();
-    scheduler.close();
   }
   
   @SuppressWarnings("unchecked")
   @Test (timeout=5000)
   public void testTaskSchedulerDetermineMinHeldContainers() throws Exception {
     RackResolver.init(new YarnConfiguration());
-    TaskSchedulerAppCallback mockApp = mock(TaskSchedulerAppCallback.class);
-    AppContext mockAppContext = mock(AppContext.class);
-    when(mockAppContext.getAMState()).thenReturn(DAGAppMasterState.RUNNING);
-    when(mockAppContext.isSession()).thenReturn(true);
 
     TezAMRMClientAsync<CookieContainerRequest> mockRMClient =
                                                   mock(TezAMRMClientAsync.class);
@@ -1016,15 +1029,15 @@ public class TestTaskScheduler {
     String appHost = "host";
     int appPort = 0;
     String appUrl = "url";
-    TaskSchedulerWithDrainableAppCallback scheduler =
-      new TaskSchedulerWithDrainableAppCallback(
-        mockApp, new AlwaysMatchesContainerMatcher(), appHost, appPort,
-        appUrl, mockRMClient, mockAppContext);
-    TaskSchedulerAppCallbackDrainable drainableAppCallback = scheduler
-        .getDrainableAppCallback();
 
-    Configuration conf = new Configuration();
-    scheduler.init(conf);
+    TaskSchedulerContext mockApp = setupMockTaskSchedulerContext(appHost, appPort, appUrl, true,
+        new Configuration());
+    final TaskSchedulerContextDrainable drainableAppCallback = createDrainableContext(mockApp);
+
+    TaskSchedulerWithDrainableContext scheduler =
+        new TaskSchedulerWithDrainableContext(drainableAppCallback, mockRMClient);
+
+    scheduler.initialize();
     RegisterApplicationMasterResponse mockRegResponse = mock(RegisterApplicationMasterResponse.class);
     Resource mockMaxResource = mock(Resource.class);
     Map<ApplicationAccessType, String> mockAcls = mock(Map.class);
@@ -1176,17 +1189,13 @@ public class TestTaskScheduler {
     AppFinalStatus finalStatus =
         new AppFinalStatus(FinalApplicationStatus.SUCCEEDED, appMsg, appUrl);
     when(mockApp.getFinalAppStatus()).thenReturn(finalStatus);
-    scheduler.stop();
-    scheduler.close();
+    scheduler.shutdown();
   }
   
   @SuppressWarnings("unchecked")
   @Test(timeout=5000)
   public void testTaskSchedulerRandomReuseExpireTime() throws Exception {
     RackResolver.init(new YarnConfiguration());
-    TaskSchedulerAppCallback mockApp = mock(TaskSchedulerAppCallback.class);
-    AppContext mockAppContext = mock(AppContext.class);
-    when(mockAppContext.getAMState()).thenReturn(DAGAppMasterState.RUNNING);
 
     TezAMRMClientAsync<CookieContainerRequest> mockRMClient =
                                                   mock(TezAMRMClientAsync.class);
@@ -1194,25 +1203,31 @@ public class TestTaskScheduler {
     String appHost = "host";
     int appPort = 0;
     String appUrl = "url";
-    TaskSchedulerWithDrainableAppCallback scheduler1 =
-      new TaskSchedulerWithDrainableAppCallback(
-        mockApp, new AlwaysMatchesContainerMatcher(), appHost, appPort,
-        appUrl, mockRMClient, mockAppContext);
-    TaskSchedulerWithDrainableAppCallback scheduler2 =
-        new TaskSchedulerWithDrainableAppCallback(
-          mockApp, new AlwaysMatchesContainerMatcher(), appHost, appPort,
-          appUrl, mockRMClient, mockAppContext);
 
     long minTime = 1000l;
     long maxTime = 100000l;
     Configuration conf1 = new Configuration();
     conf1.setLong(TezConfiguration.TEZ_AM_CONTAINER_IDLE_RELEASE_TIMEOUT_MIN_MILLIS, minTime);
     conf1.setLong(TezConfiguration.TEZ_AM_CONTAINER_IDLE_RELEASE_TIMEOUT_MAX_MILLIS, minTime);
-    scheduler1.init(conf1);
+
     Configuration conf2 = new Configuration();
     conf2.setLong(TezConfiguration.TEZ_AM_CONTAINER_IDLE_RELEASE_TIMEOUT_MIN_MILLIS, minTime);
     conf2.setLong(TezConfiguration.TEZ_AM_CONTAINER_IDLE_RELEASE_TIMEOUT_MAX_MILLIS, maxTime);
-    scheduler2.init(conf2);
+
+    TaskSchedulerContext mockApp1 = setupMockTaskSchedulerContext(appHost, appPort, appUrl, conf1);
+    TaskSchedulerContext mockApp2 = setupMockTaskSchedulerContext(appHost, appPort, appUrl, conf2);
+    final TaskSchedulerContextDrainable drainableAppCallback1 = createDrainableContext(mockApp1);
+    final TaskSchedulerContextDrainable drainableAppCallback2 = createDrainableContext(mockApp2);
+
+
+    TaskSchedulerWithDrainableContext scheduler1 =
+        new TaskSchedulerWithDrainableContext(drainableAppCallback1, mockRMClient);
+    TaskSchedulerWithDrainableContext scheduler2 =
+        new TaskSchedulerWithDrainableContext(drainableAppCallback2, mockRMClient);
+
+    scheduler1.initialize();
+    scheduler2.initialize();
+
 
     RegisterApplicationMasterResponse mockRegResponse =
                                 mock(RegisterApplicationMasterResponse.class);
@@ -1250,20 +1265,16 @@ public class TestTaskScheduler {
     String appMsg = "success";
     AppFinalStatus finalStatus =
         new AppFinalStatus(FinalApplicationStatus.SUCCEEDED, appMsg, appUrl);
-    when(mockApp.getFinalAppStatus()).thenReturn(finalStatus);
-    scheduler1.stop();
-    scheduler1.close();
-    scheduler2.stop();
-    scheduler2.close();
+    when(mockApp1.getFinalAppStatus()).thenReturn(finalStatus);
+    when(mockApp2.getFinalAppStatus()).thenReturn(finalStatus);
+    scheduler1.shutdown();
+    scheduler2.shutdown();
   }
 
   @SuppressWarnings({ "unchecked", "rawtypes" })
   @Test (timeout=5000)
   public void testTaskSchedulerPreemption() throws Exception {
     RackResolver.init(new YarnConfiguration());
-    TaskSchedulerAppCallback mockApp = mock(TaskSchedulerAppCallback.class);
-    AppContext mockAppContext = mock(AppContext.class);
-    when(mockAppContext.getAMState()).thenReturn(DAGAppMasterState.RUNNING);
 
     TezAMRMClientAsync<CookieContainerRequest> mockRMClient =
                                                   mock(TezAMRMClientAsync.class);
@@ -1271,16 +1282,18 @@ public class TestTaskScheduler {
     String appHost = "host";
     int appPort = 0;
     String appUrl = "url";
-    final TaskSchedulerWithDrainableAppCallback scheduler =
-      new TaskSchedulerWithDrainableAppCallback(
-        mockApp, new PreemptionMatcher(), appHost, appPort,
-        appUrl, mockRMClient, mockAppContext);
-    TaskSchedulerAppCallbackDrainable drainableAppCallback = scheduler
-        .getDrainableAppCallback();
 
     Configuration conf = new Configuration();
     conf.setBoolean(TezConfiguration.TEZ_AM_CONTAINER_REUSE_ENABLED, false);
-    scheduler.init(conf);
+
+    TaskSchedulerContext mockApp = setupMockTaskSchedulerContext(appHost, appPort, appUrl, false,
+        null, null, new PreemptionMatcher(), conf);
+    final TaskSchedulerContextDrainable drainableAppCallback = createDrainableContext(mockApp);
+
+    final TaskSchedulerWithDrainableContext scheduler =
+        new TaskSchedulerWithDrainableContext(drainableAppCallback, mockRMClient);
+
+    scheduler.initialize();
 
     RegisterApplicationMasterResponse mockRegResponse =
                        mock(RegisterApplicationMasterResponse.class);
@@ -1530,7 +1543,7 @@ public class TestTaskScheduler {
     scheduler.getProgress();
     scheduler.getProgress();
     scheduler.getProgress();
-    verify(mockRMClient, times(2)).releaseAssignedContainer((ContainerId)any());
+    verify(mockRMClient, times(2)).releaseAssignedContainer((ContainerId) any());
     scheduler.getProgress();
     drainableAppCallback.drain();
     // Next oldest mockTaskPri3KillA gets preempted to clear 10% of outstanding running preemptable tasks
@@ -1540,9 +1553,8 @@ public class TestTaskScheduler {
     AppFinalStatus finalStatus =
         new AppFinalStatus(FinalApplicationStatus.SUCCEEDED, "", appUrl);
     when(mockApp.getFinalAppStatus()).thenReturn(finalStatus);
-    scheduler.stop();
+    scheduler.shutdown();
     drainableAppCallback.drain();
-    scheduler.close();
   }
 
   @SuppressWarnings("unchecked")
@@ -1550,22 +1562,19 @@ public class TestTaskScheduler {
   public void testLocalityMatching() throws Exception {
 
     RackResolver.init(new Configuration());
-    TaskSchedulerAppCallback appClient = mock(TaskSchedulerAppCallback.class);
     TezAMRMClientAsync<CookieContainerRequest> amrmClient =
       mock(TezAMRMClientAsync.class);
-    AppContext mockAppContext = mock(AppContext.class);
-    when(mockAppContext.getAMState()).thenReturn(DAGAppMasterState.RUNNING);
-
-    TaskSchedulerWithDrainableAppCallback taskScheduler =
-      new TaskSchedulerWithDrainableAppCallback(
-        appClient, new AlwaysMatchesContainerMatcher(), "host", 0, "",
-        amrmClient, mockAppContext);
-    TaskSchedulerAppCallbackDrainable drainableAppCallback = taskScheduler
-        .getDrainableAppCallback();
-    
+
     Configuration conf = new Configuration();
     conf.setBoolean(TezConfiguration.TEZ_AM_CONTAINER_REUSE_ENABLED, false);
-    taskScheduler.init(conf);
+
+    TaskSchedulerContext appClient = setupMockTaskSchedulerContext("host", 0, "", conf);
+    final TaskSchedulerContextDrainable drainableAppCallback = createDrainableContext(appClient);
+
+    TaskSchedulerWithDrainableContext taskScheduler =
+        new TaskSchedulerWithDrainableContext(drainableAppCallback, amrmClient);
+
+    taskScheduler.initialize();
     
     RegisterApplicationMasterResponse mockRegResponse = mock(RegisterApplicationMasterResponse.class);
     Resource mockMaxResource = mock(Resource.class);
@@ -1693,7 +1702,7 @@ public class TestTaskScheduler {
     AppFinalStatus finalStatus = new AppFinalStatus(
         FinalApplicationStatus.SUCCEEDED, "", "");
     when(appClient.getFinalAppStatus()).thenReturn(finalStatus);
-    taskScheduler.close();
+    taskScheduler.shutdown();
   }
   
   @Test (timeout=5000)

http://git-wip-us.apache.org/repos/asf/tez/blob/82c24ac0/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 005692e..3ea0446 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
@@ -47,11 +47,13 @@ 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.ContainerSignatureMatcher;
 import org.apache.tez.dag.api.TaskLocationHint;
 import org.apache.tez.dag.api.TezConfiguration;
 import org.apache.tez.dag.api.client.DAGClientServer;
 import org.apache.tez.dag.app.AppContext;
 import org.apache.tez.dag.app.ContainerContext;
+import org.apache.tez.dag.app.ServicePluginLifecycleAbstractService;
 import org.apache.tez.dag.app.dag.impl.TaskAttemptImpl;
 import org.apache.tez.dag.app.dag.impl.TaskImpl;
 import org.apache.tez.dag.app.dag.impl.VertexImpl;
@@ -61,10 +63,10 @@ import org.apache.tez.dag.app.rm.container.AMContainerEventCompleted;
 import org.apache.tez.dag.app.rm.container.AMContainerEventType;
 import org.apache.tez.dag.app.rm.container.AMContainerMap;
 import org.apache.tez.dag.app.rm.container.AMContainerState;
-import org.apache.tez.dag.app.rm.container.ContainerSignatureMatcher;
 import org.apache.tez.dag.app.web.WebUIService;
 import org.apache.tez.dag.records.TaskAttemptTerminationCause;
 import org.apache.tez.dag.records.TezTaskAttemptID;
+import org.apache.tez.serviceplugins.api.TaskScheduler;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
@@ -96,6 +98,7 @@ public class TestTaskSchedulerEventHandler {
     protected void instantiateScheduelrs(String host, int port, String trackingUrl,
                                          AppContext appContext) {
       taskSchedulers[0] = mockTaskScheduler;
+      taskSchedulerServiceWrappers[0] = new ServicePluginLifecycleAbstractService(taskSchedulers[0]);
     }
     
     @Override
@@ -113,7 +116,7 @@ public class TestTaskSchedulerEventHandler {
   TestEventHandler mockEventHandler;
   ContainerSignatureMatcher mockSigMatcher;
   MockTaskSchedulerEventHandler schedulerHandler;
-  TaskSchedulerService mockTaskScheduler;
+  TaskScheduler mockTaskScheduler;
   AMContainerMap mockAMContainerMap;
   WebUIService mockWebUIService;
 
@@ -124,7 +127,7 @@ public class TestTaskSchedulerEventHandler {
     mockClientService = mock(DAGClientServer.class);
     mockEventHandler = new TestEventHandler();
     mockSigMatcher = mock(ContainerSignatureMatcher.class);
-    mockTaskScheduler = mock(TaskSchedulerService.class);
+    mockTaskScheduler = mock(TaskScheduler.class);
     mockAMContainerMap = mock(AMContainerMap.class);
     mockWebUIService = mock(WebUIService.class);
     when(mockAppContext.getAllContainers()).thenReturn(mockAMContainerMap);

http://git-wip-us.apache.org/repos/asf/tez/blob/82c24ac0/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 04610ab..966c95a 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
@@ -40,9 +40,13 @@ import java.util.concurrent.TimeoutException;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
 
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Maps;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.service.AbstractService;
 import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse;
 import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
+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;
@@ -54,13 +58,12 @@ import org.apache.hadoop.yarn.client.api.AMRMClient;
 import org.apache.hadoop.yarn.client.api.impl.AMRMClientImpl;
 import org.apache.hadoop.yarn.event.Event;
 import org.apache.hadoop.yarn.event.EventHandler;
+import org.apache.tez.common.ContainerSignatureMatcher;
 import org.apache.tez.dag.app.AppContext;
+import org.apache.tez.dag.app.ServicePluginLifecycleAbstractService;
 import org.apache.tez.dag.app.rm.YarnTaskSchedulerService.CookieContainerRequest;
-import org.apache.tez.dag.app.rm.TaskSchedulerService.TaskSchedulerAppCallback;
-import org.apache.tez.dag.app.rm.container.ContainerSignatureMatcher;
-
-import com.google.common.base.Preconditions;
-import com.google.common.collect.Maps;
+import org.apache.tez.serviceplugins.api.TaskScheduler;
+import org.apache.tez.serviceplugins.api.TaskSchedulerContext;
 
 class TestTaskSchedulerHelpers {
 
@@ -134,12 +137,19 @@ class TestTaskSchedulerHelpers {
 
     @Override
     public void instantiateScheduelrs(String host, int port, String trackingUrl, AppContext appContext) {
-      taskSchedulers[0] = new TaskSchedulerWithDrainableAppCallback(new TaskSchedulerAppCallbackImpl(this, 0),
-          containerSignatureMatcher, host, port, trackingUrl, amrmClientAsync,
-          appContext);
-    }
-
-    public TaskSchedulerService getSpyTaskScheduler() {
+      TaskSchedulerContext taskSchedulerContext =
+          new TaskSchedulerContextImpl(this, appContext, 0, trackingUrl, 1000, host, port,
+              getConfig());
+      TaskSchedulerContextImplWrapper wrapper =
+          new TaskSchedulerContextImplWrapper(taskSchedulerContext,
+              new CountingExecutorService(appCallbackExecutor));
+      TaskSchedulerContextDrainable drainable = new TaskSchedulerContextDrainable(wrapper);
+      taskSchedulers[0] =
+          new TaskSchedulerWithDrainableContext(drainable, amrmClientAsync);
+      taskSchedulerServiceWrappers[0] = new ServicePluginLifecycleAbstractService(taskSchedulers[0]);
+    }
+
+    public TaskScheduler getSpyTaskScheduler() {
       return taskSchedulers[0];
     }
 
@@ -147,8 +157,8 @@ class TestTaskSchedulerHelpers {
     public void serviceStart() {
       instantiateScheduelrs("host", 0, "", appContext);
       // Init the service so that reuse configuration is picked up.
-      ((AbstractService)taskSchedulers[0]).init(getConfig());
-      ((AbstractService)taskSchedulers[0]).start();
+      ((AbstractService)taskSchedulerServiceWrappers[0]).init(getConfig());
+      ((AbstractService)taskSchedulerServiceWrappers[0]).start();
       taskSchedulers[0] = spy(taskSchedulers[0]);
     }
 
@@ -188,61 +198,31 @@ class TestTaskSchedulerHelpers {
     }
   }
 
-  static class TaskSchedulerWithDrainableAppCallback extends YarnTaskSchedulerService {
+  static class TaskSchedulerWithDrainableContext extends YarnTaskSchedulerService {
 
-    private TaskSchedulerAppCallbackDrainable drainableAppCallback;
 
-    public TaskSchedulerWithDrainableAppCallback(
-        TaskSchedulerAppCallback appClient,
-        ContainerSignatureMatcher containerSignatureMatcher,
-        String appHostName, int appHostPort, String appTrackingUrl,
-        AppContext appContext) {
-      super(appClient, containerSignatureMatcher, appHostName, appHostPort,
-          appTrackingUrl, appContext);
+    public TaskSchedulerWithDrainableContext(
+        TaskSchedulerContextDrainable appClient,
+        TezAMRMClientAsync<CookieContainerRequest> client) {
+      super(appClient, client);
       shouldUnregister.set(true);
     }
 
-    public TaskSchedulerWithDrainableAppCallback(
-        TaskSchedulerAppCallback appClient,
-        ContainerSignatureMatcher containerSignatureMatcher,
-        String appHostName, int appHostPort, String appTrackingUrl,
-        TezAMRMClientAsync<CookieContainerRequest> client,
-        AppContext appContext) {
-      super(appClient, containerSignatureMatcher, appHostName, appHostPort,
-          appTrackingUrl, client, appContext);
-      shouldUnregister.set(true);
-    }
-
-    @Override
-    TaskSchedulerAppCallback createAppCallbackDelegate(
-        TaskSchedulerAppCallback realAppClient) {
-      drainableAppCallback = new TaskSchedulerAppCallbackDrainable(
-          new TaskSchedulerAppCallbackWrapper(realAppClient,
-              appCallbackExecutor));
-      return drainableAppCallback;
-    }
-    
-    @Override
-    ExecutorService createAppCallbackExecutorService() {
-      ExecutorService real = super.createAppCallbackExecutorService();
-      return new CountingExecutorService(real);
-    }
-
-    public TaskSchedulerAppCallbackDrainable getDrainableAppCallback() {
-      return drainableAppCallback;
+    public TaskSchedulerContextDrainable getDrainableAppCallback() {
+      return (TaskSchedulerContextDrainable)getContext();
     }
   }
 
   @SuppressWarnings("rawtypes")
-  static class TaskSchedulerAppCallbackDrainable implements TaskSchedulerAppCallback {
+  static class TaskSchedulerContextDrainable implements TaskSchedulerContext {
     int completedEvents;
     int invocations;
-    private TaskSchedulerAppCallback real;
+    private TaskSchedulerContext real;
     private CountingExecutorService countingExecutorService;
     final AtomicInteger count = new AtomicInteger(0);
     
-    public TaskSchedulerAppCallbackDrainable(TaskSchedulerAppCallbackWrapper real) {
-      countingExecutorService = (CountingExecutorService) real.executorService;
+    public TaskSchedulerContextDrainable(TaskSchedulerContextImplWrapper real) {
+      countingExecutorService = (CountingExecutorService) real.getExecutorService();
       this.real = real;
     }
 
@@ -303,6 +283,53 @@ class TestTaskSchedulerHelpers {
       return real.getFinalAppStatus();
     }
 
+    // Not incrementing invocations for methods which to not obtain locks,
+    // and do not go via the executor service.
+    @Override
+    public Configuration getInitialConfiguration() {
+      return real.getInitialConfiguration();
+    }
+
+    @Override
+    public String getAppTrackingUrl() {
+      return real.getAppTrackingUrl();
+    }
+
+    @Override
+    public long getCustomClusterIdentifier() {
+      return real.getCustomClusterIdentifier();
+    }
+
+    @Override
+    public ContainerSignatureMatcher getContainerSignatureMatcher() {
+      return real.getContainerSignatureMatcher();
+    }
+
+    @Override
+    public ApplicationAttemptId getApplicationAttemptId() {
+      return real.getApplicationAttemptId();
+    }
+
+    @Override
+    public String getAppHostName() {
+      return real.getAppHostName();
+    }
+
+    @Override
+    public int getAppClientPort() {
+      return real.getAppClientPort();
+    }
+
+    @Override
+    public boolean isSession() {
+      return real.isSession();
+    }
+
+    @Override
+    public AMState getAMState() {
+      return real.getAMState();
+    }
+
     @Override
     public void preemptContainer(ContainerId cId) {
       invocations++;
@@ -384,7 +411,11 @@ class TestTaskSchedulerHelpers {
       }
     }
   }
-  
+
+  static CountingExecutorService createCountingExecutingService(ExecutorService rawExecutor) {
+    return new CountingExecutorService(rawExecutor);
+  }
+
   @SuppressWarnings({"rawtypes", "unchecked"})
   private static class CountingExecutorService implements ExecutorService {
 
@@ -464,7 +495,50 @@ class TestTaskSchedulerHelpers {
         throws InterruptedException, ExecutionException, TimeoutException {
       throw new UnsupportedOperationException("Not expected to be used");
     }
-    
+  }
+
+  static TaskSchedulerContext setupMockTaskSchedulerContext(String appHost, int appPort,
+                                                            String appUrl, Configuration conf) {
+    return setupMockTaskSchedulerContext(appHost, appPort, appUrl, false, conf);
+  }
+
+  static TaskSchedulerContext setupMockTaskSchedulerContext(String appHost, int appPort,
+                                                            String appUrl, boolean isSession,
+                                                            Configuration conf) {
+    return setupMockTaskSchedulerContext(appHost, appPort, appUrl, isSession, null, null, null,
+        conf);
+  }
+
+  static TaskSchedulerContext setupMockTaskSchedulerContext(String appHost, int appPort,
+                                                            String appUrl, boolean isSession,
+                                                            ApplicationAttemptId appAttemptId,
+                                                            Long customAppIdentifier,
+                                                            ContainerSignatureMatcher containerSignatureMatcher,
+                                                            Configuration conf) {
+
+    TaskSchedulerContext mockContext = mock(TaskSchedulerContext.class);
+    when(mockContext.getAppHostName()).thenReturn(appHost);
+    when(mockContext.getAppClientPort()).thenReturn(appPort);
+    when(mockContext.getAppTrackingUrl()).thenReturn(appUrl);
+
+    when(mockContext.getAMState()).thenReturn(TaskSchedulerContext.AMState.RUNNING_APP);
+    when(mockContext.getInitialConfiguration()).thenReturn(conf);
+    when(mockContext.isSession()).thenReturn(isSession);
+    if (containerSignatureMatcher != null) {
+      when(mockContext.getContainerSignatureMatcher())
+          .thenReturn(containerSignatureMatcher);
+    } else {
+      when(mockContext.getContainerSignatureMatcher())
+          .thenReturn(new AlwaysMatchesContainerMatcher());
+    }
+    if (appAttemptId != null) {
+      when(mockContext.getApplicationAttemptId()).thenReturn(appAttemptId);
+    }
+    if (customAppIdentifier != null) {
+      when(mockContext.getCustomClusterIdentifier()).thenReturn(customAppIdentifier);
+    }
+
+    return mockContext;
   }
 
 }


[25/50] [abbrv] tez git commit: TEZ-2526. Fix version for tez-history-parser. Contributed by Sergey Shelukhin.

Posted by ss...@apache.org.
TEZ-2526. Fix version for tez-history-parser. Contributed by Sergey Shelukhin.


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

Branch: refs/heads/TEZ-2003
Commit: 06f6dcf0031880309fa4bbf0c7c55b1145ff1001
Parents: dcd767e
Author: Siddharth Seth <ss...@apache.org>
Authored: Tue Jun 2 12:47:58 2015 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Fri Aug 14 13:46:44 2015 -0700

----------------------------------------------------------------------
 TEZ-2003-CHANGES.txt                   | 1 +
 tez-plugins/tez-history-parser/pom.xml | 2 +-
 2 files changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/06f6dcf0/TEZ-2003-CHANGES.txt
----------------------------------------------------------------------
diff --git a/TEZ-2003-CHANGES.txt b/TEZ-2003-CHANGES.txt
index 55002fe..1e8abcf 100644
--- a/TEZ-2003-CHANGES.txt
+++ b/TEZ-2003-CHANGES.txt
@@ -31,5 +31,6 @@ ALL CHANGES:
   TEZ-2495. Inform TaskCommunicaor about Task and Container termination reasons.
   TEZ-2502. Fix for TezTaskRunner2 not killing tasks properly in all situations.
   TEZ-2508. rebase 06/01
+  TEZ-2526. Fix version for tez-history-parser.
 
 INCOMPATIBLE CHANGES:

http://git-wip-us.apache.org/repos/asf/tez/blob/06f6dcf0/tez-plugins/tez-history-parser/pom.xml
----------------------------------------------------------------------
diff --git a/tez-plugins/tez-history-parser/pom.xml b/tez-plugins/tez-history-parser/pom.xml
index f12e0b4..5ab0090 100644
--- a/tez-plugins/tez-history-parser/pom.xml
+++ b/tez-plugins/tez-history-parser/pom.xml
@@ -21,7 +21,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-history-parser</artifactId>
 


[10/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/619aaf39
Tree: http://git-wip-us.apache.org/repos/asf/tez/tree/619aaf39
Diff: http://git-wip-us.apache.org/repos/asf/tez/diff/619aaf39

Branch: refs/heads/TEZ-2003
Commit: 619aaf396138324495de815c87dbc309a085b437
Parents: 3a60242
Author: Siddharth Seth <ss...@apache.org>
Authored: Wed Apr 29 08:20:05 2015 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Fri Aug 14 13:46:43 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/619aaf39/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/619aaf39/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 08b59ed..1092685 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
@@ -89,6 +89,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;
 
@@ -130,7 +131,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,
@@ -144,6 +145,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;
@@ -417,7 +419,7 @@ public class Fetcher extends CallableWithNdc<FetchResult> {
   private HostFetchResult setupConnection(Collection<InputAttemptIdentifier> attempts) {
     try {
       StringBuilder baseURI = ShuffleUtils.constructBaseURIForShuffleHandler(host,
-          port, partition, appId.toString(), httpConnectionParams.isSslShuffle());
+          port, partition, appId.toString(), dagIdentifier, httpConnectionParams.isSslShuffle());
       this.url = ShuffleUtils.constructInputURL(baseURI.toString(), attempts,
           httpConnectionParams.isKeepAlive());
 
@@ -930,22 +932,22 @@ 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, int shufflePort,
         boolean asyncHttp) {
-      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, shufflePort, asyncHttp);
     }
 
     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, int shufflePort, boolean asyncHttp) {
-      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, shufflePort, asyncHttp);
     }

http://git-wip-us.apache.org/repos/asf/tez/blob/619aaf39/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 1081587..c7cc907 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
@@ -184,19 +184,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/619aaf39/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 600c332..5bfcab8 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
@@ -403,7 +403,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,
         localhostName, shufflePort, asyncHttp);

http://git-wip-us.apache.org/repos/asf/tez/blob/619aaf39/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/619aaf39/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 7678b18..85e3540 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
@@ -75,7 +75,7 @@ public class TestFetcher {
     final boolean DISABLE_LOCAL_FETCH = false;
 
     Fetcher.FetcherBuilder builder = new Fetcher.FetcherBuilder(fetcherCallback, null, null,
-        ApplicationId.newInstance(0, 1), null, "fetcherTest", conf, ENABLE_LOCAL_FETCH, HOST,
+        ApplicationId.newInstance(0, 1), 1, null, "fetcherTest", conf, ENABLE_LOCAL_FETCH, HOST,
         PORT, false);
     builder.assignWork(HOST, PORT, 0, Arrays.asList(srcAttempts));
     Fetcher fetcher = spy(builder.build());
@@ -125,7 +125,7 @@ public class TestFetcher {
     // When disabled use http fetch
     conf.setBoolean(TezRuntimeConfiguration.TEZ_RUNTIME_OPTIMIZE_LOCAL_FETCH, false);
     builder = new Fetcher.FetcherBuilder(fetcherCallback, null, null,
-        ApplicationId.newInstance(0, 1), null, "fetcherTest", conf, DISABLE_LOCAL_FETCH, HOST,
+        ApplicationId.newInstance(0, 1), 1, null, "fetcherTest", conf, DISABLE_LOCAL_FETCH, HOST,
         PORT, false);
     builder.assignWork(HOST, PORT, 0, Arrays.asList(srcAttempts));
     fetcher = spy(builder.build());
@@ -159,7 +159,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, PORT, false);
+        ApplicationId.newInstance(0, 1), 1, null, "fetcherTest", conf, true, HOST, PORT, false);
     builder.assignWork(HOST, PORT, partition, Arrays.asList(srcAttempts));
     Fetcher fetcher = spy(builder.build());
 


[18/50] [abbrv] tez git commit: TEZ-2471. NPE in LogicalIOProcessorRuntimeTask while printing thread info. (sseth)

Posted by ss...@apache.org.
TEZ-2471. NPE in LogicalIOProcessorRuntimeTask while printing thread info. (sseth)


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

Branch: refs/heads/TEZ-2003
Commit: 34633d930e78fc80dfacc23c4eace913b70a9a03
Parents: 8a38d48
Author: Siddharth Seth <ss...@apache.org>
Authored: Fri May 22 11:03:11 2015 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Fri Aug 14 13:46:44 2015 -0700

----------------------------------------------------------------------
 TEZ-2003-CHANGES.txt                                            | 1 +
 .../org/apache/tez/runtime/LogicalIOProcessorRuntimeTask.java   | 5 ++++-
 2 files changed, 5 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/34633d93/TEZ-2003-CHANGES.txt
----------------------------------------------------------------------
diff --git a/TEZ-2003-CHANGES.txt b/TEZ-2003-CHANGES.txt
index ca3383c..d651960 100644
--- a/TEZ-2003-CHANGES.txt
+++ b/TEZ-2003-CHANGES.txt
@@ -27,5 +27,6 @@ ALL CHANGES:
   TEZ-2434. Allow tasks to be killed in the Runtime.
   TEZ-2443. TaskRunner2 should call abort, NPEs while cleaning up tasks.
   TEZ-2465. Retrun the status of a kill request in TaskRunner2.
+  TEZ-2471. NPE in LogicalIOProcessorRuntimeTask while printing thread info.
 
 INCOMPATIBLE CHANGES:

http://git-wip-us.apache.org/repos/asf/tez/blob/34633d93/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 de08e56..449fa0f 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
@@ -936,7 +936,10 @@ public class LogicalIOProcessorRuntimeTask extends RuntimeTask {
     long[] threadIds = threadMXBean.getAllThreadIds();
     for (Long id : threadIds) {
       ThreadInfo threadInfo = threadMXBean.getThreadInfo(id);
-      LOG.info("ThreadId : " + id + ", name=" + threadInfo.getThreadName());
+      // The thread could have been shutdown before we read info about it.
+      if (threadInfo != null) {
+        LOG.debug("ThreadId : " + id + ", name=" + threadInfo.getThreadName());
+      }
     }
   }
   


[32/50] [abbrv] tez git commit: TEZ-2005. Define basic interface for pluggable TaskScheduler. (sseth)

Posted by ss...@apache.org.
TEZ-2005. Define basic interface for pluggable TaskScheduler. (sseth)


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

Branch: refs/heads/TEZ-2003
Commit: 82c24ac0bcfc383fce9e069272fda5aa84e180c2
Parents: af1cc72
Author: Siddharth Seth <ss...@apache.org>
Authored: Wed Jul 22 22:25:01 2015 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Fri Aug 14 13:46:45 2015 -0700

----------------------------------------------------------------------
 TEZ-2003-CHANGES.txt                            |   1 +
 .../tez/common/ContainerSignatureMatcher.java   |  64 ++++
 .../tez/common/ServicePluginLifecycle.java      |  39 ++
 .../tez/serviceplugins/api/TaskScheduler.java   |  85 +++++
 .../api/TaskSchedulerContext.java               | 114 ++++++
 .../org/apache/tez/common/TezUtilsInternal.java |   1 +
 .../tez/dag/api/TaskCommunicatorInterface.java  |  18 -
 .../org/apache/tez/dag/app/DAGAppMaster.java    |   3 +-
 .../ServicePluginLifecycleAbstractService.java  |  52 +++
 .../dag/app/rm/LocalTaskSchedulerService.java   |  77 ++--
 .../app/rm/TaskSchedulerAppCallbackImpl.java    |  89 -----
 .../app/rm/TaskSchedulerAppCallbackWrapper.java | 307 ----------------
 .../dag/app/rm/TaskSchedulerContextImpl.java    | 174 +++++++++
 .../app/rm/TaskSchedulerContextImplWrapper.java | 368 +++++++++++++++++++
 .../dag/app/rm/TaskSchedulerEventHandler.java   |  81 ++--
 .../tez/dag/app/rm/TaskSchedulerService.java    | 111 ------
 .../dag/app/rm/YarnTaskSchedulerService.java    | 121 +++---
 .../dag/app/rm/container/AMContainerImpl.java   |   1 +
 .../dag/app/rm/container/AMContainerMap.java    |   1 +
 .../rm/container/ContainerContextMatcher.java   |   1 +
 .../rm/container/ContainerSignatureMatcher.java |  60 ---
 .../tez/dag/app/rm/TestContainerReuse.java      | 148 ++------
 .../tez/dag/app/rm/TestLocalTaskScheduler.java  |  29 +-
 .../app/rm/TestLocalTaskSchedulerService.java   |  52 ++-
 .../tez/dag/app/rm/TestTaskScheduler.java       | 201 +++++-----
 .../app/rm/TestTaskSchedulerEventHandler.java   |   9 +-
 .../dag/app/rm/TestTaskSchedulerHelpers.java    | 186 +++++++---
 .../rm/TezTestServiceTaskSchedulerService.java  |  66 +---
 28 files changed, 1357 insertions(+), 1102 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/82c24ac0/TEZ-2003-CHANGES.txt
----------------------------------------------------------------------
diff --git a/TEZ-2003-CHANGES.txt b/TEZ-2003-CHANGES.txt
index 88dd0c7..a51669d 100644
--- a/TEZ-2003-CHANGES.txt
+++ b/TEZ-2003-CHANGES.txt
@@ -35,5 +35,6 @@ ALL CHANGES:
   TEZ-2621. rebase 07/14
   TEZ-2124. Change Node tracking to work per external container source.
   TEZ-2004. Define basic interface for pluggable ContainerLaunchers.
+  TEZ-2005. Define basic interface for pluggable TaskScheduler.
 
 INCOMPATIBLE CHANGES:

http://git-wip-us.apache.org/repos/asf/tez/blob/82c24ac0/tez-api/src/main/java/org/apache/tez/common/ContainerSignatureMatcher.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/common/ContainerSignatureMatcher.java b/tez-api/src/main/java/org/apache/tez/common/ContainerSignatureMatcher.java
new file mode 100644
index 0000000..c0a1245
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/common/ContainerSignatureMatcher.java
@@ -0,0 +1,64 @@
+/* Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.common;
+
+import java.util.Map;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.yarn.api.records.LocalResource;
+
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
+public interface ContainerSignatureMatcher {
+  /**
+   * Checks the compatibility between the specified container signatures.
+   *
+   * @return true if the first signature is a super set of the second
+   *         signature.
+   */
+  public boolean isSuperSet(Object cs1, Object cs2);
+  
+  /**
+   * Checks if the container signatures match exactly
+   * @return true if exact match
+   */
+  public boolean isExactMatch(Object cs1, Object cs2);
+  
+  /**
+   * Gets additional resources specified in lr2, which are not present for lr1
+   * 
+   * @param lr1
+   * @param lr2
+   * @return additional resources specified in lr2, which are not present for lr1
+   */
+  public Map<String, LocalResource> getAdditionalResources(Map<String, LocalResource> lr1,
+      Map<String, LocalResource> lr2);
+
+
+  /**
+   * Do a union of 2 signatures
+   * Pre-condition. This function should only be invoked iff cs1 is compatible with cs2.
+   * i.e. isSuperSet should not return false.
+   * @param cs1 Signature 1 Original signature
+   * @param cs2 Signature 2 New signature
+   * @return Union of 2 signatures
+   */
+  public Object union(Object cs1, Object cs2);
+
+}

http://git-wip-us.apache.org/repos/asf/tez/blob/82c24ac0/tez-api/src/main/java/org/apache/tez/common/ServicePluginLifecycle.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/common/ServicePluginLifecycle.java b/tez-api/src/main/java/org/apache/tez/common/ServicePluginLifecycle.java
new file mode 100644
index 0000000..2eaa7be
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/common/ServicePluginLifecycle.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.common;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public interface ServicePluginLifecycle {
+
+  /**
+   * Perform any additional initialization which may be required beyond the constructor.
+   */
+  void initialize() throws Exception;
+
+  /**
+   * Start the service. This will be invoked after initialization.
+   */
+  void start() throws Exception;
+
+  /**
+   * Shutdown the service. This will be invoked when the service is shutting down.
+   */
+  void shutdown() throws Exception;
+
+}

http://git-wip-us.apache.org/repos/asf/tez/blob/82c24ac0/tez-api/src/main/java/org/apache/tez/serviceplugins/api/TaskScheduler.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/serviceplugins/api/TaskScheduler.java b/tez-api/src/main/java/org/apache/tez/serviceplugins/api/TaskScheduler.java
new file mode 100644
index 0000000..a5b054f
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/serviceplugins/api/TaskScheduler.java
@@ -0,0 +1,85 @@
+/*
+ * 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.serviceplugins.api;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.service.AbstractService;
+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.common.ServicePluginLifecycle;
+
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
+public abstract class TaskScheduler implements ServicePluginLifecycle {
+
+  private final TaskSchedulerContext taskSchedulerContext;
+
+  public TaskScheduler(TaskSchedulerContext taskSchedulerContext) {
+    this.taskSchedulerContext = taskSchedulerContext;
+  }
+
+  @Override
+  public void initialize() throws Exception {
+  }
+
+  @Override
+  public void start() throws Exception {
+  }
+
+  @Override
+  public void shutdown() throws Exception {
+  }
+
+
+  public abstract Resource getAvailableResources();
+
+  public abstract int getClusterNodeCount();
+
+  public abstract void dagComplete();
+
+  public abstract Resource getTotalResources();
+
+  public abstract void blacklistNode(NodeId nodeId);
+
+  public abstract void unblacklistNode(NodeId nodeId);
+
+  public abstract void allocateTask(Object task, Resource capability,
+                                    String[] hosts, String[] racks, Priority priority,
+                                    Object containerSignature, Object clientCookie);
+
+  /**
+   * Allocate affinitized to a specific container
+   */
+  public abstract void allocateTask(Object task, Resource capability,
+                                    ContainerId containerId, Priority priority, Object containerSignature,
+                                    Object clientCookie);
+
+  /** 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);
+
+  public abstract void setShouldUnregister();
+
+  public abstract boolean hasUnregistered();
+
+
+  public final TaskSchedulerContext getContext() {
+    return taskSchedulerContext;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tez/blob/82c24ac0/tez-api/src/main/java/org/apache/tez/serviceplugins/api/TaskSchedulerContext.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/serviceplugins/api/TaskSchedulerContext.java b/tez-api/src/main/java/org/apache/tez/serviceplugins/api/TaskSchedulerContext.java
new file mode 100644
index 0000000..b2c8799
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/serviceplugins/api/TaskSchedulerContext.java
@@ -0,0 +1,114 @@
+/*
+ * 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.serviceplugins.api;
+
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
+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.FinalApplicationStatus;
+import org.apache.hadoop.yarn.api.records.NodeReport;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.tez.common.ContainerSignatureMatcher;
+
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
+
+public interface TaskSchedulerContext {
+
+  public class AppFinalStatus {
+    public final FinalApplicationStatus exitStatus;
+    public final String exitMessage;
+    public final String postCompletionTrackingUrl;
+    public AppFinalStatus(FinalApplicationStatus exitStatus,
+                          String exitMessage,
+                          String posCompletionTrackingUrl) {
+      this.exitStatus = exitStatus;
+      this.exitMessage = exitMessage;
+      this.postCompletionTrackingUrl = posCompletionTrackingUrl;
+    }
+  }
+
+  enum AMState {
+    IDLE, RUNNING_APP, COMPLETED
+  }
+
+  // TODO Post TEZ-2003. Remove references to YARN constructs like Container, ContainerStatus, NodeReport
+  // upcall to app must be outside locks
+  public void taskAllocated(Object task,
+                            Object appCookie,
+                            Container container);
+  // this may end up being called for a task+container pair that the app
+  // has not heard about. this can happen because of a race between
+  // taskAllocated() upcall and deallocateTask() downcall
+  public void containerCompleted(Object taskLastAllocated,
+                                 ContainerStatus containerStatus);
+  public void containerBeingReleased(ContainerId containerId);
+  public void nodesUpdated(List<NodeReport> updatedNodes);
+  public void appShutdownRequested();
+
+  // TODO Post TEZ-2003, this method specifically needs some cleaning up.
+  // ClientAMSecretKey is only relevant when running under YARN. As are ApplicationACLs.
+  public void setApplicationRegistrationData(
+      Resource maxContainerCapability,
+      Map<ApplicationAccessType, String> appAcls,
+      ByteBuffer clientAMSecretKey
+  );
+  public void onError(Throwable t);
+  public float getProgress();
+  public void preemptContainer(ContainerId containerId);
+
+  // TODO Post TEZ-2003. Another method which is primarily relevant to YARN clusters for unregistration.
+  public AppFinalStatus getFinalAppStatus();
+
+
+  // Getters
+
+  // TODO TEZ-2003. To be replaced by getInitialPayload
+  public Configuration getInitialConfiguration();
+
+  public String getAppTrackingUrl();
+
+  /**
+   * A custom cluster identifier allocated to schedulers to generate an AppId, if not making
+   * use of YARN
+   * @return
+   */
+  public long getCustomClusterIdentifier();
+
+  public ContainerSignatureMatcher getContainerSignatureMatcher();
+
+  /**
+   * Get the application attempt id for the running application. Relevant when running under YARN
+   * @return
+   */
+  public ApplicationAttemptId getApplicationAttemptId();
+
+  public String getAppHostName();
+
+  public int getAppClientPort();
+
+  public boolean isSession();
+
+  public AMState getAMState();
+}

http://git-wip-us.apache.org/repos/asf/tez/blob/82c24ac0/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 4c8c227..532e83c 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
@@ -45,6 +45,7 @@ import org.apache.tez.dag.api.TezConstants;
 import org.apache.tez.dag.api.records.DAGProtos;
 import org.apache.tez.dag.api.records.DAGProtos.ConfigurationProto;
 import org.apache.tez.dag.api.records.DAGProtos.PlanKeyValuePair;
+import org.apache.tez.serviceplugins.api.TaskSchedulerContext;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 

http://git-wip-us.apache.org/repos/asf/tez/blob/82c24ac0/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicatorInterface.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicatorInterface.java b/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicatorInterface.java
deleted file mode 100644
index 022cd7b..0000000
--- a/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicatorInterface.java
+++ /dev/null
@@ -1,18 +0,0 @@
-/*
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- *  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tez.dag.api;
-
-public interface TaskCommunicatorInterface {
-}

http://git-wip-us.apache.org/repos/asf/tez/blob/82c24ac0/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 d56fb95..ef27ddf 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
@@ -98,7 +98,6 @@ 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;
@@ -149,7 +148,7 @@ import org.apache.tez.dag.app.rm.TaskSchedulerEventHandler;
 import org.apache.tez.dag.app.rm.container.AMContainerEventType;
 import org.apache.tez.dag.app.rm.container.AMContainerMap;
 import org.apache.tez.dag.app.rm.container.ContainerContextMatcher;
-import org.apache.tez.dag.app.rm.container.ContainerSignatureMatcher;
+import org.apache.tez.common.ContainerSignatureMatcher;
 import org.apache.tez.dag.app.rm.node.AMNodeEventType;
 import org.apache.tez.dag.app.rm.node.AMNodeTracker;
 import org.apache.tez.dag.app.web.WebUIService;

http://git-wip-us.apache.org/repos/asf/tez/blob/82c24ac0/tez-dag/src/main/java/org/apache/tez/dag/app/ServicePluginLifecycleAbstractService.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/ServicePluginLifecycleAbstractService.java b/tez-dag/src/main/java/org/apache/tez/dag/app/ServicePluginLifecycleAbstractService.java
new file mode 100644
index 0000000..dac1b82
--- /dev/null
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/ServicePluginLifecycleAbstractService.java
@@ -0,0 +1,52 @@
+/*
+ * 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 org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.service.AbstractService;
+import org.apache.tez.common.ServicePluginLifecycle;
+
+/**
+ * Provides service lifecycle management over ServicePlugins using {@link AbstractService}
+ * @param <T>
+ */
+public class ServicePluginLifecycleAbstractService<T extends ServicePluginLifecycle> extends AbstractService {
+
+  private final T service;
+
+  public ServicePluginLifecycleAbstractService(T service) {
+    super(service.getClass().getName());
+    this.service = service;
+  }
+
+  @Override
+  public void serviceInit(Configuration unused) throws Exception {
+    service.initialize();
+  }
+
+  @Override
+  public void serviceStart() throws Exception {
+    service.start();
+  }
+
+  @Override
+  public void serviceStop() throws Exception {
+    service.shutdown();
+  }
+
+  public T getService() {
+    return service;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tez/blob/82c24ac0/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 ef789c5..476d00c 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
@@ -20,16 +20,15 @@ package org.apache.tez.dag.app.rm;
 
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
 import java.util.concurrent.PriorityBlockingQueue;
-import java.util.concurrent.TimeUnit;
 import java.util.HashMap;
 import java.util.Iterator;
 import java.util.LinkedHashMap;
 
 import com.google.common.primitives.Ints;
 
+import org.apache.tez.serviceplugins.api.TaskScheduler;
+import org.apache.tez.serviceplugins.api.TaskSchedulerContext;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -43,56 +42,30 @@ import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.tez.serviceplugins.api.TaskAttemptEndReason;
 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.rm.container.ContainerSignatureMatcher;
+import org.apache.tez.common.ContainerSignatureMatcher;
 
-import com.google.common.util.concurrent.ThreadFactoryBuilder;
-
-public class LocalTaskSchedulerService extends TaskSchedulerService {
+public class LocalTaskSchedulerService extends TaskScheduler {
 
   private static final Logger LOG = LoggerFactory.getLogger(LocalTaskSchedulerService.class);
 
-  final TaskSchedulerAppCallback realAppClient;
-  final TaskSchedulerAppCallback appClientDelegate;
   final ContainerSignatureMatcher containerSignatureMatcher;
   final PriorityBlockingQueue<TaskRequest> taskRequestQueue;
+  final Configuration conf;
   AsyncDelegateRequestHandler taskRequestHandler;
   Thread asyncDelegateRequestThread;
-  final ExecutorService appCallbackExecutor;
 
   final HashMap<Object, Container> taskAllocations;
-  final String appHostName;
-  final int appHostPort;
   final String appTrackingUrl;
-  final AppContext appContext;
   final long customContainerAppId;
 
-  public LocalTaskSchedulerService(TaskSchedulerAppCallback appClient,
-      ContainerSignatureMatcher containerSignatureMatcher, String appHostName,
-      int appHostPort, String appTrackingUrl, long customContainerAppId, AppContext appContext) {
-    super(LocalTaskSchedulerService.class.getName());
-    this.realAppClient = appClient;
-    this.appCallbackExecutor = createAppCallbackExecutorService();
-    this.containerSignatureMatcher = containerSignatureMatcher;
-    this.appClientDelegate = createAppCallbackDelegate(appClient);
-    this.appHostName = appHostName;
-    this.appHostPort = appHostPort;
-    this.appTrackingUrl = appTrackingUrl;
-    this.appContext = appContext;
+  public LocalTaskSchedulerService(TaskSchedulerContext taskSchedulerContext) {
+    super(taskSchedulerContext);
     taskRequestQueue = new PriorityBlockingQueue<TaskRequest>();
     taskAllocations = new LinkedHashMap<Object, Container>();
-    this.customContainerAppId = customContainerAppId;
-  }
-
-  private ExecutorService createAppCallbackExecutorService() {
-    return Executors.newSingleThreadExecutor(new ThreadFactoryBuilder()
-        .setNameFormat("TaskSchedulerAppCaller #%d").setDaemon(true).build());
-  }
-
-  private TaskSchedulerAppCallback createAppCallbackDelegate(
-      TaskSchedulerAppCallback realAppClient) {
-    return new TaskSchedulerAppCallbackWrapper(realAppClient,
-        appCallbackExecutor);
+    this.appTrackingUrl = taskSchedulerContext.getAppTrackingUrl();
+    this.containerSignatureMatcher = taskSchedulerContext.getContainerSignatureMatcher();
+    this.customContainerAppId = taskSchedulerContext.getCustomClusterIdentifier();
+    this.conf = taskSchedulerContext.getInitialConfiguration();
   }
 
   @Override
@@ -160,7 +133,7 @@ public class LocalTaskSchedulerService extends TaskSchedulerService {
   }
 
   @Override
-  public void serviceInit(Configuration conf) {
+  public void initialize() {
     taskRequestHandler = createRequestHandler(conf);
     asyncDelegateRequestThread = new Thread(taskRequestHandler);
     asyncDelegateRequestThread.setDaemon(true);
@@ -168,24 +141,22 @@ public class LocalTaskSchedulerService extends TaskSchedulerService {
 
   protected AsyncDelegateRequestHandler createRequestHandler(Configuration conf) {
     return new AsyncDelegateRequestHandler(taskRequestQueue,
-        new LocalContainerFactory(appContext, customContainerAppId),
+        new LocalContainerFactory(getContext().getApplicationAttemptId(), customContainerAppId),
         taskAllocations,
-        appClientDelegate,
+        getContext(),
         conf);
   }
 
   @Override
-  public void serviceStart() {
+  public void start() {
     asyncDelegateRequestThread.start();
   }
 
   @Override
-  public void serviceStop() throws InterruptedException {
+  public void shutdown() throws InterruptedException {
     if (asyncDelegateRequestThread != null) {
       asyncDelegateRequestThread.interrupt();
     }
-    appCallbackExecutor.shutdownNow();
-    appCallbackExecutor.awaitTermination(1000l, TimeUnit.MILLISECONDS);
   }
 
   @Override
@@ -202,12 +173,12 @@ public class LocalTaskSchedulerService extends TaskSchedulerService {
     AtomicInteger nextId;
     final ApplicationAttemptId customAppAttemptId;
 
-    public LocalContainerFactory(AppContext appContext, long appIdLong) {
+    public LocalContainerFactory(ApplicationAttemptId appAttemptId, long customAppId) {
       this.nextId = new AtomicInteger(1);
       ApplicationId appId = ApplicationId
-          .newInstance(appIdLong, appContext.getApplicationAttemptId().getApplicationId().getId());
+          .newInstance(customAppId, appAttemptId.getApplicationId().getId());
       this.customAppAttemptId = ApplicationAttemptId
-          .newInstance(appId, appContext.getApplicationAttemptId().getAttemptId());
+          .newInstance(appId, appAttemptId.getAttemptId());
     }
 
     public Container createContainer(Resource capability, Priority priority) {
@@ -330,18 +301,18 @@ public class LocalTaskSchedulerService extends TaskSchedulerService {
     final BlockingQueue<TaskRequest> taskRequestQueue;
     final LocalContainerFactory localContainerFactory;
     final HashMap<Object, Container> taskAllocations;
-    final TaskSchedulerAppCallback appClientDelegate;
+    final TaskSchedulerContext taskSchedulerContext;
     final int MAX_TASKS;
 
     AsyncDelegateRequestHandler(BlockingQueue<TaskRequest> taskRequestQueue,
         LocalContainerFactory localContainerFactory,
         HashMap<Object, Container> taskAllocations,
-        TaskSchedulerAppCallback appClientDelegate,
+        TaskSchedulerContext taskSchedulerContext,
         Configuration conf) {
       this.taskRequestQueue = taskRequestQueue;
       this.localContainerFactory = localContainerFactory;
       this.taskAllocations = taskAllocations;
-      this.appClientDelegate = appClientDelegate;
+      this.taskSchedulerContext = taskSchedulerContext;
       this.MAX_TASKS = conf.getInt(TezConfiguration.TEZ_AM_INLINE_TASK_EXECUTION_MAX_TASKS,
           TezConfiguration.TEZ_AM_INLINE_TASK_EXECUTION_MAX_TASKS_DEFAULT);
     }
@@ -407,13 +378,13 @@ public class LocalTaskSchedulerService extends TaskSchedulerService {
       Container container = localContainerFactory.createContainer(request.capability,
           request.priority);
       taskAllocations.put(request.task, container);
-      appClientDelegate.taskAllocated(request.task, request.clientCookie, container);
+      taskSchedulerContext.taskAllocated(request.task, request.clientCookie, container);
     }
 
     void deallocateTask(DeallocateTaskRequest request) {
       Container container = taskAllocations.remove(request.task);
       if (container != null) {
-        appClientDelegate.containerBeingReleased(container.getId());
+        taskSchedulerContext.containerBeingReleased(container.getId());
       }
       else {
         boolean deallocationBeforeAllocation = false;

http://git-wip-us.apache.org/repos/asf/tez/blob/82c24ac0/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerAppCallbackImpl.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerAppCallbackImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerAppCallbackImpl.java
deleted file mode 100644
index ea37e94..0000000
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerAppCallbackImpl.java
+++ /dev/null
@@ -1,89 +0,0 @@
-/*
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- *  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tez.dag.app.rm;
-
-import java.nio.ByteBuffer;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
-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.NodeReport;
-import org.apache.hadoop.yarn.api.records.Resource;
-
-public class TaskSchedulerAppCallbackImpl implements TaskSchedulerService.TaskSchedulerAppCallback{
-
-  private final TaskSchedulerEventHandler tseh;
-  private final int schedulerId;
-
-  public TaskSchedulerAppCallbackImpl(TaskSchedulerEventHandler tseh, int schedulerId) {
-    this.tseh = tseh;
-    this.schedulerId = schedulerId;
-  }
-
-  @Override
-  public void taskAllocated(Object task, Object appCookie, Container container) {
-    tseh.taskAllocated(schedulerId, task, appCookie, container);
-  }
-
-  @Override
-  public void containerCompleted(Object taskLastAllocated, ContainerStatus containerStatus) {
-    tseh.containerCompleted(schedulerId, taskLastAllocated, containerStatus);
-  }
-
-  @Override
-  public void containerBeingReleased(ContainerId containerId) {
-    tseh.containerBeingReleased(schedulerId, containerId);
-  }
-
-  @Override
-  public void nodesUpdated(List<NodeReport> updatedNodes) {
-    tseh.nodesUpdated(schedulerId, updatedNodes);
-  }
-
-  @Override
-  public void appShutdownRequested() {
-    tseh.appShutdownRequested(schedulerId);
-  }
-
-  @Override
-  public void setApplicationRegistrationData(Resource maxContainerCapability,
-                                             Map<ApplicationAccessType, String> appAcls,
-                                             ByteBuffer clientAMSecretKey) {
-    tseh.setApplicationRegistrationData(schedulerId, maxContainerCapability, appAcls, clientAMSecretKey);
-  }
-
-  @Override
-  public void onError(Throwable t) {
-    tseh.onError(schedulerId, t);
-  }
-
-  @Override
-  public float getProgress() {
-    return tseh.getProgress(schedulerId);
-  }
-
-  @Override
-  public void preemptContainer(ContainerId containerId) {
-    tseh.preemptContainer(schedulerId, containerId);
-  }
-
-  @Override
-  public AppFinalStatus getFinalAppStatus() {
-    return tseh.getFinalAppStatus();
-  }
-}

http://git-wip-us.apache.org/repos/asf/tez/blob/82c24ac0/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerAppCallbackWrapper.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerAppCallbackWrapper.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerAppCallbackWrapper.java
deleted file mode 100644
index 5de8032..0000000
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerAppCallbackWrapper.java
+++ /dev/null
@@ -1,307 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF 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.nio.ByteBuffer;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Future;
-
-import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
-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.NodeReport;
-import org.apache.hadoop.yarn.api.records.Resource;
-import org.apache.tez.dag.api.TezUncheckedException;
-import org.apache.tez.dag.app.rm.TaskSchedulerService.TaskSchedulerAppCallback;
-
-/**
- * Makes use of an ExecutionService to invoke application callbacks. Methods
- * which return values wait for execution to complete - effectively waiting for
- * all previous events in the queue to complete.
- */
-class TaskSchedulerAppCallbackWrapper implements TaskSchedulerAppCallback {
-
-  private TaskSchedulerAppCallback real;
-
-  ExecutorService executorService;
-  
-  /**
-   * @param real the actual TaskSchedulerAppCallback
-   * @param executorService the ExecutorService to be used to send these events.
-   */
-  public TaskSchedulerAppCallbackWrapper(TaskSchedulerAppCallback real,
-      ExecutorService executorService) {
-    this.real = real;
-    this.executorService = executorService;
-  }
-
-  @Override
-  public void taskAllocated(Object task, Object appCookie, Container container) {
-    executorService.submit(new TaskAllocatedCallable(real, task, appCookie,
-        container));
-  }
-
-  @Override
-  public void containerCompleted(Object taskLastAllocated,
-      ContainerStatus containerStatus) {
-    executorService.submit(new ContainerCompletedCallable(real,
-        taskLastAllocated, containerStatus));
-  }
-
-  @Override
-  public void containerBeingReleased(ContainerId containerId) {
-    executorService
-        .submit(new ContainerBeingReleasedCallable(real, containerId));
-  }
-
-  @Override
-  public void nodesUpdated(List<NodeReport> updatedNodes) {
-    executorService.submit(new NodesUpdatedCallable(real, updatedNodes));
-  }
-
-  @Override
-  public void appShutdownRequested() {
-    executorService.submit(new AppShudownRequestedCallable(real));
-  }
-
-  @Override
-  public void setApplicationRegistrationData(Resource maxContainerCapability,
-      Map<ApplicationAccessType, String> appAcls, ByteBuffer key) {
-    executorService.submit(new SetApplicationRegistrationDataCallable(real,
-        maxContainerCapability, appAcls, key));
-  }
-
-  @Override
-  public void onError(Throwable t) {
-    executorService.submit(new OnErrorCallable(real, t));
-  }
-
-  @Override
-  public float getProgress() {
-    Future<Float> progressFuture = executorService
-        .submit(new GetProgressCallable(real));
-    try {
-      return progressFuture.get();
-    } catch (Exception e) {
-      throw new TezUncheckedException(e);
-    }
-  }
-  
-  @Override
-  public void preemptContainer(ContainerId containerId) {
-    executorService.submit(new PreemptContainerCallable(real, containerId));
-  }
-
-  @Override
-  public AppFinalStatus getFinalAppStatus() {
-    Future<AppFinalStatus> appFinalStatusFuture = executorService
-        .submit(new GetFinalAppStatusCallable(real));
-    try {
-      return appFinalStatusFuture.get();
-    } catch (Exception e) {
-      throw new TezUncheckedException(e);
-    }
-  }
-  
-  
-  static abstract class TaskSchedulerAppCallbackBase {
-
-    protected TaskSchedulerAppCallback app;
-
-    public TaskSchedulerAppCallbackBase(TaskSchedulerAppCallback app) {
-      this.app = app;
-    }
-  }
-
-  static class TaskAllocatedCallable extends TaskSchedulerAppCallbackBase
-      implements Callable<Void> {
-    private final Object task;
-    private final Object appCookie;
-    private final Container container;
-
-    public TaskAllocatedCallable(TaskSchedulerAppCallback app, Object task,
-        Object appCookie, Container container) {
-      super(app);
-      this.task = task;
-      this.appCookie = appCookie;
-      this.container = container;
-    }
-
-    @Override
-    public Void call() throws Exception {
-      app.taskAllocated(task, appCookie, container);
-      return null;
-    }
-  }
-
-  static class ContainerCompletedCallable extends TaskSchedulerAppCallbackBase
-      implements Callable<Void> {
-
-    private final Object taskLastAllocated;
-    private final ContainerStatus containerStatus;
-
-    public ContainerCompletedCallable(TaskSchedulerAppCallback app,
-        Object taskLastAllocated, ContainerStatus containerStatus) {
-      super(app);
-      this.taskLastAllocated = taskLastAllocated;
-      this.containerStatus = containerStatus;
-    }
-
-    @Override
-    public Void call() throws Exception {
-      app.containerCompleted(taskLastAllocated, containerStatus);
-      return null;
-    }
-  }
-
-  static class ContainerBeingReleasedCallable extends
-      TaskSchedulerAppCallbackBase implements Callable<Void> {
-    private final ContainerId containerId;
-
-    public ContainerBeingReleasedCallable(TaskSchedulerAppCallback app,
-        ContainerId containerId) {
-      super(app);
-      this.containerId = containerId;
-    }
-
-    @Override
-    public Void call() throws Exception {
-      app.containerBeingReleased(containerId);
-      return null;
-    }
-  }
-
-  static class NodesUpdatedCallable extends TaskSchedulerAppCallbackBase
-      implements Callable<Void> {
-    private final List<NodeReport> updatedNodes;
-
-    public NodesUpdatedCallable(TaskSchedulerAppCallback app,
-        List<NodeReport> updatedNodes) {
-      super(app);
-      this.updatedNodes = updatedNodes;
-    }
-
-    @Override
-    public Void call() throws Exception {
-      app.nodesUpdated(updatedNodes);
-      return null;
-    }
-  }
-
-  static class AppShudownRequestedCallable extends TaskSchedulerAppCallbackBase
-      implements Callable<Void> {
-
-    public AppShudownRequestedCallable(TaskSchedulerAppCallback app) {
-      super(app);
-    }
-
-    @Override
-    public Void call() throws Exception {
-      app.appShutdownRequested();
-      return null;
-    }
-  }
-
-  static class SetApplicationRegistrationDataCallable extends
-      TaskSchedulerAppCallbackBase implements Callable<Void> {
-
-    private final Resource maxContainerCapability;
-    private final Map<ApplicationAccessType, String> appAcls;
-    private final ByteBuffer key;
-
-    public SetApplicationRegistrationDataCallable(TaskSchedulerAppCallback app,
-        Resource maxContainerCapability,
-        Map<ApplicationAccessType, String> appAcls,
-        ByteBuffer key) {
-      super(app);
-      this.maxContainerCapability = maxContainerCapability;
-      this.appAcls = appAcls;
-      this.key = key;
-    }
-
-    @Override
-    public Void call() throws Exception {
-      app.setApplicationRegistrationData(maxContainerCapability, appAcls, key);
-      return null;
-    }
-  }
-
-  static class OnErrorCallable extends TaskSchedulerAppCallbackBase implements
-      Callable<Void> {
-
-    private final Throwable throwable;
-
-    public OnErrorCallable(TaskSchedulerAppCallback app, Throwable throwable) {
-      super(app);
-      this.throwable = throwable;
-    }
-
-    @Override
-    public Void call() throws Exception {
-      app.onError(throwable);
-      return null;
-    }
-  }
-
-  static class PreemptContainerCallable extends TaskSchedulerAppCallbackBase 
-      implements Callable<Void> {
-    private final ContainerId containerId;
-    
-    public PreemptContainerCallable(TaskSchedulerAppCallback app, ContainerId id) {
-      super(app);
-      this.containerId = id;
-    }
-    
-    @Override
-    public Void call() throws Exception {
-      app.preemptContainer(containerId);
-      return null;
-    }
-  }
-  
-  static class GetProgressCallable extends TaskSchedulerAppCallbackBase
-      implements Callable<Float> {
-
-    public GetProgressCallable(TaskSchedulerAppCallback app) {
-      super(app);
-    }
-
-    @Override
-    public Float call() throws Exception {
-      return app.getProgress();
-    }
-  }
-
-  static class GetFinalAppStatusCallable extends TaskSchedulerAppCallbackBase
-      implements Callable<AppFinalStatus> {
-
-    public GetFinalAppStatusCallable(TaskSchedulerAppCallback app) {
-      super(app);
-    }
-
-    @Override
-    public AppFinalStatus call() throws Exception {
-      return app.getFinalAppStatus();
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/tez/blob/82c24ac0/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerContextImpl.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerContextImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerContextImpl.java
new file mode 100644
index 0000000..890870e
--- /dev/null
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerContextImpl.java
@@ -0,0 +1,174 @@
+/*
+ * 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.nio.ByteBuffer;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
+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.NodeReport;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.tez.common.ContainerSignatureMatcher;
+import org.apache.tez.dag.api.TezUncheckedException;
+import org.apache.tez.dag.app.AppContext;
+import org.apache.tez.serviceplugins.api.TaskSchedulerContext;
+
+public class TaskSchedulerContextImpl implements TaskSchedulerContext {
+
+  private final TaskSchedulerEventHandler tseh;
+  private final AppContext appContext;
+  private final int schedulerId;
+  private final String trackingUrl;
+  private final long customClusterIdentifier;
+  private final String appHostName;
+  private final int clientPort;
+  private final Configuration conf;
+
+  public TaskSchedulerContextImpl(TaskSchedulerEventHandler tseh, AppContext appContext,
+                                  int schedulerId, String trackingUrl, long customClusterIdentifier,
+                                  String appHostname, int clientPort,
+                                  Configuration conf) {
+    this.tseh = tseh;
+    this.appContext = appContext;
+    this.schedulerId = schedulerId;
+    this.trackingUrl = trackingUrl;
+    this.customClusterIdentifier = customClusterIdentifier;
+    this.appHostName = appHostname;
+    this.clientPort = clientPort;
+    this.conf = conf;
+
+  }
+
+  @Override
+  public void taskAllocated(Object task, Object appCookie, Container container) {
+    tseh.taskAllocated(schedulerId, task, appCookie, container);
+  }
+
+  @Override
+  public void containerCompleted(Object taskLastAllocated, ContainerStatus containerStatus) {
+    tseh.containerCompleted(schedulerId, taskLastAllocated, containerStatus);
+  }
+
+  @Override
+  public void containerBeingReleased(ContainerId containerId) {
+    tseh.containerBeingReleased(schedulerId, containerId);
+  }
+
+  @Override
+  public void nodesUpdated(List<NodeReport> updatedNodes) {
+    tseh.nodesUpdated(schedulerId, updatedNodes);
+  }
+
+  @Override
+  public void appShutdownRequested() {
+    tseh.appShutdownRequested(schedulerId);
+  }
+
+  @Override
+  public void setApplicationRegistrationData(Resource maxContainerCapability,
+                                             Map<ApplicationAccessType, String> appAcls,
+                                             ByteBuffer clientAMSecretKey) {
+    tseh.setApplicationRegistrationData(schedulerId, maxContainerCapability, appAcls, clientAMSecretKey);
+  }
+
+  @Override
+  public void onError(Throwable t) {
+    tseh.onError(schedulerId, t);
+  }
+
+  @Override
+  public float getProgress() {
+    return tseh.getProgress(schedulerId);
+  }
+
+  @Override
+  public void preemptContainer(ContainerId containerId) {
+    tseh.preemptContainer(schedulerId, containerId);
+  }
+
+  @Override
+  public AppFinalStatus getFinalAppStatus() {
+    return tseh.getFinalAppStatus();
+  }
+
+  @Override
+  public Configuration getInitialConfiguration() {
+    return conf;
+  }
+
+
+  @Override
+  public String getAppTrackingUrl() {
+    return trackingUrl;
+  }
+
+  @Override
+  public long getCustomClusterIdentifier() {
+    return customClusterIdentifier;
+  }
+
+  @Override
+  public ContainerSignatureMatcher getContainerSignatureMatcher() {
+    return tseh.getContainerSignatureMatcher();
+  }
+
+  @Override
+  public ApplicationAttemptId getApplicationAttemptId() {
+    return appContext.getApplicationAttemptId();
+  }
+
+  @Override
+  public String getAppHostName() {
+    return appHostName;
+  }
+
+  @Override
+  public int getAppClientPort() {
+    return clientPort;
+  }
+
+  @Override
+  public boolean isSession() {
+    return appContext.isSession();
+  }
+
+  @Override
+  public AMState getAMState() {
+    switch (appContext.getAMState()) {
+
+      case NEW:
+      case INITED:
+      case IDLE:
+        return AMState.IDLE;
+      case RECOVERING:
+        // TODO Is this correct for recovery ?
+      case RUNNING:
+        return AMState.RUNNING_APP;
+      case SUCCEEDED:
+      case FAILED:
+      case KILLED:
+      case ERROR:
+        return AMState.COMPLETED;
+      default:
+        throw new TezUncheckedException("Unexpected state " + appContext.getAMState());
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/tez/blob/82c24ac0/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerContextImplWrapper.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerContextImplWrapper.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerContextImplWrapper.java
new file mode 100644
index 0000000..e64ef43
--- /dev/null
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerContextImplWrapper.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.dag.app.rm;
+
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
+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.NodeReport;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.tez.common.ContainerSignatureMatcher;
+import org.apache.tez.dag.api.TezUncheckedException;
+import org.apache.tez.serviceplugins.api.TaskSchedulerContext;
+
+/**
+ * Makes use of an ExecutionService to invoke application callbacks. Methods
+ * which return values wait for execution to complete - effectively waiting for
+ * all previous events in the queue to complete.
+ */
+class TaskSchedulerContextImplWrapper implements TaskSchedulerContext {
+
+  private TaskSchedulerContext real;
+
+  private ExecutorService executorService;
+  
+  /**
+   * @param real the actual TaskSchedulerAppCallback
+   * @param executorService the ExecutorService to be used to send these events.
+   */
+  public TaskSchedulerContextImplWrapper(TaskSchedulerContext real,
+                                         ExecutorService executorService) {
+    this.real = real;
+    this.executorService = executorService;
+  }
+
+  @Override
+  public void taskAllocated(Object task, Object appCookie, Container container) {
+    executorService.submit(new TaskAllocatedCallable(real, task, appCookie,
+        container));
+  }
+
+  @Override
+  public void containerCompleted(Object taskLastAllocated,
+      ContainerStatus containerStatus) {
+    executorService.submit(new ContainerCompletedCallable(real,
+        taskLastAllocated, containerStatus));
+  }
+
+  @Override
+  public void containerBeingReleased(ContainerId containerId) {
+    executorService
+        .submit(new ContainerBeingReleasedCallable(real, containerId));
+  }
+
+  @Override
+  public void nodesUpdated(List<NodeReport> updatedNodes) {
+    executorService.submit(new NodesUpdatedCallable(real, updatedNodes));
+  }
+
+  @Override
+  public void appShutdownRequested() {
+    executorService.submit(new AppShudownRequestedCallable(real));
+  }
+
+  @Override
+  public void setApplicationRegistrationData(Resource maxContainerCapability,
+      Map<ApplicationAccessType, String> appAcls, ByteBuffer key) {
+    executorService.submit(new SetApplicationRegistrationDataCallable(real,
+        maxContainerCapability, appAcls, key));
+  }
+
+  @Override
+  public void onError(Throwable t) {
+    executorService.submit(new OnErrorCallable(real, t));
+  }
+
+  @Override
+  public float getProgress() {
+    Future<Float> progressFuture = executorService
+        .submit(new GetProgressCallable(real));
+    try {
+      return progressFuture.get();
+    } catch (Exception e) {
+      throw new TezUncheckedException(e);
+    }
+  }
+  
+  @Override
+  public void preemptContainer(ContainerId containerId) {
+    executorService.submit(new PreemptContainerCallable(real, containerId));
+  }
+
+  @Override
+  public AppFinalStatus getFinalAppStatus() {
+    Future<AppFinalStatus> appFinalStatusFuture = executorService
+        .submit(new GetFinalAppStatusCallable(real));
+    try {
+      return appFinalStatusFuture.get();
+    } catch (Exception e) {
+      throw new TezUncheckedException(e);
+    }
+  }
+
+  // Getters which do not need to go through a thread. Underlying implementation
+  // does not use locks.
+
+  @Override
+  public Configuration getInitialConfiguration() {
+    return real.getInitialConfiguration();
+  }
+
+  @Override
+  public String getAppTrackingUrl() {
+    return real.getAppTrackingUrl();
+  }
+
+  @Override
+  public long getCustomClusterIdentifier() {
+    return real.getCustomClusterIdentifier();
+  }
+
+  @Override
+  public ContainerSignatureMatcher getContainerSignatureMatcher() {
+    return real.getContainerSignatureMatcher();
+  }
+
+  @Override
+  public ApplicationAttemptId getApplicationAttemptId() {
+    return real.getApplicationAttemptId();
+  }
+
+  @Override
+  public String getAppHostName() {
+    return real.getAppHostName();
+  }
+
+  @Override
+  public int getAppClientPort() {
+    return real.getAppClientPort();
+  }
+
+  @Override
+  public boolean isSession() {
+    return real.isSession();
+  }
+
+  @Override
+  public AMState getAMState() {
+    return real.getAMState();
+  }
+  // End of getters which do not need to go through a thread. Underlying implementation
+  // does not use locks.
+
+
+  static abstract class TaskSchedulerContextCallbackBase {
+
+    protected TaskSchedulerContext app;
+
+    public TaskSchedulerContextCallbackBase(TaskSchedulerContext app) {
+      this.app = app;
+    }
+  }
+
+  static class TaskAllocatedCallable extends TaskSchedulerContextCallbackBase
+      implements Callable<Void> {
+    private final Object task;
+    private final Object appCookie;
+    private final Container container;
+
+    public TaskAllocatedCallable(TaskSchedulerContext app, Object task,
+        Object appCookie, Container container) {
+      super(app);
+      this.task = task;
+      this.appCookie = appCookie;
+      this.container = container;
+    }
+
+    @Override
+    public Void call() throws Exception {
+      app.taskAllocated(task, appCookie, container);
+      return null;
+    }
+  }
+
+  static class ContainerCompletedCallable extends TaskSchedulerContextCallbackBase
+      implements Callable<Void> {
+
+    private final Object taskLastAllocated;
+    private final ContainerStatus containerStatus;
+
+    public ContainerCompletedCallable(TaskSchedulerContext app,
+        Object taskLastAllocated, ContainerStatus containerStatus) {
+      super(app);
+      this.taskLastAllocated = taskLastAllocated;
+      this.containerStatus = containerStatus;
+    }
+
+    @Override
+    public Void call() throws Exception {
+      app.containerCompleted(taskLastAllocated, containerStatus);
+      return null;
+    }
+  }
+
+  static class ContainerBeingReleasedCallable extends
+      TaskSchedulerContextCallbackBase implements Callable<Void> {
+    private final ContainerId containerId;
+
+    public ContainerBeingReleasedCallable(TaskSchedulerContext app,
+        ContainerId containerId) {
+      super(app);
+      this.containerId = containerId;
+    }
+
+    @Override
+    public Void call() throws Exception {
+      app.containerBeingReleased(containerId);
+      return null;
+    }
+  }
+
+  static class NodesUpdatedCallable extends TaskSchedulerContextCallbackBase
+      implements Callable<Void> {
+    private final List<NodeReport> updatedNodes;
+
+    public NodesUpdatedCallable(TaskSchedulerContext app,
+        List<NodeReport> updatedNodes) {
+      super(app);
+      this.updatedNodes = updatedNodes;
+    }
+
+    @Override
+    public Void call() throws Exception {
+      app.nodesUpdated(updatedNodes);
+      return null;
+    }
+  }
+
+  static class AppShudownRequestedCallable extends TaskSchedulerContextCallbackBase
+      implements Callable<Void> {
+
+    public AppShudownRequestedCallable(TaskSchedulerContext app) {
+      super(app);
+    }
+
+    @Override
+    public Void call() throws Exception {
+      app.appShutdownRequested();
+      return null;
+    }
+  }
+
+  static class SetApplicationRegistrationDataCallable extends
+      TaskSchedulerContextCallbackBase implements Callable<Void> {
+
+    private final Resource maxContainerCapability;
+    private final Map<ApplicationAccessType, String> appAcls;
+    private final ByteBuffer key;
+
+    public SetApplicationRegistrationDataCallable(TaskSchedulerContext app,
+        Resource maxContainerCapability,
+        Map<ApplicationAccessType, String> appAcls,
+        ByteBuffer key) {
+      super(app);
+      this.maxContainerCapability = maxContainerCapability;
+      this.appAcls = appAcls;
+      this.key = key;
+    }
+
+    @Override
+    public Void call() throws Exception {
+      app.setApplicationRegistrationData(maxContainerCapability, appAcls, key);
+      return null;
+    }
+  }
+
+  static class OnErrorCallable extends TaskSchedulerContextCallbackBase implements
+      Callable<Void> {
+
+    private final Throwable throwable;
+
+    public OnErrorCallable(TaskSchedulerContext app, Throwable throwable) {
+      super(app);
+      this.throwable = throwable;
+    }
+
+    @Override
+    public Void call() throws Exception {
+      app.onError(throwable);
+      return null;
+    }
+  }
+
+  static class PreemptContainerCallable extends TaskSchedulerContextCallbackBase
+      implements Callable<Void> {
+    private final ContainerId containerId;
+    
+    public PreemptContainerCallable(TaskSchedulerContext app, ContainerId id) {
+      super(app);
+      this.containerId = id;
+    }
+    
+    @Override
+    public Void call() throws Exception {
+      app.preemptContainer(containerId);
+      return null;
+    }
+  }
+  
+  static class GetProgressCallable extends TaskSchedulerContextCallbackBase
+      implements Callable<Float> {
+
+    public GetProgressCallable(TaskSchedulerContext app) {
+      super(app);
+    }
+
+    @Override
+    public Float call() throws Exception {
+      return app.getProgress();
+    }
+  }
+
+  static class GetFinalAppStatusCallable extends TaskSchedulerContextCallbackBase
+      implements Callable<AppFinalStatus> {
+
+    public GetFinalAppStatusCallable(TaskSchedulerContext app) {
+      super(app);
+    }
+
+    @Override
+    public AppFinalStatus call() throws Exception {
+      return app.getFinalAppStatus();
+    }
+  }
+
+  @VisibleForTesting
+  @InterfaceAudience.Private
+  ExecutorService getExecutorService() {
+    return executorService;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tez/blob/82c24ac0/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 1ad0059..d8cf080 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
@@ -25,11 +25,19 @@ import java.nio.ByteBuffer;
 import java.util.List;
 import java.util.Map;
 import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
 import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 
 import com.google.common.annotations.VisibleForTesting;
-import org.apache.tez.dag.app.rm.TaskSchedulerService.TaskSchedulerAppCallback.AppFinalStatus;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.tez.dag.app.ServicePluginLifecycleAbstractService;
+import org.apache.tez.serviceplugins.api.TaskScheduler;
+import org.apache.tez.serviceplugins.api.TaskSchedulerContext;
+import org.apache.tez.serviceplugins.api.TaskSchedulerContext.AppFinalStatus;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -62,7 +70,6 @@ import org.apache.tez.dag.app.dag.event.DAGAppMasterEvent;
 import org.apache.tez.dag.app.dag.event.DAGAppMasterEventSchedulingServiceError;
 import org.apache.tez.dag.app.dag.event.DAGAppMasterEventType;
 import org.apache.tez.dag.app.dag.event.DAGEventSchedulerUpdateTAAssigned;
-import org.apache.tez.dag.app.rm.TaskSchedulerService.TaskSchedulerAppCallback;
 import org.apache.tez.dag.app.rm.container.AMContainer;
 import org.apache.tez.dag.app.rm.container.AMContainerEventAssignTA;
 import org.apache.tez.dag.app.rm.container.AMContainerEventCompleted;
@@ -70,7 +77,7 @@ import org.apache.tez.dag.app.rm.container.AMContainerEventLaunchRequest;
 import org.apache.tez.dag.app.rm.container.AMContainerEventStopRequest;
 import org.apache.tez.dag.app.rm.container.AMContainerEventTASucceeded;
 import org.apache.tez.dag.app.rm.container.AMContainerState;
-import org.apache.tez.dag.app.rm.container.ContainerSignatureMatcher;
+import org.apache.tez.common.ContainerSignatureMatcher;
 import org.apache.tez.dag.app.rm.node.AMNodeEventContainerAllocated;
 import org.apache.tez.dag.app.rm.node.AMNodeEventNodeCountUpdated;
 import org.apache.tez.dag.app.rm.node.AMNodeEventStateChanged;
@@ -106,7 +113,12 @@ public class TaskSchedulerEventHandler extends AbstractService implements
       new AtomicBoolean(false);
   private final WebUIService webUI;
   private final String[] taskSchedulerClasses;
-  protected final TaskSchedulerService []taskSchedulers;
+  protected final TaskScheduler[]taskSchedulers;
+  protected final ServicePluginLifecycleAbstractService []taskSchedulerServiceWrappers;
+
+  // Single executor service shared by all Schedulers for context callbacks
+  @VisibleForTesting
+  final ExecutorService appCallbackExecutor;
 
   private final boolean isPureLocalMode;
   // If running in non local-only mode, the YARN task scheduler will always run to take care of
@@ -147,6 +159,7 @@ public class TaskSchedulerEventHandler extends AbstractService implements
     this.webUI = webUI;
     this.historyUrl = getHistoryUrl();
     this.isPureLocalMode = isPureLocalMode;
+    this.appCallbackExecutor = createAppCallbackExecutorService();
     if (this.webUI != null) {
       this.webUI.setHistoryUrl(this.historyUrl);
     }
@@ -181,7 +194,8 @@ public class TaskSchedulerEventHandler extends AbstractService implements
         this.yarnTaskSchedulerIndex = foundYarnTaskSchedulerIndex;
       }
     }
-    taskSchedulers = new TaskSchedulerService[this.taskSchedulerClasses.length];
+    taskSchedulers = new TaskScheduler[this.taskSchedulerClasses.length];
+    taskSchedulerServiceWrappers = new ServicePluginLifecycleAbstractService[this.taskSchedulerClasses.length];
   }
 
   public Map<ApplicationAccessType, String> getApplicationAcls() {
@@ -205,6 +219,12 @@ public class TaskSchedulerEventHandler extends AbstractService implements
     return taskSchedulers[schedulerId].getTotalResources();
   }
 
+  private ExecutorService createAppCallbackExecutorService() {
+    return Executors.newSingleThreadExecutor(
+        new ThreadFactoryBuilder().setNameFormat("TaskSchedulerAppCallbackExecutor #%d").setDaemon(true)
+            .build());
+  }
+
   public synchronized void handleEvent(AMSchedulerEvent sEvent) {
     LOG.info("Processing the event " + sEvent.toString());
     switch (sEvent.getType()) {
@@ -315,7 +335,8 @@ public class TaskSchedulerEventHandler extends AbstractService implements
       // stopped.
       // AMNodeImpl blacklisting logic does not account for KILLED attempts.
       sendEvent(new AMNodeEventTaskAttemptEnded(appContext.getAllContainers().
-          get(attemptContainerId).getContainer().getNodeId(), event.getSchedulerId(), attemptContainerId,
+          get(attemptContainerId).getContainer().getNodeId(), event.getSchedulerId(),
+          attemptContainerId,
           attempt.getID(), event.getState() == TaskAttemptState.FAILED));
     }
   }
@@ -389,32 +410,30 @@ public class TaskSchedulerEventHandler extends AbstractService implements
         event);
   }
 
-  private TaskSchedulerService createTaskScheduler(String host, int port, String trackingUrl,
+  private TaskScheduler createTaskScheduler(String host, int port, String trackingUrl,
                                                    AppContext appContext,
                                                    String schedulerClassName,
                                                    long customAppIdIdentifier,
                                                    int schedulerId) {
-    TaskSchedulerAppCallback appCallback = new TaskSchedulerAppCallbackImpl(this, schedulerId);
+    TaskSchedulerContext rawContext =
+        new TaskSchedulerContextImpl(this, appContext, schedulerId, trackingUrl,
+            customAppIdIdentifier, host, port, getConfig());
+    TaskSchedulerContext wrappedContext = new TaskSchedulerContextImplWrapper(rawContext, appCallbackExecutor);
     if (schedulerClassName.equals(TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT)) {
       LOG.info("Creating TaskScheduler: YarnTaskSchedulerService");
-      return new YarnTaskSchedulerService(appCallback, this.containerSignatureMatcher,
-          host, port, trackingUrl, appContext);
+      return new YarnTaskSchedulerService(wrappedContext);
     } else if (schedulerClassName.equals(TezConstants.TEZ_AM_SERVICE_PLUGINS_LOCAL_MODE_NAME_DEFAULT)) {
       LOG.info("Creating TaskScheduler: Local TaskScheduler");
-      return new LocalTaskSchedulerService(appCallback, this.containerSignatureMatcher,
-          host, port, trackingUrl, customAppIdIdentifier, appContext);
+      return new LocalTaskSchedulerService(wrappedContext);
     } 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);
+      Class<? extends TaskScheduler> taskSchedulerClazz =
+          (Class<? extends TaskScheduler>) ReflectionUtils.getClazz(schedulerClassName);
       try {
-        Constructor<? extends TaskSchedulerService> ctor = taskSchedulerClazz
-            .getConstructor(TaskSchedulerAppCallback.class, AppContext.class, String.class,
-                int.class, String.class, long.class, Configuration.class);
+        Constructor<? extends TaskScheduler> ctor = taskSchedulerClazz
+            .getConstructor(TaskSchedulerContext.class);
         ctor.setAccessible(true);
-        return ctor.newInstance(appCallback, appContext, host, port, trackingUrl, customAppIdIdentifier,
-            getConfig());
+        return ctor.newInstance(wrappedContext);
       } catch (NoSuchMethodException e) {
         throw new TezUncheckedException(e);
       } catch (InvocationTargetException e) {
@@ -444,6 +463,7 @@ public class TaskSchedulerEventHandler extends AbstractService implements
           customAppIdIdentifier);
       taskSchedulers[i] = createTaskScheduler(host, port,
           trackingUrl, appContext, taskSchedulerClasses[i], customAppIdIdentifier, i);
+      taskSchedulerServiceWrappers[i] = new ServicePluginLifecycleAbstractService(taskSchedulers[i]);
     }
   }
 
@@ -460,8 +480,8 @@ public class TaskSchedulerEventHandler extends AbstractService implements
     instantiateScheduelrs(serviceAddr.getHostName(), serviceAddr.getPort(), trackingUrl, appContext);
 
     for (int i = 0 ; i < taskSchedulers.length ; i++) {
-      taskSchedulers[i].init(getConfig());
-      taskSchedulers[i].start();
+      taskSchedulerServiceWrappers[i].init(getConfig());
+      taskSchedulerServiceWrappers[i].start();
       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 ?
@@ -510,7 +530,7 @@ public class TaskSchedulerEventHandler extends AbstractService implements
   }
   
   @Override
-  public void serviceStop() {
+  public void serviceStop() throws InterruptedException {
     synchronized(this) {
       this.stopEventHandling = true;
       if (eventHandlingThread != null)
@@ -518,9 +538,12 @@ public class TaskSchedulerEventHandler extends AbstractService implements
     }
     for (int i = 0 ; i < taskSchedulers.length ; i++) {
       if (taskSchedulers[i] != null) {
-        taskSchedulers[i].stop();
+        taskSchedulerServiceWrappers[i].stop();
       }
     }
+    LOG.info("Shutting down AppCallbackExecutor");
+    appCallbackExecutor.shutdownNow();
+    appCallbackExecutor.awaitTermination(1000l, TimeUnit.MILLISECONDS);
   }
 
   // TODO TEZ-2003 Consolidate TaskSchedulerAppCallback methods once these methods are moved into context
@@ -716,6 +739,10 @@ public class TaskSchedulerEventHandler extends AbstractService implements
     }
   }
 
+  public ContainerSignatureMatcher getContainerSignatureMatcher() {
+    return containerSignatureMatcher;
+  }
+
   public boolean hasUnregistered() {
     boolean result = true;
     for (int i = 0 ; i < taskSchedulers.length ; i++) {
@@ -757,4 +784,10 @@ public class TaskSchedulerEventHandler extends AbstractService implements
 
     return historyUrl;
   }
+
+  @VisibleForTesting
+  @InterfaceAudience.Private
+  ExecutorService getContextExecutorService() {
+    return appCallbackExecutor;
+  }
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/82c24ac0/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
deleted file mode 100644
index 25fd13e..0000000
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerService.java
+++ /dev/null
@@ -1,111 +0,0 @@
-/**
-* Licensed to the Apache Software Foundation (ASF) under one
-* or more contributor license agreements.  See the NOTICE file
-* distributed with this work for additional information
-* regarding copyright ownership.  The ASF licenses this file
-* to you under the Apache License, Version 2.0 (the
-* "License"); you may not use this file except in compliance
-* with the License.  You may obtain a copy of the License at
-*
-*     http://www.apache.org/licenses/LICENSE-2.0
-*
-* Unless required by applicable law or agreed to in writing, software
-* distributed under the License is distributed on an "AS IS" BASIS,
-* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-* See the License for the specific language governing permissions and
-* limitations under the License.
-*/
-
-package org.apache.tez.dag.app.rm;
-
-import java.nio.ByteBuffer;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.hadoop.service.AbstractService;
-import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
-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.FinalApplicationStatus;
-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.serviceplugins.api.TaskAttemptEndReason;
-
-public abstract class TaskSchedulerService extends AbstractService{
-
-  public TaskSchedulerService(String name) {
-    super(name);
-  }
-
-  public abstract Resource getAvailableResources();
-
-  public abstract int getClusterNodeCount();
-
-  public abstract void dagComplete();
-
-  public abstract Resource getTotalResources();
-
-  public abstract void blacklistNode(NodeId nodeId);
-
-  public abstract void unblacklistNode(NodeId nodeId);
-
-  public abstract void allocateTask(Object task, Resource capability,
-      String[] hosts, String[] racks, Priority priority,
-      Object containerSignature, Object clientCookie);
-  
-  /**
-   * Allocate affinitized to a specific container
-   */
-  public abstract void allocateTask(Object task, Resource capability,
-      ContainerId containerId, Priority priority, Object containerSignature,
-      Object clientCookie);
-
-  /** 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);
-
-  public abstract void setShouldUnregister();
-
-  public abstract boolean hasUnregistered();
-
-  public interface TaskSchedulerAppCallback {
-    public class AppFinalStatus {
-      public final FinalApplicationStatus exitStatus;
-      public final String exitMessage;
-      public final String postCompletionTrackingUrl;
-      public AppFinalStatus(FinalApplicationStatus exitStatus,
-                             String exitMessage,
-                             String posCompletionTrackingUrl) {
-        this.exitStatus = exitStatus;
-        this.exitMessage = exitMessage;
-        this.postCompletionTrackingUrl = posCompletionTrackingUrl;
-      }
-    }
-    // upcall to app must be outside locks
-    public void taskAllocated(Object task,
-                               Object appCookie,
-                               Container container);
-    // this may end up being called for a task+container pair that the app
-    // has not heard about. this can happen because of a race between
-    // taskAllocated() upcall and deallocateTask() downcall
-    public void containerCompleted(Object taskLastAllocated,
-                                    ContainerStatus containerStatus);
-    public void containerBeingReleased(ContainerId containerId);
-    public void nodesUpdated(List<NodeReport> updatedNodes);
-    public void appShutdownRequested();
-    public void setApplicationRegistrationData(
-                                Resource maxContainerCapability,
-                                Map<ApplicationAccessType, String> appAcls,
-                                ByteBuffer clientAMSecretKey
-                                );
-    public void onError(Throwable t);
-    public float getProgress();
-    public void preemptContainer(ContainerId containerId);
-    public AppFinalStatus getFinalAppStatus();
-
-  }
-}


[30/50] [abbrv] tez git commit: TEZ-2005. Define basic interface for pluggable TaskScheduler. (sseth)

Posted by ss...@apache.org.
http://git-wip-us.apache.org/repos/asf/tez/blob/82c24ac0/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 506e991..7d209bc 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
@@ -20,14 +20,10 @@ 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.hadoop.conf.Configuration;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
@@ -37,30 +33,24 @@ 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.serviceplugins.api.TaskAttemptEndReason;
-import org.apache.tez.dag.app.AppContext;
 import org.apache.tez.service.TezTestServiceConfConstants;
+import org.apache.tez.serviceplugins.api.TaskScheduler;
+import org.apache.tez.serviceplugins.api.TaskSchedulerContext;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 
-public class TezTestServiceTaskSchedulerService extends TaskSchedulerService {
+public class TezTestServiceTaskSchedulerService extends TaskScheduler {
 
   private static final Logger
       LOG = LoggerFactory.getLogger(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>();
 
@@ -77,20 +67,14 @@ public class TezTestServiceTaskSchedulerService extends TaskSchedulerService {
 
   // 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) {
+  public TezTestServiceTaskSchedulerService(TaskSchedulerContext taskSchedulerContext) {
     // Accepting configuration here to allow setting up fields as final
-    super(TezTestServiceTaskSchedulerService.class.getName());
-    this.appCallbackExecutor = createAppCallbackExecutorService();
-    this.appClientDelegate = createAppCallbackDelegate(appClient);
-    this.appContext = appContext;
+    super(taskSchedulerContext);
     this.serviceHosts = new LinkedList<String>();
-    this.containerFactory = new ContainerFactory(appContext, customAppIdIdentifier);
+    this.containerFactory = new ContainerFactory(taskSchedulerContext.getApplicationAttemptId(),
+        taskSchedulerContext.getCustomClusterIdentifier());
 
+    Configuration conf = taskSchedulerContext.getInitialConfiguration();
     this.memoryPerInstance = conf
         .getInt(TezTestServiceConfConstants.TEZ_TEST_SERVICE_MEMORY_PER_INSTANCE_MB, -1);
     Preconditions.checkArgument(memoryPerInstance > 0,
@@ -112,10 +96,6 @@ public class TezTestServiceTaskSchedulerService extends TaskSchedulerService {
     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);
@@ -138,13 +118,6 @@ public class TezTestServiceTaskSchedulerService extends TaskSchedulerService {
   }
 
   @Override
-  public void serviceStop() {
-    if (!this.isStopped.getAndSet(true)) {
-      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
@@ -185,7 +158,7 @@ public class TezTestServiceTaskSchedulerService extends TaskSchedulerService {
     Container container =
         containerFactory.createContainer(resourcePerContainer, priority, host, containerPort);
     runningTasks.put(task, container.getId());
-    appClientDelegate.taskAllocated(task, clientCookie, container);
+    getContext().taskAllocated(task, clientCookie, container);
   }
 
 
@@ -196,7 +169,7 @@ public class TezTestServiceTaskSchedulerService extends TaskSchedulerService {
     Container container =
         containerFactory.createContainer(resourcePerContainer, priority, host, containerPort);
     runningTasks.put(task, container.getId());
-    appClientDelegate.taskAllocated(task, clientCookie, container);
+    getContext().taskAllocated(task, clientCookie, container);
   }
 
   @Override
@@ -208,7 +181,7 @@ public class TezTestServiceTaskSchedulerService extends TaskSchedulerService {
           " The query may hang since this \"unknown\" container is now taking up a slot permanently");
       return false;
     }
-    appClientDelegate.containerBeingReleased(containerId);
+    getContext().containerBeingReleased(containerId);
     return true;
   }
 
@@ -229,17 +202,6 @@ public class TezTestServiceTaskSchedulerService extends TaskSchedulerService {
     return true;
   }
 
-  private ExecutorService createAppCallbackExecutorService() {
-    return Executors.newSingleThreadExecutor(new ThreadFactoryBuilder()
-        .setNameFormat("TezTestTaskSchedulerAppCaller").setDaemon(true).build());
-  }
-
-  private TaskSchedulerAppCallback createAppCallbackDelegate(
-      TaskSchedulerAppCallback realAppClient) {
-    return new TaskSchedulerAppCallbackWrapper(realAppClient,
-        appCallbackExecutor);
-  }
-
   private String selectHost(String[] requestedHosts) {
     String host;
     if (requestedHosts != null && requestedHosts.length > 0) {
@@ -257,12 +219,12 @@ public class TezTestServiceTaskSchedulerService extends TaskSchedulerService {
     AtomicInteger nextId;
     final ApplicationAttemptId customAppAttemptId;
 
-    public ContainerFactory(AppContext appContext, long appIdLong) {
+    public ContainerFactory(ApplicationAttemptId appAttemptId, long appIdLong) {
       this.nextId = new AtomicInteger(1);
       ApplicationId appId = ApplicationId
-          .newInstance(appIdLong, appContext.getApplicationAttemptId().getApplicationId().getId());
+          .newInstance(appIdLong, appAttemptId.getApplicationId().getId());
       this.customAppAttemptId = ApplicationAttemptId
-          .newInstance(appId, appContext.getApplicationAttemptId().getAttemptId());
+          .newInstance(appId, appAttemptId.getAttemptId());
     }
 
     public Container createContainer(Resource capability, Priority priority, String hostname, int port) {


[48/50] [abbrv] tez git commit: TEZ-2713 addendum. Fix CHANGES.txt

Posted by ss...@apache.org.
TEZ-2713 addendum. 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/4b0ffdd4
Tree: http://git-wip-us.apache.org/repos/asf/tez/tree/4b0ffdd4
Diff: http://git-wip-us.apache.org/repos/asf/tez/diff/4b0ffdd4

Branch: refs/heads/TEZ-2003
Commit: 4b0ffdd4bd6f8d1bd9542d1ffac98dee4820311b
Parents: 919b3ed
Author: Siddharth Seth <ss...@apache.org>
Authored: Wed Aug 12 10:25:48 2015 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Fri Aug 14 13:49:07 2015 -0700

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


http://git-wip-us.apache.org/repos/asf/tez/blob/4b0ffdd4/TEZ-2003-CHANGES.txt
----------------------------------------------------------------------
diff --git a/TEZ-2003-CHANGES.txt b/TEZ-2003-CHANGES.txt
index adb800b..154fb11 100644
--- a/TEZ-2003-CHANGES.txt
+++ b/TEZ-2003-CHANGES.txt
@@ -47,5 +47,6 @@ ALL CHANGES:
   TEZ-2675. Add javadocs for new pluggable components, fix problems reported by jenkins
   TEZ-2678. Fix comments from reviews - part 1.
   TEZ-2707. Fix comments from reviews - part 2.
+  TEZ-2713. Add tests for node handling when there's multiple schedulers.
 
 INCOMPATIBLE CHANGES:


[21/50] [abbrv] tez git commit: TEZ-2434. Allow tasks to be killed in the runtime. (sseth)

Posted by ss...@apache.org.
TEZ-2434. Allow tasks to be killed in the runtime. (sseth)


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

Branch: refs/heads/TEZ-2003
Commit: 73f73167d4085b137672dbc9ac8c3e4a0c126a8d
Parents: 34f053f
Author: Siddharth Seth <ss...@apache.org>
Authored: Mon May 11 23:34:43 2015 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Fri Aug 14 13:46:44 2015 -0700

----------------------------------------------------------------------
 TEZ-2003-CHANGES.txt                            |   1 +
 .../org/apache/tez/runtime/task/EndReason.java  |  29 ++
 .../tez/runtime/task/TaskRunner2Callable.java   | 132 ++++++
 .../tez/runtime/task/TaskRunner2Result.java     |  48 ++
 .../org/apache/tez/runtime/task/TezChild.java   |  20 +-
 .../apache/tez/runtime/task/TezTaskRunner.java  |   1 +
 .../apache/tez/runtime/task/TezTaskRunner2.java | 434 +++++++++++++++++++
 7 files changed, 655 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/73f73167/TEZ-2003-CHANGES.txt
----------------------------------------------------------------------
diff --git a/TEZ-2003-CHANGES.txt b/TEZ-2003-CHANGES.txt
index 66c110f..5d2e40a 100644
--- a/TEZ-2003-CHANGES.txt
+++ b/TEZ-2003-CHANGES.txt
@@ -24,5 +24,6 @@ ALL CHANGES:
   TEZ-2420. TaskRunner returning before executing the task.
   TEZ-2433. Fixes after rebase 05/08
   TEZ-2438. tez-tools version in the branch is incorrect.
+  TEZ-2434. Allow tasks to be killed in the Runtime.
 
 INCOMPATIBLE CHANGES:

http://git-wip-us.apache.org/repos/asf/tez/blob/73f73167/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/EndReason.java
----------------------------------------------------------------------
diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/EndReason.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/EndReason.java
new file mode 100644
index 0000000..8dc7a87
--- /dev/null
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/EndReason.java
@@ -0,0 +1,29 @@
+/*
+ * 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.task;
+
+public enum EndReason {
+  SUCCESS(false),
+  CONTAINER_STOP_REQUESTED(false),
+  KILL_REQUESTED(true),
+  COMMUNICATION_FAILURE(false),
+  TASK_ERROR(false);
+
+  private final boolean isActionable;
+
+  EndReason(boolean isActionable) {
+    this.isActionable = isActionable;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tez/blob/73f73167/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TaskRunner2Callable.java
----------------------------------------------------------------------
diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TaskRunner2Callable.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TaskRunner2Callable.java
new file mode 100644
index 0000000..7315bbd
--- /dev/null
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TaskRunner2Callable.java
@@ -0,0 +1,132 @@
+/*
+ * 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.task;
+
+import java.lang.reflect.UndeclaredThrowableException;
+import java.security.PrivilegedExceptionAction;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.tez.common.CallableWithNdc;
+import org.apache.tez.runtime.LogicalIOProcessorRuntimeTask;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * This class is responsible for running a {@link LogicalIOProcessorRuntimeTask}.
+ * It does not worry about reporting errors, heartbeats etc.
+ *
+ * Returns success / interrupt / failure status via it's return parameter.
+ *
+ * It's the responsibility of the invoker to handle whatever exceptions may be generated by this.
+ */
+public class TaskRunner2Callable extends CallableWithNdc<TaskRunner2Callable.TaskRunner2CallableResult> {
+
+
+
+  private static final Logger LOG = LoggerFactory.getLogger(TaskRunner2Callable.class);
+
+  private final LogicalIOProcessorRuntimeTask task;
+  private final UserGroupInformation ugi;
+  private final AtomicBoolean stopRequested = new AtomicBoolean(false);
+
+  private volatile Thread ownThread;
+
+  public TaskRunner2Callable(LogicalIOProcessorRuntimeTask task,
+                             UserGroupInformation ugi) {
+    this.task = task;
+    this.ugi = ugi;
+  }
+
+  @Override
+  public TaskRunner2CallableResult callInternal() throws Exception {
+    ownThread = Thread.currentThread();
+    if (stopRequested.get()) {
+      return new TaskRunner2CallableResult(null);
+    }
+    try {
+      return ugi.doAs(new PrivilegedExceptionAction<TaskRunner2CallableResult>() {
+        @Override
+        public TaskRunner2CallableResult run() throws Exception {
+          if (stopRequested.get() || Thread.currentThread().isInterrupted()) {
+            return new TaskRunner2CallableResult(null);
+          }
+          LOG.info("Initializing task" + ", taskAttemptId=" + task.getTaskAttemptID());
+          task.initialize();
+
+          if (!stopRequested.get() && !Thread.currentThread().isInterrupted()) {
+            LOG.info("Running task, taskAttemptId=" + task.getTaskAttemptID());
+            task.run();
+          } else {
+            LOG.info("Stopped before running the processor.");
+            return new TaskRunner2CallableResult(null);
+          }
+
+          if (!stopRequested.get() && !Thread.currentThread().isInterrupted()) {
+            LOG.info("Closing task, taskAttemptId=" + task.getTaskAttemptID());
+            task.close();
+            task.setFrameworkCounters();
+          } else {
+            LOG.info("Stopped before closing the processor");
+            return new TaskRunner2CallableResult(null);
+          }
+          LOG.info("Task completed, taskAttemptId=" + task.getTaskAttemptID() + ", askedToStop=" + stopRequested.get());
+
+
+          return new TaskRunner2CallableResult(null);
+        }
+      });
+    } catch (Throwable t) {
+      if (t instanceof UndeclaredThrowableException) {
+        t = t.getCause();
+      }
+      return new TaskRunner2CallableResult(t);
+    } finally {
+      // If a stop was requested. Make sure the interrupt status is set during the cleanup.
+
+      // One drawback of not communicating out from here is that task complete messages will only
+      // be sent out after cleanup is complete.
+      // For a successful task, however, this should be almost no delay since close has already happened.
+      maybeFixInterruptStatus();
+      LOG.info("Cleaning up task {}, stopRequested={}", task.getTaskAttemptID(), stopRequested.get());
+      task.cleanup();
+    }
+  }
+
+  private void maybeFixInterruptStatus() {
+    if (stopRequested.get() && !Thread.currentThread().isInterrupted()) {
+      Thread.currentThread().interrupt();
+    }
+  }
+
+
+  public void interruptTask() {
+    // Ensure the task is only interrupted once.
+    if (!stopRequested.getAndSet(true)) {
+      if (ownThread != null) {
+        ownThread.interrupt();
+      }
+    }
+  }
+
+  public static class TaskRunner2CallableResult {
+    final Throwable error;
+
+    public TaskRunner2CallableResult(Throwable error) {
+      this.error = error;
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/tez/blob/73f73167/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TaskRunner2Result.java
----------------------------------------------------------------------
diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TaskRunner2Result.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TaskRunner2Result.java
new file mode 100644
index 0000000..07b32ce
--- /dev/null
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TaskRunner2Result.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.runtime.task;
+
+public class TaskRunner2Result {
+  final EndReason endReason;
+  final Throwable error;
+  final boolean containerShutdownRequested;
+
+  public TaskRunner2Result(EndReason endReason, Throwable error, boolean containerShutdownRequested) {
+    this.endReason = endReason;
+    this.error = error;
+    this.containerShutdownRequested = containerShutdownRequested;
+  }
+
+  public EndReason getEndReason() {
+    return endReason;
+  }
+
+  public Throwable getError() {
+    return error;
+  }
+
+  public boolean isContainerShutdownRequested() {
+    return containerShutdownRequested;
+  }
+
+  @Override
+  public String toString() {
+    return "TaskRunner2Result{" +
+        "endReason=" + endReason +
+        ", error=" + error +
+        ", containerShutdownRequested=" + containerShutdownRequested +
+        '}';
+  }
+}

http://git-wip-us.apache.org/repos/asf/tez/blob/73f73167/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 4c8bebc..fff39a0 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
@@ -248,27 +248,27 @@ public class TezChild {
         cleanupOnTaskChanged(containerTask);
 
         // Execute the Actual Task
-        TezTaskRunner taskRunner = new TezTaskRunner(defaultConf, childUGI,
+        TezTaskRunner2 taskRunner = new TezTaskRunner2(defaultConf, childUGI,
             localDirs, containerTask.getTaskSpec(), appAttemptNumber,
             serviceConsumerMetadata, serviceProviderEnvMap, startedInputsMap, taskReporter,
             executor, objectRegistry, pid, executionContext, memAvailable);
         boolean shouldDie;
         try {
-          shouldDie = !taskRunner.run();
+          TaskRunner2Result result = taskRunner.run();
+          shouldDie = result.isContainerShutdownRequested();
           if (shouldDie) {
             LOG.info("Got a shouldDie notification via heartbeats for container {}. Shutting down", containerIdString);
             shutdown();
             return new ContainerExecutionResult(ContainerExecutionResult.ExitStatus.SUCCESS, null,
                 "Asked to die by the AM");
           }
-        } catch (IOException e) {
-          handleError(e);
-          return new ContainerExecutionResult(ContainerExecutionResult.ExitStatus.EXECUTION_FAILURE,
-              e, "TaskExecutionFailure: " + e.getMessage());
-        } catch (TezException e) {
-          handleError(e);
-          return new ContainerExecutionResult(ContainerExecutionResult.ExitStatus.EXECUTION_FAILURE,
-              e, "TaskExecutionFailure: " + e.getMessage());
+          if (result.getError() != null) {
+            Throwable e = result.getError();
+            handleError(result.getError());
+            return new ContainerExecutionResult(
+                ContainerExecutionResult.ExitStatus.EXECUTION_FAILURE,
+                e, "TaskExecutionFailure: " + e.getMessage());
+          }
         } finally {
           FileSystem.closeAllForUGI(childUGI);
         }

http://git-wip-us.apache.org/repos/asf/tez/blob/73f73167/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 dd4620a..a82d87b 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
@@ -106,6 +106,7 @@ public class TezTaskRunner implements TezUmbilical, ErrorReporter {
     if (!Thread.currentThread().isInterrupted()) {
       taskFuture = executor.submit(callable);
     } else {
+      taskReporter.unregisterTask(task.getTaskAttemptID());
       return isShutdownRequested();
     }
     try {

http://git-wip-us.apache.org/repos/asf/tez/blob/73f73167/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezTaskRunner2.java
----------------------------------------------------------------------
diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezTaskRunner2.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezTaskRunner2.java
new file mode 100644
index 0000000..73e5c76
--- /dev/null
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezTaskRunner2.java
@@ -0,0 +1,434 @@
+/*
+ * 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.task;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.Map;
+import java.util.concurrent.ExecutionException;
+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.Preconditions;
+import com.google.common.collect.Multimap;
+import com.google.common.util.concurrent.ListenableFuture;
+import com.google.common.util.concurrent.ListeningExecutorService;
+import org.apache.commons.lang.exception.ExceptionUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSError;
+import org.apache.hadoop.security.UserGroupInformation;
+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.api.ExecutionContext;
+import org.apache.tez.runtime.api.ObjectRegistry;
+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.apache.tez.runtime.task.TaskRunner2Callable.TaskRunner2CallableResult;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class TezTaskRunner2 {
+
+
+  private static final Logger LOG = LoggerFactory.getLogger(TezTaskRunner2.class);
+
+  private final LogicalIOProcessorRuntimeTask task;
+  private final UserGroupInformation ugi;
+
+  private final TaskReporterInterface taskReporter;
+  private final ListeningExecutorService executor;
+  private final UmbilicalAndErrorHandler umbilicalAndErrorHandler;
+
+  // TODO It may be easier to model this as a state machine.
+
+  // Indicates whether a kill has been requested.
+  private final AtomicBoolean killTaskRequested = new AtomicBoolean(false);
+
+  // Indicates whether a stop container has been requested.
+  private final AtomicBoolean stopContainerRequested = new AtomicBoolean(false);
+
+  // Indicates whether the task is complete.
+  private final AtomicBoolean taskComplete = new AtomicBoolean(false);
+
+  // Separate flag from firstException, since an error can be reported without an exception.
+  private final AtomicBoolean errorSeen = new AtomicBoolean(false);
+
+  private volatile EndReason firstEndReason = null;
+
+  // The first exception which caused the task to fail. This could come in from the
+  // TaskRunnerCallable, a failure to heartbeat, or a signalFatalError on the context.
+  private volatile Throwable firstException;
+  private volatile EventMetaData exceptionSourceInfo;
+  private final AtomicBoolean errorReporterToAm = new AtomicBoolean(false);
+
+  private boolean oobSignalErrorInProgress = false;
+  private final Lock oobSignalLock = new ReentrantLock();
+  private final Condition oobSignalCondition = oobSignalLock.newCondition();
+
+  private volatile long taskKillStartTime  = 0;
+
+  // The callable which is being used to execute the task.
+  private volatile TaskRunner2Callable taskRunnerCallable;
+
+  public TezTaskRunner2(Configuration tezConf, UserGroupInformation ugi, String[] localDirs,
+                        TaskSpec taskSpec, int appAttemptNumber,
+                        Map<String, ByteBuffer> serviceConsumerMetadata,
+                        Map<String, String> serviceProviderEnvMap,
+                        Multimap<String, String> startedInputsMap,
+                        TaskReporterInterface taskReporter, ListeningExecutorService executor,
+                        ObjectRegistry objectRegistry, String pid,
+                        ExecutionContext executionContext, long memAvailable) throws
+      IOException {
+    this.ugi = ugi;
+    this.taskReporter = taskReporter;
+    this.executor = executor;
+    this.umbilicalAndErrorHandler = new UmbilicalAndErrorHandler();
+    this.task = new LogicalIOProcessorRuntimeTask(taskSpec, appAttemptNumber, tezConf, localDirs,
+        umbilicalAndErrorHandler, serviceConsumerMetadata, serviceProviderEnvMap, startedInputsMap,
+        objectRegistry, pid, executionContext, memAvailable);
+  }
+
+  /**
+   * Throws an exception only when there was a communication error reported by
+   * the TaskReporter.
+   *
+   * Otherwise, this takes care of all communication with the AM for a a running task - which
+   * includes informing the AM about Failures and Success.
+   *
+   * If a kill request is made to the task, it will not communicate this information to
+   * the AM - since a task KILL is an external event, and whoever invoked it should
+   * be able to track it.
+   *
+   * @return
+   */
+  public TaskRunner2Result run() {
+    try {
+      ListenableFuture<TaskRunner2CallableResult> future = null;
+      synchronized (this) {
+        if (isRunningState()) {
+          // Safe to do this within a synchronized block because we're providing
+          // the handler on which the Reporter will communicate back. Assuming
+          // the register call doesn't end up hanging.
+          taskRunnerCallable = new TaskRunner2Callable(task, ugi);
+          taskReporter.registerTask(task, umbilicalAndErrorHandler);
+          future = executor.submit(taskRunnerCallable);
+        }
+      }
+
+      if (future == null) {
+        return logAndReturnEndResult(firstEndReason, firstException, stopContainerRequested.get());
+      }
+
+      TaskRunner2CallableResult executionResult = null;
+      // The task started. Wait for it to complete.
+      try {
+        executionResult = future.get();
+      } catch (Throwable e) {
+        if (e instanceof ExecutionException) {
+          e = e.getCause();
+        }
+        synchronized (this) {
+          if (isRunningState()) {
+            trySettingEndReason(EndReason.TASK_ERROR);
+            registerFirstException(e, null);
+            LOG.warn("Exception from RunnerCallable", e);
+          }
+        }
+      }
+      if (executionResult != null) {
+        synchronized (this) {
+          if (isRunningState()) {
+            if (executionResult.error != null) {
+              trySettingEndReason(EndReason.TASK_ERROR);
+              registerFirstException(executionResult.error, null);
+            } else {
+              trySettingEndReason(EndReason.SUCCESS);
+              taskComplete.set(true);
+            }
+          }
+        }
+      }
+
+      switch (firstEndReason) {
+        case SUCCESS:
+          try {
+            taskReporter.taskSucceeded(task.getTaskAttemptID());
+            return logAndReturnEndResult(EndReason.SUCCESS, null, stopContainerRequested.get());
+          } catch (IOException e) {
+            // Comm failure. Task can't do much.
+            handleFinalStatusUpdateFailure(e, true);
+            return logAndReturnEndResult(EndReason.COMMUNICATION_FAILURE, e, stopContainerRequested.get());
+          } catch (TezException e) {
+            // Failure from AM. Task can't do much.
+            handleFinalStatusUpdateFailure(e, true);
+            return logAndReturnEndResult(EndReason.COMMUNICATION_FAILURE, e, stopContainerRequested.get());
+          }
+        case CONTAINER_STOP_REQUESTED:
+          // Don't need to send any more communication updates to the AM.
+          return logAndReturnEndResult(firstEndReason, null, stopContainerRequested.get());
+        case KILL_REQUESTED:
+          // Kill is currently not reported to the AM via the TaskRunner. Fix this when the umbilical
+          // supports an indication of kill, if required.
+          return logAndReturnEndResult(firstEndReason, null, stopContainerRequested.get());
+        case COMMUNICATION_FAILURE:
+          // Already seen a communication failure. There's no point trying to report another one.
+          return logAndReturnEndResult(firstEndReason, firstException, stopContainerRequested.get());
+        case TASK_ERROR:
+          // Don't report an error again if it was reported via signalFatalError
+          if (errorReporterToAm.get()) {
+            return logAndReturnEndResult(firstEndReason, firstException, stopContainerRequested.get());
+          } else {
+            String message;
+            if (firstException instanceof FSError) {
+              message = "Encountered an FSError while executing task: " + task.getTaskAttemptID();
+            } else if (firstException instanceof Error) {
+              message = "Encountered an Error while executing task: " + task.getTaskAttemptID();
+            } else {
+              message = "Failure while running task: " + task.getTaskAttemptID();
+            }
+            try {
+              taskReporter.taskFailed(task.getTaskAttemptID(), firstException, message, exceptionSourceInfo);
+              return logAndReturnEndResult(firstEndReason, firstException, stopContainerRequested.get());
+            } catch (IOException e) {
+              // Comm failure. Task can't do much.
+              handleFinalStatusUpdateFailure(e, true);
+              return logAndReturnEndResult(firstEndReason, firstException, stopContainerRequested.get());
+            } catch (TezException e) {
+              // Failure from AM. Task can't do much.
+              handleFinalStatusUpdateFailure(e, true);
+              return logAndReturnEndResult(firstEndReason, firstException, stopContainerRequested.get());
+            }
+          }
+        default:
+          LOG.error("Unexpected EndReason. File a bug");
+          return logAndReturnEndResult(EndReason.TASK_ERROR, new RuntimeException("Unexpected EndReason"), stopContainerRequested.get());
+
+      }
+    } finally {
+      // Clear the interrupted status of the blocking thread, in case it is set after the
+      // InterruptedException was invoked.
+      oobSignalLock.lock();
+      try {
+        while (oobSignalErrorInProgress) {
+          try {
+            oobSignalCondition.await();
+          } catch (InterruptedException e) {
+            LOG.warn("Interrupted while waiting for OOB fatal error to complete");
+            Thread.currentThread().interrupt();
+          }
+        }
+      } finally {
+        oobSignalLock.unlock();
+      }
+      taskReporter.unregisterTask(task.getTaskAttemptID());
+      if (taskKillStartTime != 0) {
+        LOG.info("Time taken to interrupt task={}", (System.currentTimeMillis() - taskKillStartTime));
+      }
+      Thread.interrupted();
+    }
+  }
+
+  public void killTask() {
+    synchronized (this) {
+      if (isRunningState()) {
+        trySettingEndReason(EndReason.KILL_REQUESTED);
+        if (taskRunnerCallable != null) {
+          taskKillStartTime = System.currentTimeMillis();
+          taskRunnerCallable.interruptTask();
+        }
+      }
+    }
+  }
+
+
+  // Checks and changes on these states should happen within a synchronized block,
+  // to ensure the first event is the one that is captured and causes specific behaviour.
+  private boolean isRunningState() {
+    return !taskComplete.get() && !killTaskRequested.get() && !stopContainerRequested.get() &&
+        !errorSeen.get();
+  }
+
+  class UmbilicalAndErrorHandler implements TezUmbilical, ErrorReporter {
+
+    @Override
+    public void addEvents(Collection<TezEvent> events) {
+      // Incoming events from the running task.
+      // Only add these if the task is running.
+      if (isRunningState()) {
+        taskReporter.addEvents(task.getTaskAttemptID(), events);
+      }
+    }
+
+    @Override
+    public void signalFatalError(TezTaskAttemptID taskAttemptID, Throwable t, String message,
+                                 EventMetaData sourceInfo) {
+      // Fatal error reported by the task.
+      boolean isFirstError = false;
+      synchronized (TezTaskRunner2.this) {
+        if (isRunningState()) {
+          if (trySettingEndReason(EndReason.TASK_ERROR)) {
+            if (t == null) {
+              t = new RuntimeException(
+                  message == null ? "FatalError: No user message or exception specified" : message);
+            }
+            registerFirstException(t, sourceInfo);
+            LOG.info("Received notification of a fatal error which will cause the task to die", t);
+            isFirstError = true;
+            errorReporterToAm.set(true);
+            oobSignalErrorInProgress = true;
+          } else {
+            LOG.info(
+                "Ignoring fatal error since the task has ended for reason: {}. IgnoredError: {} ",
+                firstEndReason, (t == null ? message : t.getMessage()));
+          }
+        }
+      }
+
+      // Informing the TaskReporter here because the running task may not be interruptable.
+      // Has to be outside the lock.
+      if (isFirstError) {
+        killTask();
+        try {
+          taskReporter.taskFailed(taskAttemptID, t, getTaskDiagnosticsString(t, message), sourceInfo);
+        } catch (IOException e) {
+          // Comm failure. Task can't do much. The main exception is already registered.
+          handleFinalStatusUpdateFailure(e, true);
+        } catch (TezException e) {
+          // Failure from AM. Task can't do much. The main exception is already registered.
+          handleFinalStatusUpdateFailure(e, true);
+        } finally {
+          oobSignalLock.lock();
+          try {
+            // This message is being sent outside of the main thread, which may end up completing before
+            // this thread runs. Make sure the main run thread does not end till this completes.
+            oobSignalErrorInProgress = false;
+            oobSignalCondition.signal();
+          } finally {
+            oobSignalLock.unlock();
+          }
+        }
+      }
+    }
+
+    @Override
+    public boolean canCommit(TezTaskAttemptID taskAttemptID) throws IOException {
+      // Task checking whether it can commit.
+
+      // Not getting a lock here. It should be alright for the to check with the reporter
+      // on whether a task can commit.
+      if (isRunningState()) {
+        return taskReporter.canCommit(taskAttemptID);
+        // If there's a communication failure here, let it propagate through to the task.
+        // which may throw it back or handle it appropriately.
+      } else {
+        // Don't throw an error since the task is already in the process of shutting down.
+        LOG.info("returning canCommit=false since task is not in a running state");
+        return false;
+      }
+    }
+
+
+    @Override
+    public void reportError(Throwable t) {
+      // Umbilical reporting an error during heartbeat
+      boolean isFirstError = false;
+      synchronized (TezTaskRunner2.this) {
+        if (isRunningState()) {
+          LOG.info("TaskReporter reporter error which will cause the task to fail", t);
+          if (trySettingEndReason(EndReason.COMMUNICATION_FAILURE)) {
+            registerFirstException(t, null);
+            isFirstError = true;
+          }
+          // A race is possible between a task succeeding, and a subsequent timed heartbeat failing.
+          // These errors can be ignored, since a task can only succeed if the synchronous taskSucceeded
+          // method does not throw an exception, in which case task success is registered with the AM.
+          // Leave subsequent heartbeat errors to the next entity to communicate using the TaskReporter
+        } else {
+          LOG.info("Ignoring Communication failure since task with id=" + task.getTaskAttemptID()
+              + " is already complete, is failing or has been asked to terminate");
+        }
+      }
+      // Since this error came from the taskReporter - there's no point attempting to report a failure back to it.
+      if (isFirstError) {
+        killTask();
+      }
+    }
+
+    @Override
+    public void shutdownRequested() {
+      // Umbilical informing about a shutdown request for the container.
+      boolean isFirstTerminate = false;
+      synchronized (TezTaskRunner2.this) {
+        isFirstTerminate = trySettingEndReason(EndReason.CONTAINER_STOP_REQUESTED);
+        // Respect stopContainerRequested since it can come in at any point, despite a previous failure.
+        stopContainerRequested.set(true);
+      }
+
+      if (isFirstTerminate) {
+        killTask();
+      }
+    }
+  }
+
+  private synchronized boolean trySettingEndReason(EndReason endReason) {
+    if (isRunningState()) {
+      firstEndReason = endReason;
+      return true;
+    }
+    return false;
+  }
+
+
+  private void registerFirstException(Throwable t, EventMetaData sourceInfo) {
+    Preconditions.checkState(isRunningState());
+    errorSeen.set(true);
+    firstException = t;
+    this.exceptionSourceInfo = sourceInfo;
+  }
+
+
+  private String getTaskDiagnosticsString(Throwable t, String message) {
+    String diagnostics;
+    if (t != null && message != null) {
+      diagnostics = "exceptionThrown=" + ExceptionUtils.getStackTrace(t) + ", errorMessage="
+          + message;
+    } else if (t == null && message == null) {
+      diagnostics = "Unknown error";
+    } else {
+      diagnostics = t != null ? "exceptionThrown=" + ExceptionUtils.getStackTrace(t)
+          : " errorMessage=" + message;
+    }
+    return diagnostics;
+  }
+
+  private TaskRunner2Result logAndReturnEndResult(EndReason endReason, Throwable firstError,
+                                                  boolean stopContainerRequested) {
+    TaskRunner2Result result = new TaskRunner2Result(endReason, firstError, stopContainerRequested);
+    LOG.info("TaskRunnerResult for {} : {}  ", task.getTaskAttemptID(), result);
+    return result;
+  }
+
+  private void handleFinalStatusUpdateFailure(Throwable t, boolean successReportAttempted) {
+    // TODO Ideally differentiate between FAILED/KILLED
+    LOG.warn("Failure while reporting state= {} to AM", (successReportAttempted ? "success" : "failure/killed"), t);
+  }
+}
\ No newline at end of file


[11/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/3a602428
Tree: http://git-wip-us.apache.org/repos/asf/tez/tree/3a602428
Diff: http://git-wip-us.apache.org/repos/asf/tez/diff/3a602428

Branch: refs/heads/TEZ-2003
Commit: 3a60242875720efa5c367e55a434264b098309f5
Parents: a5f872e
Author: Siddharth Seth <ss...@apache.org>
Authored: Tue Apr 28 13:41:12 2015 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Fri Aug 14 13:46:43 2015 -0700

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


http://git-wip-us.apache.org/repos/asf/tez/blob/3a602428/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/3a602428/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 7cdf292..cbaed99 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,27 +17,21 @@
 
 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.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;
@@ -61,7 +55,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;
@@ -150,7 +143,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]);
@@ -173,6 +166,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/3a602428/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 ab9fafe..2208220 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
@@ -47,11 +47,9 @@ 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.TezConfiguration;
+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;
@@ -82,7 +80,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;
@@ -98,11 +98,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>();
@@ -118,6 +120,8 @@ public class TestTaskAttemptListenerImplTezDag {
     doReturn(amContainerMap).when(appContext).getAllContainers();
     doReturn(clock).when(appContext).getClock();
     
+    doReturn(appAttemptId).when(appContext).getApplicationAttemptId();
+    doReturn(credentials).when(appContext).getAppCredentials();
     NodeId nodeId = NodeId.newInstance("localhost", 0);
     AMContainer amContainer = mock(AMContainer.class);
     Container container = mock(Container.class);
@@ -160,7 +164,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);
 
@@ -190,7 +194,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);
@@ -320,7 +324,6 @@ public class TestTaskAttemptListenerImplTezDag {
       int fromEventId, int maxEvents, int nextFromEventId,
       List<TezEvent> sendEvents) throws IOException, TezException {
     ContainerId containerId = createContainerId(appId, 1);
-    long requestId = 0;
     Vertex vertex = mock(Vertex.class);
 
     doReturn(vertex).when(dag).getVertex(vertexID);
@@ -328,13 +331,13 @@ public class TestTaskAttemptListenerImplTezDag {
     TaskAttemptEventInfo eventInfo = new TaskAttemptEventInfo(nextFromEventId, sendEvents, 0);
     doReturn(eventInfo).when(vertex).getTaskAttemptTezEvents(taskAttemptID, fromEventId, 0, maxEvents);
 
-    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();
     doReturn(maxEvents).when(request).getMaxEvents();
     doReturn(fromEventId).when(request).getStartIndex();
@@ -348,6 +351,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(


[29/50] [abbrv] tez git commit: TEZ-2004. Define basic interface for pluggable ContainerLaunchers. (sseth)

Posted by ss...@apache.org.
TEZ-2004. Define basic interface for pluggable ContainerLaunchers. (sseth)


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

Branch: refs/heads/TEZ-2003
Commit: af1cc7236f100bcc7efcf9b48aae24357fa851bd
Parents: a5dfca2
Author: Siddharth Seth <ss...@apache.org>
Authored: Mon Jul 20 15:52:24 2015 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Fri Aug 14 13:46:45 2015 -0700

----------------------------------------------------------------------
 TEZ-2003-CHANGES.txt                            |   1 +
 .../serviceplugins/api/ContainerEndReason.java  |  31 +++++
 .../api/ContainerLaunchRequest.java             |  81 +++++++++++
 .../serviceplugins/api/ContainerLauncher.java   |  46 +++++++
 .../api/ContainerLauncherContext.java           |  54 ++++++++
 .../api/ContainerLauncherOperationBase.java     |  58 ++++++++
 .../api/ContainerStopRequest.java               |  47 +++++++
 .../api/TaskAttemptEndReason.java               |  32 +++++
 .../org/apache/tez/common/TezUtilsInternal.java |   9 +-
 .../apache/tez/dag/api/ContainerEndReason.java  |  27 ----
 .../tez/dag/api/TaskAttemptEndReason.java       |  27 ----
 .../apache/tez/dag/api/TaskCommunicator.java    |   9 ++
 .../tez/dag/api/TaskCommunicatorContext.java    |   2 +-
 .../tez/dag/api/TaskCommunicatorInterface.java  |  18 +++
 .../java/org/apache/tez/dag/app/AppContext.java |   5 +
 .../dag/app/ContainerLauncherContextImpl.java   | 101 ++++++++++++++
 .../org/apache/tez/dag/app/DAGAppMaster.java    |  15 +++
 .../apache/tez/dag/app/TaskAttemptListener.java |   6 +-
 .../dag/app/TaskAttemptListenerImpTezDag.java   |   4 +-
 .../dag/app/TaskCommunicatorContextImpl.java    |   6 +-
 .../tez/dag/app/TezTaskCommunicatorImpl.java    |  12 +-
 .../tez/dag/app/launcher/ContainerLauncher.java |  29 ----
 .../dag/app/launcher/ContainerLauncherImpl.java | 128 ++++++++----------
 .../app/launcher/ContainerLauncherRouter.java   |  52 +++++++-
 .../tez/dag/app/launcher/ContainerOp.java       |  62 +++++++++
 .../app/launcher/LocalContainerLauncher.java    | 123 ++++++++---------
 .../tez/dag/app/rm/AMSchedulerEventTAEnded.java |   2 +-
 .../dag/app/rm/LocalTaskSchedulerService.java   |   2 +-
 .../tez/dag/app/rm/NMCommunicatorEvent.java     |  18 ++-
 .../rm/NMCommunicatorLaunchRequestEvent.java    |  11 +-
 .../app/rm/NMCommunicatorStopRequestEvent.java  |   4 +-
 .../tez/dag/app/rm/TaskSchedulerService.java    |   2 +-
 .../dag/app/rm/YarnTaskSchedulerService.java    |   2 +-
 .../rm/container/AMContainerEventCompleted.java |   2 +-
 .../dag/app/rm/container/AMContainerImpl.java   |   9 +-
 .../apache/tez/dag/app/MockDAGAppMaster.java    |  63 ++++-----
 .../app/TestTaskAttemptListenerImplTezDag.java  |   4 +-
 .../app/TestTaskAttemptListenerImplTezDag2.java |   2 +-
 .../tez/dag/app/rm/TestContainerReuse.java      |   4 +-
 .../app/rm/TestLocalTaskSchedulerService.java   |   1 -
 .../dag/app/rm/container/TestAMContainer.java   |   4 +-
 .../TezTestServiceContainerLauncher.java        | 133 +++++++++----------
 .../TezTestServiceNoOpContainerLauncher.java    |  53 +++-----
 .../rm/TezTestServiceTaskSchedulerService.java  |   2 +-
 .../TezTestServiceTaskCommunicatorImpl.java     |   4 +-
 45 files changed, 887 insertions(+), 420 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/af1cc723/TEZ-2003-CHANGES.txt
----------------------------------------------------------------------
diff --git a/TEZ-2003-CHANGES.txt b/TEZ-2003-CHANGES.txt
index 604947c..88dd0c7 100644
--- a/TEZ-2003-CHANGES.txt
+++ b/TEZ-2003-CHANGES.txt
@@ -34,5 +34,6 @@ ALL CHANGES:
   TEZ-2526. Fix version for tez-history-parser.
   TEZ-2621. rebase 07/14
   TEZ-2124. Change Node tracking to work per external container source.
+  TEZ-2004. Define basic interface for pluggable ContainerLaunchers.
 
 INCOMPATIBLE CHANGES:

http://git-wip-us.apache.org/repos/asf/tez/blob/af1cc723/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ContainerEndReason.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ContainerEndReason.java b/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ContainerEndReason.java
new file mode 100644
index 0000000..ab8619f
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ContainerEndReason.java
@@ -0,0 +1,31 @@
+/*
+ * 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.serviceplugins.api;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
+public enum ContainerEndReason {
+  NODE_FAILED, // Completed because the node running the container was marked as dead
+  INTERNAL_PREEMPTION, // Preempted by the AM, due to an internal decision
+  EXTERNAL_PREEMPTION, // Preempted due to cluster contention
+  APPLICATION_ERROR, // An error in the AM caused by user code
+  FRAMEWORK_ERROR, // An error in the AM - likely a bug.
+  LAUNCH_FAILED, // Failure to launch the container
+  COMPLETED, // Completed via normal flow
+  OTHER
+}

http://git-wip-us.apache.org/repos/asf/tez/blob/af1cc723/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ContainerLaunchRequest.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ContainerLaunchRequest.java b/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ContainerLaunchRequest.java
new file mode 100644
index 0000000..cfd7ca7
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ContainerLaunchRequest.java
@@ -0,0 +1,81 @@
+/*
+ * 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.serviceplugins.api;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
+import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.hadoop.yarn.api.records.Token;
+
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
+public class ContainerLaunchRequest extends ContainerLauncherOperationBase {
+
+  private final ContainerLaunchContext clc;
+  private final Container container;
+  private final String schedulerName;
+  private final String taskCommName;
+
+  public ContainerLaunchRequest(NodeId nodeId,
+                                ContainerId containerId,
+                                Token containerToken,
+                                ContainerLaunchContext clc,
+                                Container container, String schedulerName, String taskCommName) {
+    super(nodeId, containerId, containerToken);
+    this.clc = clc;
+    this.container = container;
+    this.schedulerName = schedulerName;
+    this.taskCommName = taskCommName;
+  }
+
+
+  // TODO Post TEZ-2003. TEZ-2625. ContainerLaunchContext needs to be built here instead of being passed in.
+  // Basic specifications need to be provided here
+  public ContainerLaunchContext getContainerLaunchContext() {
+    return clc;
+  }
+
+  /**
+   * Get the name of the task communicator which will be used to communicate
+   * with the task that will run in this container.
+   * @return
+   */
+  public String getTaskCommunicatorName() {
+    return taskCommName;
+  }
+
+  /**
+   * Get the name of the scheduler which allocated this container.
+   * @return
+   */
+  public String getSchedulerName() {
+    return schedulerName;
+  }
+
+  @Override
+  public String toString() {
+    return "ContainerLaunchRequest{" +
+        "nodeId=" + getNodeId() +
+        ", containerId=" + getContainerId() +
+        ", clc=" + clc +
+        ", container=" + container +
+        ", schedulerName='" + schedulerName + '\'' +
+        ", taskCommName='" + taskCommName + '\'' +
+        '}';
+  }
+}

http://git-wip-us.apache.org/repos/asf/tez/blob/af1cc723/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ContainerLauncher.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ContainerLauncher.java b/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ContainerLauncher.java
new file mode 100644
index 0000000..218edb6
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ContainerLauncher.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.serviceplugins.api;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.service.AbstractService;
+
+/**
+ * Plugin to allow custom container launchers to be written to launch containers on different types
+ * of executors.
+ */
+
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
+public abstract class ContainerLauncher extends AbstractService {
+
+  private final ContainerLauncherContext containerLauncherContext;
+
+  // TODO TEZ-2003 Simplify this by moving away from AbstractService. Potentially Guava AbstractService.
+  // A serviceInit(Configuration) is not likely to be very useful, and will expose unnecessary internal
+  // configuration to the services if populated with the AM Configuration
+  public ContainerLauncher(String name, ContainerLauncherContext containerLauncherContext) {
+    super(name);
+    this.containerLauncherContext = containerLauncherContext;
+  }
+
+  public final ContainerLauncherContext getContext() {
+    return this.containerLauncherContext;
+  }
+
+  public abstract void launchContainer(ContainerLaunchRequest launchRequest);
+  public abstract void stopContainer(ContainerStopRequest stopRequest);
+}

http://git-wip-us.apache.org/repos/asf/tez/blob/af1cc723/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ContainerLauncherContext.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ContainerLauncherContext.java b/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ContainerLauncherContext.java
new file mode 100644
index 0000000..836dc4a
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ContainerLauncherContext.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.serviceplugins.api;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
+public interface ContainerLauncherContext {
+
+  // TODO Post TEZ-2003. Tez abstraction for ContainerId, NodeId, other YARN constructs
+
+  // Reporting APIs
+  void containerLaunched(ContainerId containerId);
+
+  void containerLaunchFailed(ContainerId containerId, String diagnostics);
+
+  void containerStopRequested(ContainerId containerId);
+
+  void containerStopFailed(ContainerId containerId, String diagnostics);
+
+  // TODO Post TEZ-2003. TaskAttemptEndReason does not belong here, and is an unnecessary leak.
+  // ContainerCompleted is normally generated by the scheduler in case of YARN since the RM informs about completion.
+  // For other sources, there may not be a central entity making this information available. The ContainerLauncher
+  // on the stop request will likely be the best place to generate it.
+  void containerCompleted(ContainerId containerId, int exitStatus, String diagnostics, TaskAttemptEndReason endReason);
+
+  // Lookup APIs
+
+  // TODO TEZ-2003. To be replaced by getInitialPayload once the DAG API is changed.
+  Configuration getInitialConfiguration();
+
+  int getNumNodes(String sourceName);
+
+  ApplicationAttemptId getApplicationAttemptId();
+
+  Object getTaskCommunicatorMetaInfo(String taskCommName);
+}

http://git-wip-us.apache.org/repos/asf/tez/blob/af1cc723/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ContainerLauncherOperationBase.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ContainerLauncherOperationBase.java b/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ContainerLauncherOperationBase.java
new file mode 100644
index 0000000..29e0420
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ContainerLauncherOperationBase.java
@@ -0,0 +1,58 @@
+/*
+ * 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.serviceplugins.api;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.hadoop.yarn.api.records.Token;
+
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
+public class ContainerLauncherOperationBase {
+
+  private final NodeId nodeId;
+  private final ContainerId containerId;
+  private final Token containerToken;
+
+  public ContainerLauncherOperationBase(NodeId nodeId,
+                                        ContainerId containerId,
+                                        Token containerToken) {
+    this.nodeId = nodeId;
+    this.containerId = containerId;
+    this.containerToken = containerToken;
+  }
+
+  public NodeId getNodeId() {
+    return nodeId;
+  }
+
+  public ContainerId getContainerId() {
+    return containerId;
+  }
+
+  public Token getContainerToken() {
+    return containerToken;
+  }
+
+  @Override
+  public String toString() {
+    return "ContainerLauncherOperationBase{" +
+        "nodeId=" + nodeId +
+        ", containerId=" + containerId +
+        '}';
+  }
+}

http://git-wip-us.apache.org/repos/asf/tez/blob/af1cc723/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ContainerStopRequest.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ContainerStopRequest.java b/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ContainerStopRequest.java
new file mode 100644
index 0000000..cb0af31
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ContainerStopRequest.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.serviceplugins.api;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.hadoop.yarn.api.records.Token;
+
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
+public class ContainerStopRequest extends ContainerLauncherOperationBase {
+
+  private final String schedulerName;
+  private final String taskCommName;
+
+  public ContainerStopRequest(NodeId nodeId,
+                              ContainerId containerId,
+                              Token containerToken, String schedulerName, String taskCommName) {
+    super(nodeId, containerId, containerToken);
+    this.schedulerName = schedulerName;
+    this.taskCommName = taskCommName;
+  }
+
+  @Override
+  public String toString() {
+    return "ContainerStopRequest{" +
+        "nodeId=" + getNodeId() +
+        ", containerId=" + getContainerId() +
+        ", schedulerName='" + schedulerName + '\'' +
+        ", taskCommName='" + taskCommName + '\'' +
+        '}';
+  }
+}

http://git-wip-us.apache.org/repos/asf/tez/blob/af1cc723/tez-api/src/main/java/org/apache/tez/serviceplugins/api/TaskAttemptEndReason.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/serviceplugins/api/TaskAttemptEndReason.java b/tez-api/src/main/java/org/apache/tez/serviceplugins/api/TaskAttemptEndReason.java
new file mode 100644
index 0000000..4255c28
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/serviceplugins/api/TaskAttemptEndReason.java
@@ -0,0 +1,32 @@
+/*
+ * 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.serviceplugins.api;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
+public enum TaskAttemptEndReason {
+  NODE_FAILED, // Completed because the node running the container was marked as dead
+  COMMUNICATION_ERROR, // Communication error with the task
+  SERVICE_BUSY, // External service busy
+  INTERNAL_PREEMPTION, // Preempted by the AM, due to an internal decision
+  EXTERNAL_PREEMPTION, // Preempted due to cluster contention
+  APPLICATION_ERROR, // An error in the AM caused by user code
+  FRAMEWORK_ERROR, // An error in the AM - likely a bug.
+  CONTAINER_EXITED,
+  OTHER // Unknown reason
+}

http://git-wip-us.apache.org/repos/asf/tez/blob/af1cc723/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 0bdeb79..4c8c227 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
@@ -35,13 +35,12 @@ import java.util.zip.Inflater;
 import com.google.protobuf.ByteString;
 import com.google.protobuf.Descriptors;
 import com.google.protobuf.TextFormat;
-import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 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.serviceplugins.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;
@@ -256,6 +255,8 @@ public class TezUtilsInternal {
         return TaskAttemptTerminationCause.FRAMEWORK_ERROR;
       case NODE_FAILED:
         return TaskAttemptTerminationCause.NODE_FAILED;
+      case CONTAINER_EXITED:
+        return TaskAttemptTerminationCause.CONTAINER_EXITED;
       case OTHER:
         return TaskAttemptTerminationCause.UNKNOWN_ERROR;
       default:
@@ -283,6 +284,8 @@ public class TezUtilsInternal {
         return TaskAttemptEndReason.FRAMEWORK_ERROR;
       case NODE_FAILED:
         return TaskAttemptEndReason.NODE_FAILED;
+      case CONTAINER_EXITED:
+        return TaskAttemptEndReason.CONTAINER_EXITED;
       case INTERRUPTED_BY_SYSTEM:
       case INTERRUPTED_BY_USER:
       case UNKNOWN_ERROR:
@@ -296,7 +299,7 @@ public class TezUtilsInternal {
       case OUTPUT_LOST:
       case TASK_HEARTBEAT_ERROR:
       case CONTAINER_LAUNCH_FAILED:
-      case CONTAINER_EXITED:
+
       case CONTAINER_STOPPED:
       case NODE_DISK_ERROR:
       default:

http://git-wip-us.apache.org/repos/asf/tez/blob/af1cc723/tez-common/src/main/java/org/apache/tez/dag/api/ContainerEndReason.java
----------------------------------------------------------------------
diff --git a/tez-common/src/main/java/org/apache/tez/dag/api/ContainerEndReason.java b/tez-common/src/main/java/org/apache/tez/dag/api/ContainerEndReason.java
deleted file mode 100644
index e13e886..0000000
--- a/tez-common/src/main/java/org/apache/tez/dag/api/ContainerEndReason.java
+++ /dev/null
@@ -1,27 +0,0 @@
-/*
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- *  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tez.dag.api;
-
-// TODO TEZ-2003 Expose as a public API
-public enum ContainerEndReason {
-  NODE_FAILED, // Completed because the node running the container was marked as dead
-  INTERNAL_PREEMPTION, // Preempted by the AM, due to an internal decision
-  EXTERNAL_PREEMPTION, // Preempted due to cluster contention
-  APPLICATION_ERROR, // An error in the AM caused by user code
-  FRAMEWORK_ERROR, // An error in the AM - likely a bug.
-  LAUNCH_FAILED, // Failure to launch the container
-  COMPLETED, // Completed via normal flow
-  OTHER
-}

http://git-wip-us.apache.org/repos/asf/tez/blob/af1cc723/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
deleted file mode 100644
index de78d21..0000000
--- a/tez-common/src/main/java/org/apache/tez/dag/api/TaskAttemptEndReason.java
+++ /dev/null
@@ -1,27 +0,0 @@
-/*
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tez.dag.api;
-
-// TODO TEZ-2003 Expose as a public API
-public enum TaskAttemptEndReason {
-  NODE_FAILED, // Completed because the node running the container was marked as dead
-  COMMUNICATION_ERROR, // Communication error with the task
-  SERVICE_BUSY, // External service busy
-  INTERNAL_PREEMPTION, // Preempted by the AM, due to an internal decision
-  EXTERNAL_PREEMPTION, // Preempted due to cluster contention
-  APPLICATION_ERROR, // An error in the AM caused by user code
-  FRAMEWORK_ERROR, // An error in the AM - likely a bug.
-  OTHER // Unknown reason
-}

http://git-wip-us.apache.org/repos/asf/tez/blob/af1cc723/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 d0a006b..05e437c 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,8 @@ 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.serviceplugins.api.ContainerEndReason;
+import org.apache.tez.serviceplugins.api.TaskAttemptEndReason;
 import org.apache.tez.dag.records.TezTaskAttemptID;
 import org.apache.tez.runtime.api.impl.TaskSpec;
 
@@ -87,4 +89,11 @@ public abstract class TaskCommunicator extends AbstractService {
   // 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);
+
+  /**
+   * Share meta-information such as host:port information where the Task Communicator may be listening.
+   * Primarily for use by compatible launchers to learn this information.
+   * @return
+   */
+  public abstract Object getMetaInfo();
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/af1cc723/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 56345ab..b6e63f7 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,13 +16,13 @@ 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;
 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.serviceplugins.api.TaskAttemptEndReason;
 import org.apache.tez.dag.records.TezTaskAttemptID;
 
 

http://git-wip-us.apache.org/repos/asf/tez/blob/af1cc723/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicatorInterface.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicatorInterface.java b/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicatorInterface.java
new file mode 100644
index 0000000..022cd7b
--- /dev/null
+++ b/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicatorInterface.java
@@ -0,0 +1,18 @@
+/*
+ * 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;
+
+public interface TaskCommunicatorInterface {
+}

http://git-wip-us.apache.org/repos/asf/tez/blob/af1cc723/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 1ccb10b..516fcef 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
@@ -118,4 +118,9 @@ public interface AppContext {
   public Integer getTaskCommunicatorIdentifier(String name);
   public Integer getTaskScheduerIdentifier(String name);
   public Integer getContainerLauncherIdentifier(String name);
+
+  public String getTaskCommunicatorName(int taskCommId);
+  public String getTaskSchedulerName(int schedulerId);
+  public String getContainerLauncherName(int launcherId);
+
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/af1cc723/tez-dag/src/main/java/org/apache/tez/dag/app/ContainerLauncherContextImpl.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/ContainerLauncherContextImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/ContainerLauncherContextImpl.java
new file mode 100644
index 0000000..997775a
--- /dev/null
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/ContainerLauncherContextImpl.java
@@ -0,0 +1,101 @@
+/*
+ * 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 org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.tez.common.TezUtilsInternal;
+import org.apache.tez.serviceplugins.api.ContainerLauncherContext;
+import org.apache.tez.serviceplugins.api.TaskAttemptEndReason;
+import org.apache.tez.dag.app.rm.container.AMContainerEvent;
+import org.apache.tez.dag.app.rm.container.AMContainerEventCompleted;
+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.AMContainerEventStopFailed;
+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 ContainerLauncherContextImpl implements ContainerLauncherContext {
+
+  private final AppContext context;
+  private final TaskAttemptListener tal;
+
+  public ContainerLauncherContextImpl(AppContext appContext, TaskAttemptListener tal) {
+    this.context = appContext;
+    this.tal = tal;
+  }
+
+  @Override
+  public void containerLaunched(ContainerId containerId) {
+    context.getEventHandler().handle(
+        new AMContainerEventLaunched(containerId));
+    ContainerLaunchedEvent lEvt = new ContainerLaunchedEvent(
+        containerId, context.getClock().getTime(), context.getApplicationAttemptId());
+    context.getHistoryHandler().handle(new DAGHistoryEvent(
+        null, lEvt));
+
+  }
+
+  @Override
+  public void containerLaunchFailed(ContainerId containerId, String diagnostics) {
+    context.getEventHandler().handle(new AMContainerEventLaunchFailed(containerId, diagnostics));
+  }
+
+  @Override
+  public void containerStopRequested(ContainerId containerId) {
+    context.getEventHandler().handle(
+        new AMContainerEvent(containerId, AMContainerEventType.C_NM_STOP_SENT));
+  }
+
+  @Override
+  public void containerStopFailed(ContainerId containerId, String diagnostics) {
+    context.getEventHandler().handle(
+        new AMContainerEventStopFailed(containerId, diagnostics));
+  }
+
+  @Override
+  public void containerCompleted(ContainerId containerId, int exitStatus, String diagnostics,
+                                 TaskAttemptEndReason endReason) {
+    context.getEventHandler().handle(new AMContainerEventCompleted(containerId, exitStatus, diagnostics, TezUtilsInternal
+        .fromTaskAttemptEndReason(
+            endReason)));
+  }
+
+  @Override
+  public Configuration getInitialConfiguration() {
+    return context.getAMConf();
+  }
+
+  @Override
+  public int getNumNodes(String sourceName) {
+    int sourceIndex = context.getTaskScheduerIdentifier(sourceName);
+    int numNodes = context.getNodeTracker().getNumNodes(sourceIndex);
+    return numNodes;
+  }
+
+  @Override
+  public ApplicationAttemptId getApplicationAttemptId() {
+    return context.getApplicationAttemptId();
+  }
+
+  @Override
+  public Object getTaskCommunicatorMetaInfo(String taskCommName) {
+    int taskCommId = context.getTaskCommunicatorIdentifier(taskCommName);
+    return tal.getTaskCommunicator(taskCommId).getMetaInfo();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/tez/blob/af1cc723/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 04e72db..d56fb95 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
@@ -1551,6 +1551,21 @@ public class DAGAppMaster extends AbstractService {
     }
 
     @Override
+    public String getTaskCommunicatorName(int taskCommId) {
+      return taskCommunicators.inverse().get(taskCommId);
+    }
+
+    @Override
+    public String getTaskSchedulerName(int schedulerId) {
+      return taskSchedulers.inverse().get(schedulerId);
+    }
+
+    @Override
+    public String getContainerLauncherName(int launcherId) {
+      return containerLaunchers.inverse().get(launcherId);
+    }
+
+    @Override
     public Map<ApplicationAccessType, String> getApplicationACLs() {
       if (getServiceState() != STATE.STARTED) {
         throw new TezUncheckedException(

http://git-wip-us.apache.org/repos/asf/tez/blob/af1cc723/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 92e38ae..2eec2fb 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
@@ -18,11 +18,9 @@
 
 package org.apache.tez.dag.app;
 
-import java.net.InetSocketAddress;
-
 import org.apache.hadoop.yarn.api.records.ContainerId;
-import org.apache.tez.dag.api.ContainerEndReason;
-import org.apache.tez.dag.api.TaskAttemptEndReason;
+import org.apache.tez.serviceplugins.api.ContainerEndReason;
+import org.apache.tez.serviceplugins.api.TaskAttemptEndReason;
 import org.apache.tez.dag.app.dag.DAG;
 import org.apache.tez.dag.api.TaskCommunicator;
 import org.apache.tez.dag.app.rm.container.AMContainerTask;

http://git-wip-us.apache.org/repos/asf/tez/blob/af1cc723/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 e2d44e2..47b63dd 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
@@ -28,7 +28,7 @@ import java.util.concurrent.ConcurrentMap;
 
 import com.google.common.annotations.VisibleForTesting;
 import org.apache.commons.collections4.ListUtils;
-import org.apache.tez.dag.api.ContainerEndReason;
+import org.apache.tez.serviceplugins.api.ContainerEndReason;
 import org.apache.tez.dag.app.dag.event.TaskAttemptEvent;
 import org.apache.tez.dag.app.dag.event.TaskAttemptEventStatusUpdate;
 import org.apache.tez.runtime.api.events.TaskStatusUpdateEvent;
@@ -43,7 +43,7 @@ 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.serviceplugins.api.TaskAttemptEndReason;
 import org.apache.tez.dag.api.TaskHeartbeatResponse;
 import org.apache.tez.dag.api.TezConstants;
 import org.apache.tez.dag.api.TezException;

http://git-wip-us.apache.org/repos/asf/tez/blob/af1cc723/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 790066f..50e006d 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,10 +17,6 @@ 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;
@@ -30,7 +26,7 @@ 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.serviceplugins.api.TaskAttemptEndReason;
 import org.apache.tez.dag.api.TaskCommunicatorContext;
 import org.apache.tez.dag.api.TaskHeartbeatRequest;
 import org.apache.tez.dag.api.TaskHeartbeatResponse;

http://git-wip-us.apache.org/repos/asf/tez/blob/af1cc723/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 83322f2..0374022 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
@@ -41,8 +41,8 @@ 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.ContainerEndReason;
-import org.apache.tez.dag.api.TaskAttemptEndReason;
+import org.apache.tez.serviceplugins.api.ContainerEndReason;
+import org.apache.tez.serviceplugins.api.TaskAttemptEndReason;
 import org.apache.tez.dag.api.TaskCommunicator;
 import org.apache.tez.dag.api.TaskCommunicatorContext;
 import org.apache.tez.dag.api.TaskHeartbeatRequest;
@@ -180,7 +180,8 @@ public class TezTaskCommunicatorImpl extends TaskCommunicator {
 
   @Override
   public void registerRunningContainer(ContainerId containerId, String host, int port) {
-    ContainerInfo oldInfo = registeredContainers.putIfAbsent(containerId, new ContainerInfo(containerId, host, port));
+    ContainerInfo oldInfo = registeredContainers.putIfAbsent(containerId,
+        new ContainerInfo(containerId, host, port));
     if (oldInfo != null) {
       throw new TezUncheckedException("Multiple registrations for containerId: " + containerId);
     }
@@ -267,6 +268,11 @@ public class TezTaskCommunicatorImpl extends TaskCommunicator {
     // Nothing to do at the moment. Some of the TODOs from TaskAttemptListener apply here.
   }
 
+  @Override
+  public Object getMetaInfo() {
+    return address;
+  }
+
   protected String getTokenIdentifier() {
     return tokenIdentifier;
   }

http://git-wip-us.apache.org/repos/asf/tez/blob/af1cc723/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
deleted file mode 100644
index ea07a1d..0000000
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncher.java
+++ /dev/null
@@ -1,29 +0,0 @@
-/**
-* Licensed to the Apache Software Foundation (ASF) under one
-* or more contributor license agreements.  See the NOTICE file
-* distributed with this work for additional information
-* regarding copyright ownership.  The ASF licenses this file
-* to you under the Apache License, Version 2.0 (the
-* "License"); you may not use this file except in compliance
-* with the License.  You may obtain a copy of the License at
-*
-*     http://www.apache.org/licenses/LICENSE-2.0
-*
-* Unless required by applicable law or agreed to in writing, software
-* distributed under the License is distributed on an "AS IS" BASIS,
-* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-* See the License for the specific language governing permissions and
-* limitations under the License.
-*/
-
-package org.apache.tez.dag.app.launcher;
-
-
-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
-    extends EventHandler<NMCommunicatorEvent> {
-
-}

http://git-wip-us.apache.org/repos/asf/tez/blob/af1cc723/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 a12fb04..fe0178c 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,12 +30,15 @@ import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 
 import org.apache.commons.lang.exception.ExceptionUtils;
+import org.apache.tez.serviceplugins.api.ContainerLaunchRequest;
+import org.apache.tez.serviceplugins.api.ContainerLauncher;
+import org.apache.tez.serviceplugins.api.ContainerLauncherContext;
+import org.apache.tez.serviceplugins.api.ContainerStopRequest;
 import org.apache.tez.dag.api.TezConstants;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
-import org.apache.hadoop.service.AbstractService;
 import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.StartContainersResponse;
@@ -45,57 +48,43 @@ import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
 import org.apache.hadoop.yarn.api.records.Token;
 import org.apache.hadoop.yarn.client.api.impl.ContainerManagementProtocolProxy;
 import org.apache.hadoop.yarn.client.api.impl.ContainerManagementProtocolProxy.ContainerManagementProtocolProxyData;
-import org.apache.hadoop.yarn.util.Clock;
 import org.apache.hadoop.yarn.util.Records;
 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.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.AMContainerEventStopFailed;
-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 com.google.common.util.concurrent.ThreadFactoryBuilder;
 
 
-// TODO XXX: See what part of this lifecycle and state management can be simplified.
+// TODO See what part of this lifecycle and state management can be simplified.
 // Ideally, no state - only sendStart / sendStop.
 
-// TODO XXX: Review this entire code and clean it up.
+// TODO Review this entire code and clean it up.
 
 /**
  * This class is responsible for launching of containers.
  */
-public class ContainerLauncherImpl extends AbstractService implements
-    ContainerLauncher {
+public class ContainerLauncherImpl extends ContainerLauncher {
 
-  // TODO XXX Ensure the same thread is used to launch / stop the same container. Or - ensure event ordering.
+  // TODO Ensure the same thread is used to launch / stop the same container. Or - ensure event ordering.
   static final Logger LOG = LoggerFactory.getLogger(ContainerLauncherImpl.class);
 
-  private ConcurrentHashMap<ContainerId, Container> containers =
-    new ConcurrentHashMap<ContainerId, Container>();
-  private AppContext context;
+  private final ConcurrentHashMap<ContainerId, Container> containers =
+    new ConcurrentHashMap<>();
   protected ThreadPoolExecutor launcherPool;
   protected static final int INITIAL_POOL_SIZE = 10;
-  private int limitOnPoolSize;
+  private final int limitOnPoolSize;
+  private final Configuration conf;
   private Thread eventHandlingThread;
-  protected BlockingQueue<NMCommunicatorEvent> eventQueue =
-      new LinkedBlockingQueue<NMCommunicatorEvent>();
-  private Clock clock;
+  protected BlockingQueue<ContainerOp> eventQueue = new LinkedBlockingQueue<>();
   private ContainerManagementProtocolProxy cmProxy;
   private AtomicBoolean serviceStopped = new AtomicBoolean(false);
 
-  private Container getContainer(NMCommunicatorEvent event) {
-    ContainerId id = event.getContainerId();
+  private Container getContainer(ContainerOp event) {
+    ContainerId id = event.getBaseOperation().getContainerId();
     Container c = containers.get(id);
     if(c == null) {
-      c = new Container(event.getContainerId(),
-          event.getNodeId().toString(), event.getContainerToken());
+      c = new Container(id,
+          event.getBaseOperation().getNodeId().toString(), event.getBaseOperation().getContainerToken());
       Container old = containers.putIfAbsent(id, c);
       if(old != null) {
         c = old;
@@ -111,6 +100,7 @@ public class ContainerLauncherImpl extends AbstractService implements
     }
   }
 
+
   private static enum ContainerState {
     PREP, FAILED, RUNNING, DONE, KILLED_BEFORE_LAUNCH
   }
@@ -135,7 +125,7 @@ public class ContainerLauncherImpl extends AbstractService implements
     }
 
     @SuppressWarnings("unchecked")
-    public synchronized void launch(NMCommunicatorLaunchRequestEvent event) {
+    public synchronized void launch(ContainerLaunchRequest event) {
       LOG.info("Launching Container with Id: " + event.getContainerId());
       if(this.state == ContainerState.KILLED_BEFORE_LAUNCH) {
         state = ContainerState.DONE;
@@ -171,13 +161,7 @@ public class ContainerLauncherImpl extends AbstractService implements
 
         // after launching, send launched event to task attempt to move
         // it from ASSIGNED to RUNNING state
-        context.getEventHandler().handle(
-            new AMContainerEventLaunched(containerID));
-        ContainerLaunchedEvent lEvt = new ContainerLaunchedEvent(
-            containerID, clock.getTime(), context.getApplicationAttemptId());
-        context.getHistoryHandler().handle(new DAGHistoryEvent(
-            null, lEvt));
-
+        getContext().containerLaunched(containerID);
         this.state = ContainerState.RUNNING;
       } catch (Throwable t) {
         String message = "Container launch failed for " + containerID + " : "
@@ -217,16 +201,14 @@ public class ContainerLauncherImpl extends AbstractService implements
 
             // If stopContainer returns without an error, assuming the stop made
             // it over to the NodeManager.
-          context.getEventHandler().handle(
-              new AMContainerEvent(containerID, AMContainerEventType.C_NM_STOP_SENT));
+          getContext().containerStopRequested(containerID);
         } catch (Throwable t) {
 
           // ignore the cleanup failure
           String message = "cleanup failed for container "
             + this.containerID + " : "
             + ExceptionUtils.getStackTrace(t);
-          context.getEventHandler().handle(
-              new AMContainerEventStopFailed(containerID, message));
+          getContext().containerStopFailed(containerID, message);
           LOG.warn(message);
           this.state = ContainerState.DONE;
           return;
@@ -240,15 +222,9 @@ public class ContainerLauncherImpl extends AbstractService implements
     }
   }
 
-  public ContainerLauncherImpl(AppContext context) {
-    super(ContainerLauncherImpl.class.getName());
-    this.context = context;
-    this.clock = context.getClock();
-  }
-
-  @Override
-  public synchronized void serviceInit(Configuration config) {
-    Configuration conf = new Configuration(config);
+  public ContainerLauncherImpl(ContainerLauncherContext containerLauncherContext) {
+    super(ContainerLauncherImpl.class.getName(), containerLauncherContext);
+    this.conf = new Configuration(containerLauncherContext.getInitialConfiguration());
     conf.setInt(
         CommonConfigurationKeysPublic.IPC_CLIENT_CONNECTION_MAXIDLETIME_KEY,
         0);
@@ -262,7 +238,7 @@ public class ContainerLauncherImpl extends AbstractService implements
   public void serviceStart() {
     // pass a copy of config to ContainerManagementProtocolProxy until YARN-3497 is fixed
     cmProxy =
-        new ContainerManagementProtocolProxy(new Configuration(getConfig()));
+        new ContainerManagementProtocolProxy(conf);
 
     ThreadFactory tf = new ThreadFactoryBuilder().setNameFormat(
         "ContainerLauncher #%d").setDaemon(true).build();
@@ -275,7 +251,7 @@ public class ContainerLauncherImpl extends AbstractService implements
     eventHandlingThread = new Thread() {
       @Override
       public void run() {
-        NMCommunicatorEvent event = null;
+        ContainerOp event = null;
         while (!Thread.currentThread().isInterrupted()) {
           try {
             event = eventQueue.take();
@@ -293,9 +269,8 @@ public class ContainerLauncherImpl extends AbstractService implements
 
             // nodes where containers will run at *this* point of time. This is
             // *not* the cluster size and doesn't need to be.
-            int yarnSourceIndex =
-                context.getTaskScheduerIdentifier(TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT);
-            int numNodes = context.getNodeTracker().getNumNodes(yarnSourceIndex);
+            int numNodes = getContext().getNumNodes(
+                TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT);
             int idealPoolSize = Math.min(limitOnPoolSize, numNodes);
 
             if (poolSize < idealPoolSize) {
@@ -347,7 +322,7 @@ public class ContainerLauncherImpl extends AbstractService implements
     }
   }
 
-  protected EventProcessor createEventProcessor(NMCommunicatorEvent event) {
+  protected EventProcessor createEventProcessor(ContainerOp event) {
     return new EventProcessor(event);
   }
 
@@ -361,32 +336,29 @@ public class ContainerLauncherImpl extends AbstractService implements
    * Setup and start the container on remote nodemanager.
    */
   class EventProcessor implements Runnable {
-    private NMCommunicatorEvent event;
+    private ContainerOp event;
 
-    EventProcessor(NMCommunicatorEvent event) {
+    EventProcessor(ContainerOp event) {
       this.event = event;
     }
 
     @Override
     public void run() {
-      LOG.info("Processing the event " + event.toString());
+      LOG.info("Processing operation {}", event.toString());
 
       // Load ContainerManager tokens before creating a connection.
       // TODO: Do it only once per NodeManager.
-      ContainerId containerID = event.getContainerId();
+      ContainerId containerID = event.getBaseOperation().getContainerId();
 
       Container c = getContainer(event);
-      switch(event.getType()) {
-
-      case CONTAINER_LAUNCH_REQUEST:
-        NMCommunicatorLaunchRequestEvent launchEvent
-            = (NMCommunicatorLaunchRequestEvent) event;
-        c.launch(launchEvent);
-        break;
-
-      case CONTAINER_STOP_REQUEST:
-        c.kill();
-        break;
+      switch(event.getOpType()) {
+        case LAUNCH_REQUEST:
+          ContainerLaunchRequest launchRequest = event.getLaunchRequest();
+          c.launch(launchRequest);
+          break;
+        case STOP_REQUEST:
+          c.kill();
+          break;
       }
       removeContainerIfDone(containerID);
     }
@@ -408,13 +380,23 @@ public class ContainerLauncherImpl extends AbstractService implements
   void sendContainerLaunchFailedMsg(ContainerId containerId,
       String message) {
     LOG.error(message);
-    context.getEventHandler().handle(new AMContainerEventLaunchFailed(containerId, message));
+    getContext().containerLaunchFailed(containerId, message);
+  }
+
+
+  @Override
+  public void launchContainer(ContainerLaunchRequest launchRequest) {
+    try {
+      eventQueue.put(new ContainerOp(ContainerOp.OPType.LAUNCH_REQUEST, launchRequest));
+    } catch (InterruptedException e) {
+      throw new TezUncheckedException(e);
+    }
   }
 
   @Override
-  public void handle(NMCommunicatorEvent event) {
+  public void stopContainer(ContainerStopRequest stopRequest) {
     try {
-      eventQueue.put(event);
+      eventQueue.put(new ContainerOp(ContainerOp.OPType.STOP_REQUEST, stopRequest));
     } catch (InterruptedException e) {
       throw new TezUncheckedException(e);
     }

http://git-wip-us.apache.org/repos/asf/tez/blob/af1cc723/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 db145f4..9f741cf 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
@@ -23,12 +23,18 @@ 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.serviceplugins.api.ContainerLaunchRequest;
+import org.apache.tez.serviceplugins.api.ContainerLauncher;
+import org.apache.tez.serviceplugins.api.ContainerLauncherContext;
+import org.apache.tez.serviceplugins.api.ContainerStopRequest;
 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.ContainerLauncherContextImpl;
 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.apache.tez.dag.app.rm.NMCommunicatorLaunchRequestEvent;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -38,11 +44,15 @@ public class ContainerLauncherRouter extends AbstractService
   static final Logger LOG = LoggerFactory.getLogger(ContainerLauncherImpl.class);
 
   private final ContainerLauncher containerLaunchers[];
+  private final ContainerLauncherContext containerLauncherContexts[];
+  private final AppContext appContext;
 
   @VisibleForTesting
-  public ContainerLauncherRouter(ContainerLauncher containerLauncher) {
+  public ContainerLauncherRouter(ContainerLauncher containerLauncher, AppContext context) {
     super(ContainerLauncherRouter.class.getName());
+    this.appContext = context;
     containerLaunchers = new ContainerLauncher[] {containerLauncher};
+    containerLauncherContexts = new ContainerLauncherContext[] {containerLauncher.getContext()};
   }
 
   // Accepting conf to setup final parameters, if required.
@@ -53,6 +63,7 @@ public class ContainerLauncherRouter extends AbstractService
                                  boolean isPureLocalMode) throws UnknownHostException {
     super(ContainerLauncherRouter.class.getName());
 
+    this.appContext = context;
     if (containerLauncherClassIdentifiers == null || containerLauncherClassIdentifiers.length == 0) {
       if (isPureLocalMode) {
         containerLauncherClassIdentifiers =
@@ -62,16 +73,21 @@ public class ContainerLauncherRouter extends AbstractService
             new String[]{TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT};
       }
     }
+    containerLauncherContexts = new ContainerLauncherContext[containerLauncherClassIdentifiers.length];
     containerLaunchers = new ContainerLauncher[containerLauncherClassIdentifiers.length];
 
+
     for (int i = 0; i < containerLauncherClassIdentifiers.length; i++) {
+      ContainerLauncherContext containerLauncherContext = new ContainerLauncherContextImpl(context, taskAttemptListener);
+      containerLauncherContexts[i] = containerLauncherContext;
       containerLaunchers[i] = createContainerLauncher(containerLauncherClassIdentifiers[i], context,
-          taskAttemptListener, workingDirectory, isPureLocalMode, conf);
+          containerLauncherContext, taskAttemptListener, workingDirectory, isPureLocalMode, conf);
     }
   }
 
   private ContainerLauncher createContainerLauncher(String containerLauncherClassIdentifier,
                                                     AppContext context,
+                                                    ContainerLauncherContext containerLauncherContext,
                                                     TaskAttemptListener taskAttemptListener,
                                                     String workingDirectory,
                                                     boolean isPureLocalMode,
@@ -79,12 +95,15 @@ public class ContainerLauncherRouter extends AbstractService
       UnknownHostException {
     if (containerLauncherClassIdentifier.equals(TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT)) {
       LOG.info("Creating DefaultContainerLauncher");
-      return new ContainerLauncherImpl(context);
+      return new ContainerLauncherImpl(containerLauncherContext);
     } else if (containerLauncherClassIdentifier
         .equals(TezConstants.TEZ_AM_SERVICE_PLUGINS_LOCAL_MODE_NAME_DEFAULT)) {
       LOG.info("Creating LocalContainerLauncher");
+      // TODO Post TEZ-2003. LocalContainerLauncher is special cased, since it makes use of
+      // extensive internals which are only available at runtime. Will likely require
+      // some kind of runtime binding of parameters in the payload to work correctly.
       return
-          new LocalContainerLauncher(context, taskAttemptListener, workingDirectory, isPureLocalMode);
+          new LocalContainerLauncher(containerLauncherContext, context, taskAttemptListener, workingDirectory, isPureLocalMode);
     } else {
       LOG.info("Creating container launcher : " + containerLauncherClassIdentifier);
       Class<? extends ContainerLauncher> containerLauncherClazz =
@@ -92,9 +111,9 @@ public class ContainerLauncherRouter extends AbstractService
               containerLauncherClassIdentifier);
       try {
         Constructor<? extends ContainerLauncher> ctor = containerLauncherClazz
-            .getConstructor(AppContext.class, Configuration.class, TaskAttemptListener.class);
+            .getConstructor(ContainerLauncherContext.class);
         ctor.setAccessible(true);
-        return ctor.newInstance(context, conf, taskAttemptListener);
+        return ctor.newInstance(containerLauncherContext);
       } catch (NoSuchMethodException e) {
         throw new TezUncheckedException(e);
       } catch (InvocationTargetException e) {
@@ -141,6 +160,25 @@ public class ContainerLauncherRouter extends AbstractService
 
   @Override
   public void handle(NMCommunicatorEvent event) {
-    containerLaunchers[event.getLauncherId()].handle(event);
+    int launcherId = event.getLauncherId();
+    String schedulerName = appContext.getTaskSchedulerName(event.getSchedulerId());
+    String taskCommName = appContext.getTaskCommunicatorName(event.getTaskCommId());
+    switch (event.getType()) {
+      case CONTAINER_LAUNCH_REQUEST:
+        NMCommunicatorLaunchRequestEvent launchEvent = (NMCommunicatorLaunchRequestEvent) event;
+        ContainerLaunchRequest launchRequest =
+            new ContainerLaunchRequest(launchEvent.getNodeId(), launchEvent.getContainerId(),
+                launchEvent.getContainerToken(), launchEvent.getContainerLaunchContext(),
+                launchEvent.getContainer(), schedulerName,
+                taskCommName);
+        containerLaunchers[launcherId].launchContainer(launchRequest);
+        break;
+      case CONTAINER_STOP_REQUEST:
+        ContainerStopRequest stopRequest =
+            new ContainerStopRequest(event.getNodeId(), event.getContainerId(),
+                event.getContainerToken(), schedulerName, taskCommName);
+        containerLaunchers[launcherId].stopContainer(stopRequest);
+        break;
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/af1cc723/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerOp.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerOp.java b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerOp.java
new file mode 100644
index 0000000..c62de66
--- /dev/null
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerOp.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.dag.app.launcher;
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.tez.serviceplugins.api.ContainerLaunchRequest;
+import org.apache.tez.serviceplugins.api.ContainerLauncherOperationBase;
+import org.apache.tez.serviceplugins.api.ContainerStopRequest;
+
+@InterfaceAudience.Private
+public class ContainerOp {
+  enum OPType {
+    LAUNCH_REQUEST, STOP_REQUEST
+  }
+
+  final ContainerLauncherOperationBase command;
+  final OPType opType;
+
+  public ContainerOp(OPType opType, ContainerLauncherOperationBase command) {
+    this.opType = opType;
+    this.command = command;
+  }
+
+  public OPType getOpType() {
+    return opType;
+  }
+
+  public ContainerLauncherOperationBase getBaseOperation() {
+    return command;
+  }
+
+  public ContainerLaunchRequest getLaunchRequest() {
+    Preconditions.checkState(opType == OPType.LAUNCH_REQUEST);
+    return (ContainerLaunchRequest) command;
+  }
+
+  public ContainerStopRequest getStopRequest() {
+    Preconditions.checkState(opType == OPType.STOP_REQUEST);
+    return (ContainerStopRequest) command;
+  }
+
+  @Override
+  public String toString() {
+    return "ContainerOp{" +
+        "opType=" + opType +
+        ", command=" + command +
+        '}';
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tez/blob/af1cc723/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 fe23409..a1b8e29 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,11 +44,15 @@ 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.serviceplugins.api.ContainerLaunchRequest;
+import org.apache.tez.serviceplugins.api.ContainerLauncher;
+import org.apache.tez.serviceplugins.api.ContainerLauncherContext;
+import org.apache.tez.serviceplugins.api.ContainerStopRequest;
+import org.apache.tez.serviceplugins.api.TaskAttemptEndReason;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.security.Credentials;
-import org.apache.hadoop.service.AbstractService;
 import org.apache.hadoop.yarn.api.ApplicationConstants.Environment;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.util.AuxiliaryServiceHelper;
@@ -60,17 +64,6 @@ 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.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;
-import org.apache.tez.dag.app.rm.container.AMContainerEvent;
-import org.apache.tez.dag.app.rm.container.AMContainerEventCompleted;
-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.dag.records.TaskAttemptTerminationCause;
 import org.apache.tez.runtime.api.ExecutionContext;
 import org.apache.tez.runtime.api.impl.ExecutionContextImpl;
 import org.apache.tez.runtime.library.common.shuffle.ShuffleUtils;
@@ -82,17 +75,17 @@ import org.apache.tez.runtime.task.TezChild;
  * Since all (sub)tasks share the same local directory, they must be executed
  * sequentially in order to avoid creating/deleting the same files/dirs.
  */
-public class LocalContainerLauncher extends AbstractService implements
-  ContainerLauncher {
+public class LocalContainerLauncher extends ContainerLauncher {
 
   private static final Logger LOG = LoggerFactory.getLogger(LocalContainerLauncher.class);
+
   private final AppContext context;
   private final AtomicBoolean serviceStopped = new AtomicBoolean(false);
   private final String workingDirectory;
   private final TaskAttemptListener tal;
   private final Map<String, String> localEnv;
   private final ExecutionContext executionContext;
-  private int numExecutors;
+  private final int numExecutors;
   private final boolean isPureLocalMode;
 
   private final ConcurrentHashMap<ContainerId, RunningTaskCallback>
@@ -102,23 +95,25 @@ public class LocalContainerLauncher extends AbstractService implements
   private final ExecutorService callbackExecutor = Executors.newFixedThreadPool(1,
       new ThreadFactoryBuilder().setDaemon(true).setNameFormat("CallbackExecutor").build());
 
-  private BlockingQueue<NMCommunicatorEvent> eventQueue =
-      new LinkedBlockingQueue<NMCommunicatorEvent>();
+  private BlockingQueue<ContainerOp> eventQueue = new LinkedBlockingQueue<>();
   private Thread eventHandlingThread;
 
 
   private ListeningExecutorService taskExecutorService;
 
 
-
-  public LocalContainerLauncher(AppContext context,
+  public LocalContainerLauncher(ContainerLauncherContext containerLauncherContext,
+                                AppContext context,
                                 TaskAttemptListener taskAttemptListener,
                                 String workingDirectory,
                                 boolean isPureLocalMode) throws UnknownHostException {
-    super(LocalContainerLauncher.class.getName());
+    // TODO Post TEZ-2003. Most of this information is dynamic and only available after the AM
+    // starts up. It's not possible to set these up via a static payload.
+    // Will need some kind of mechanism to dynamically crate payloads / bind to parameters
+    // after the AM starts up.
+    super(LocalContainerLauncher.class.getName(), containerLauncherContext);
     this.context = context;
     this.tal = taskAttemptListener;
-
     this.workingDirectory = workingDirectory;
     this.isPureLocalMode = isPureLocalMode;
     if (isPureLocalMode) {
@@ -133,11 +128,8 @@ public class LocalContainerLauncher extends AbstractService implements
     String host = isPureLocalMode ? InetAddress.getLocalHost().getHostName() :
         System.getenv(Environment.NM_HOST.name());
     executionContext = new ExecutionContextImpl(host);
-  }
 
-  @Override
-  public synchronized void serviceInit(Configuration conf) {
-    numExecutors = conf.getInt(TezConfiguration.TEZ_AM_INLINE_TASK_EXECUTION_MAX_TASKS,
+    numExecutors = getContext().getInitialConfiguration().getInt(TezConfiguration.TEZ_AM_INLINE_TASK_EXECUTION_MAX_TASKS,
         TezConfiguration.TEZ_AM_INLINE_TASK_EXECUTION_MAX_TASKS_DEFAULT);
     Preconditions.checkState(numExecutors >=1, "Must have at least 1 executor");
     ExecutorService rawExecutor = Executors.newFixedThreadPool(numExecutors,
@@ -169,20 +161,22 @@ public class LocalContainerLauncher extends AbstractService implements
     callbackExecutor.shutdownNow();
   }
 
+
+
   // Thread to monitor the queue of incoming NMCommunicator events
   private class TezSubTaskRunner implements Runnable {
     @Override
     public void run() {
       while (!Thread.currentThread().isInterrupted() && !serviceStopped.get()) {
-        NMCommunicatorEvent event;
+        ContainerOp event;
         try {
           event = eventQueue.take();
-          switch (event.getType()) {
-            case CONTAINER_LAUNCH_REQUEST:
-              launch((NMCommunicatorLaunchRequestEvent) event);
+          switch (event.getOpType()) {
+            case LAUNCH_REQUEST:
+              launch(event.getLaunchRequest());
               break;
-            case CONTAINER_STOP_REQUEST:
-              stop((NMCommunicatorStopRequestEvent)event);
+            case STOP_REQUEST:
+              stop(event.getStopRequest());
               break;
           }
         } catch (InterruptedException e) {
@@ -200,7 +194,7 @@ public class LocalContainerLauncher extends AbstractService implements
 
   @SuppressWarnings("unchecked")
   void sendContainerLaunchFailedMsg(ContainerId containerId, String message) {
-    context.getEventHandler().handle(new AMContainerEventLaunchFailed(containerId, message));
+    getContext().containerLaunchFailed(containerId, message);
   }
 
   private void handleLaunchFailed(Throwable t, ContainerId containerId) {
@@ -215,16 +209,17 @@ public class LocalContainerLauncher extends AbstractService implements
   }
 
   //launch tasks
-  private void launch(NMCommunicatorLaunchRequestEvent event) {
+  private void launch(ContainerLaunchRequest event) {
 
     String tokenIdentifier = context.getApplicationID().toString();
     try {
       TezChild tezChild;
       try {
+        int taskCommId = context.getTaskCommunicatorIdentifier(event.getTaskCommunicatorName());
         tezChild =
             createTezChild(context.getAMConf(), event.getContainerId(), tokenIdentifier,
                 context.getApplicationAttemptId().getAttemptId(), context.getLocalDirs(),
-                ((TezTaskCommunicatorImpl)tal.getTaskCommunicator(event.getTaskCommId())).getUmbilical(),
+                ((TezTaskCommunicatorImpl)tal.getTaskCommunicator(taskCommId)).getUmbilical(),
                 TezCommonUtils.parseCredentialsBytes(event.getContainerLaunchContext().getTokens().array()));
       } catch (InterruptedException e) {
         handleLaunchFailed(e, event.getContainerId());
@@ -238,7 +233,7 @@ public class LocalContainerLauncher extends AbstractService implements
       }
       ListenableFuture<TezChild.ContainerExecutionResult> runningTaskFuture =
           taskExecutorService.submit(createSubTask(tezChild, event.getContainerId()));
-      RunningTaskCallback callback = new RunningTaskCallback(context, event.getContainerId());
+      RunningTaskCallback callback = new RunningTaskCallback(event.getContainerId());
       runningContainers.put(event.getContainerId(), callback);
       Futures.addCallback(runningTaskFuture, callback, callbackExecutor);
     } catch (RejectedExecutionException e) {
@@ -246,7 +241,7 @@ public class LocalContainerLauncher extends AbstractService implements
     }
   }
 
-  private void stop(NMCommunicatorStopRequestEvent event) {
+  private void stop(ContainerStopRequest event) {
     // A stop_request will come in when a task completes and reports back or a preemption decision
     // is made. Currently the LocalTaskScheduler does not support preemption. Also preemption
     // will not work in local mode till Tez supports task preemption instead of container preemption.
@@ -263,18 +258,15 @@ public class LocalContainerLauncher extends AbstractService implements
       // This will need to be fixed once interrupting tasks is supported.
     }
     // Send this event to maintain regular control flow. This isn't of much use though.
-    context.getEventHandler().handle(
-        new AMContainerEvent(event.getContainerId(), AMContainerEventType.C_NM_STOP_SENT));
+    getContext().containerStopRequested(event.getContainerId());
   }
 
   private class RunningTaskCallback
       implements FutureCallback<TezChild.ContainerExecutionResult> {
 
-    private final AppContext appContext;
     private final ContainerId containerId;
 
-    RunningTaskCallback(AppContext appContext, ContainerId containerId) {
-      this.appContext = appContext;
+    RunningTaskCallback(ContainerId containerId) {
       this.containerId = containerId;
     }
 
@@ -286,16 +278,16 @@ public class LocalContainerLauncher extends AbstractService implements
           result.getExitStatus() ==
               TezChild.ContainerExecutionResult.ExitStatus.ASKED_TO_DIE) {
         LOG.info("Container: " + containerId + " completed successfully");
-        appContext.getEventHandler().handle(
-            new AMContainerEventCompleted(containerId, result.getExitStatus().getExitCode(),
-                null, TaskAttemptTerminationCause.CONTAINER_EXITED));
+        getContext()
+            .containerCompleted(containerId, result.getExitStatus().getExitCode(), null,
+                TaskAttemptEndReason.CONTAINER_EXITED);
       } else {
         LOG.info("Container: " + containerId + " completed but with errors");
-        appContext.getEventHandler().handle(
-            new AMContainerEventCompleted(containerId, result.getExitStatus().getExitCode(),
-                result.getErrorMessage() == null ?
-                    (result.getThrowable() == null ? null : result.getThrowable().getMessage()) :
-                    result.getErrorMessage(), TaskAttemptTerminationCause.APPLICATION_ERROR));
+        getContext().containerCompleted(
+            containerId, result.getExitStatus().getExitCode(),
+            result.getErrorMessage() == null ?
+                (result.getThrowable() == null ? null : result.getThrowable().getMessage()) :
+                result.getErrorMessage(), TaskAttemptEndReason.APPLICATION_ERROR);
       }
     }
 
@@ -307,16 +299,14 @@ public class LocalContainerLauncher extends AbstractService implements
       if (!(t instanceof CancellationException)) {
         LOG.info("Container: " + containerId + ": Execution Failed: ", t);
         // Inform of failure with exit code 1.
-        appContext.getEventHandler()
-            .handle(new AMContainerEventCompleted(containerId,
-                TezChild.ContainerExecutionResult.ExitStatus.EXECUTION_FAILURE.getExitCode(),
-                t.getMessage(), TaskAttemptTerminationCause.APPLICATION_ERROR));
+        getContext().containerCompleted(containerId,
+            TezChild.ContainerExecutionResult.ExitStatus.EXECUTION_FAILURE.getExitCode(),
+            t.getMessage(), TaskAttemptEndReason.APPLICATION_ERROR);
       } else {
         LOG.info("Ignoring CancellationException - triggered by LocalContainerLauncher");
-        appContext.getEventHandler()
-            .handle(new AMContainerEventCompleted(containerId,
-                TezChild.ContainerExecutionResult.ExitStatus.SUCCESS.getExitCode(),
-                "CancellationException", TaskAttemptTerminationCause.CONTAINER_EXITED));
+        getContext().containerCompleted(containerId,
+            TezChild.ContainerExecutionResult.ExitStatus.SUCCESS.getExitCode(),
+            "CancellationException", TaskAttemptEndReason.COMMUNICATION_ERROR.CONTAINER_EXITED);
       }
     }
   }
@@ -334,12 +324,7 @@ public class LocalContainerLauncher extends AbstractService implements
         // TezTaskRunner needs to be fixed to ensure this.
         Thread.interrupted();
         // Inform about the launch request now that the container has been allocated a thread to execute in.
-        context.getEventHandler().handle(new AMContainerEventLaunched(containerId));
-        ContainerLaunchedEvent lEvt =
-            new ContainerLaunchedEvent(containerId, context.getClock().getTime(),
-                context.getApplicationAttemptId());
-
-        context.getHistoryHandler().handle(new DAGHistoryEvent(context.getCurrentDAGID(), lEvt));
+        getContext().containerLaunched(containerId);
         return tezChild.run();
       }
     };
@@ -368,11 +353,19 @@ public class LocalContainerLauncher extends AbstractService implements
   }
 
 
+  @Override
+  public void launchContainer(ContainerLaunchRequest launchRequest) {
+    try {
+      eventQueue.put(new ContainerOp(ContainerOp.OPType.LAUNCH_REQUEST, launchRequest));
+    } catch (InterruptedException e) {
+      throw new TezUncheckedException(e);
+    }
+  }
 
   @Override
-  public void handle(NMCommunicatorEvent event) {
+  public void stopContainer(ContainerStopRequest stopRequest) {
     try {
-      eventQueue.put(event);
+      eventQueue.put(new ContainerOp(ContainerOp.OPType.STOP_REQUEST, stopRequest));
     } catch (InterruptedException e) {
       throw new TezUncheckedException(e);
     }

http://git-wip-us.apache.org/repos/asf/tez/blob/af1cc723/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 a775948..33763e7 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,7 +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.serviceplugins.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;

http://git-wip-us.apache.org/repos/asf/tez/blob/af1cc723/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 a234e07..ef789c5 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,7 +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.serviceplugins.api.TaskAttemptEndReason;
 import org.apache.tez.dag.api.TezConfiguration;
 import org.apache.tez.dag.api.TezUncheckedException;
 import org.apache.tez.dag.app.AppContext;

http://git-wip-us.apache.org/repos/asf/tez/blob/af1cc723/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 f86894f..dc50c37 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
@@ -29,14 +29,19 @@ public class NMCommunicatorEvent extends AbstractEvent<NMCommunicatorEventType>
   private final NodeId nodeId;
   private final Token containerToken;
   private final int launcherId;
+  private final int schedulerId;
+  private final int taskCommId;
 
   public NMCommunicatorEvent(ContainerId containerId, NodeId nodeId,
-      Token containerToken, NMCommunicatorEventType type, int launcherId) {
+                             Token containerToken, NMCommunicatorEventType type, int launcherId,
+                             int schedulerId, int taskCommId) {
     super(type);
     this.containerId = containerId;
     this.nodeId = nodeId;
     this.containerToken = containerToken;
     this.launcherId = launcherId;
+    this.schedulerId = schedulerId;
+    this.taskCommId = taskCommId;
   }
 
   public ContainerId getContainerId() {
@@ -55,9 +60,18 @@ public class NMCommunicatorEvent extends AbstractEvent<NMCommunicatorEventType>
     return launcherId;
   }
 
+  public int getSchedulerId() {
+    return schedulerId;
+  }
+
+  public int getTaskCommId() {
+    return taskCommId;
+  }
+
   public String toSrting() {
     return super.toString() + " for container " + containerId + ", nodeId: "
-        + nodeId + ", launcherId: " + launcherId;
+        + nodeId + ", launcherId: " + launcherId + ", schedulerId=" + schedulerId +
+        ", taskCommId=" + taskCommId;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/tez/blob/af1cc723/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 a38345c..c57b6be 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
@@ -26,15 +26,14 @@ 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, int launcherId, int taskCommId) {
+      Container container, int launcherId, int schedulerId, int taskCommId) {
     super(container.getId(), container.getNodeId(), container
-        .getContainerToken(), NMCommunicatorEventType.CONTAINER_LAUNCH_REQUEST, launcherId);
+        .getContainerToken(), NMCommunicatorEventType.CONTAINER_LAUNCH_REQUEST,
+        launcherId, schedulerId, taskCommId);
     this.clc = clc;
     this.container = container;
-    this.taskCommId = taskCommId;
   }
 
   public ContainerLaunchContext getContainerLaunchContext() {
@@ -45,10 +44,6 @@ 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/af1cc723/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 c9b5c44..352f450 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, int launcherId) {
+      Token containerToken, int launcherId, int schedulerId, int taskCommId) {
     super(containerId, nodeId, containerToken,
-        NMCommunicatorEventType.CONTAINER_STOP_REQUEST, launcherId);
+        NMCommunicatorEventType.CONTAINER_STOP_REQUEST, launcherId, schedulerId, taskCommId);
   }
 
 }


[14/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/98eeb773
Tree: http://git-wip-us.apache.org/repos/asf/tez/tree/98eeb773
Diff: http://git-wip-us.apache.org/repos/asf/tez/diff/98eeb773

Branch: refs/heads/TEZ-2003
Commit: 98eeb773ec028ca6268ea9f44adccba68f6c2da3
Parents: 0d72009
Author: Siddharth Seth <ss...@apache.org>
Authored: Tue Apr 7 13:22:09 2015 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Fri Aug 14 13:46:43 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/98eeb773/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/98eeb773/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/98eeb773/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 3798b6f..a6994d2 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
@@ -258,6 +258,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/98eeb773/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 bba06fd..a4a707b 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 {
 
@@ -440,7 +442,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;


[49/50] [abbrv] tez git commit: TEZ-2713. Add tests for node handling when there's multiple schedulers. (sseth)

Posted by ss...@apache.org.
TEZ-2713. Add tests for node handling when there's multiple schedulers. (sseth)


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

Branch: refs/heads/TEZ-2003
Commit: 919b3ed316b9f4c93aecd5a30383a4b3971dbfc6
Parents: 815c7cd
Author: Siddharth Seth <ss...@apache.org>
Authored: Wed Aug 12 10:24:33 2015 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Fri Aug 14 13:49:07 2015 -0700

----------------------------------------------------------------------
 .../tez/dag/app/rm/node/AMNodeTracker.java      |   8 +
 .../tez/dag/app/rm/node/TestAMNodeTracker.java  | 275 +++++++++++++++----
 2 files changed, 231 insertions(+), 52 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/919b3ed3/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeTracker.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeTracker.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeTracker.java
index 751276e..1aa8472 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeTracker.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeTracker.java
@@ -116,6 +116,14 @@ public class AMNodeTracker extends AbstractService implements
     return perSourceNodeTrackers.get(schedulerId).get(nodeId);
   }
 
+  /**
+   * Retrieve the number of nodes from this source on which containers may be running
+   *
+   * This number may differ from the total number of nodes available from the source
+   *
+   * @param schedulerId the schedulerId for which the node count is required
+   * @return the number of nodes from the scheduler on which containers have been allocated
+   */
   public int getNumNodes(int schedulerId) {
     return perSourceNodeTrackers.get(schedulerId).getNumNodes();
   }

http://git-wip-us.apache.org/repos/asf/tez/blob/919b3ed3/tez-dag/src/test/java/org/apache/tez/dag/app/rm/node/TestAMNodeTracker.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/node/TestAMNodeTracker.java b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/node/TestAMNodeTracker.java
index 84d2e1f..def80da 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/node/TestAMNodeTracker.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/node/TestAMNodeTracker.java
@@ -19,6 +19,9 @@
 package org.apache.tez.dag.app.rm.node;
 
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 import static org.mockito.Mockito.doReturn;
@@ -123,6 +126,55 @@ public class TestAMNodeTracker {
   }
 
   @Test (timeout = 5000)
+  public void testMultipleSourcesNodeRegistration() {
+    AppContext appContext = mock(AppContext.class);
+    AMNodeTracker amNodeTracker = new AMNodeTracker(eventHandler, appContext);
+    doReturn(amNodeTracker).when(appContext).getNodeTracker();
+
+    amNodeTracker.init(new Configuration(false));
+    amNodeTracker.start();
+
+    NodeId nodeId1 = NodeId.newInstance("source01", 3333);
+    NodeId nodeId2 = NodeId.newInstance("source02", 3333);
+
+    amNodeTracker.nodeSeen(nodeId1, 0);
+    amNodeTracker.nodeSeen(nodeId2, 1);
+
+    assertEquals(1, amNodeTracker.getNumNodes(0));
+    assertEquals(1, amNodeTracker.getNumNodes(1));
+    assertNotNull(amNodeTracker.get(nodeId1, 0));
+    assertNull(amNodeTracker.get(nodeId2, 0));
+    assertNull(amNodeTracker.get(nodeId1, 1));
+    assertNotNull(amNodeTracker.get(nodeId2, 1));
+  }
+
+  @Test (timeout = 5000)
+  public void testMultipleSourcesNodeCountUpdated() {
+    AppContext appContext = mock(AppContext.class);
+    AMNodeTracker amNodeTracker = new AMNodeTracker(eventHandler, appContext);
+    doReturn(amNodeTracker).when(appContext).getNodeTracker();
+
+    amNodeTracker.init(new Configuration(false));
+    amNodeTracker.start();
+
+    NodeId nodeId1 = NodeId.newInstance("source01", 3333);
+    NodeId nodeId2 = NodeId.newInstance("source02", 3333);
+
+    amNodeTracker.nodeSeen(nodeId1, 0);
+    amNodeTracker.nodeSeen(nodeId2, 1);
+    amNodeTracker.handle(new AMNodeEventNodeCountUpdated(10, 0));
+    amNodeTracker.handle(new AMNodeEventNodeCountUpdated(20, 1));
+
+    // NodeCountUpdate does not reflect in getNumNodes.
+    assertEquals(1, amNodeTracker.getNumNodes(0));
+    assertEquals(1, amNodeTracker.getNumNodes(1));
+    assertNotNull(amNodeTracker.get(nodeId1, 0));
+    assertNull(amNodeTracker.get(nodeId2, 0));
+    assertNull(amNodeTracker.get(nodeId1, 1));
+    assertNotNull(amNodeTracker.get(nodeId2, 1));
+  }
+
+  @Test (timeout = 5000)
   public void testSingleNodeNotBlacklisted() {
     AppContext appContext = mock(AppContext.class);
     Configuration conf = new Configuration(false);
@@ -142,32 +194,61 @@ public class TestAMNodeTracker {
     amNodeTracker.init(conf);
     amNodeTracker.start();
 
-    amNodeTracker.handle(new AMNodeEventNodeCountUpdated(1, 0));
-    NodeId nodeId = NodeId.newInstance("host1", 1234);
-    amNodeTracker.nodeSeen(nodeId, 0);
+    _testSingleNodeNotBlacklisted(amNodeTracker, handler, 0);
+  }
 
-    AMNodeImpl node = (AMNodeImpl) amNodeTracker.get(nodeId, 0);
+  @Test (timeout = 5000)
+  public void testSingleNodeNotBlacklistedAlternateScheduler() {
+    AppContext appContext = mock(AppContext.class);
+    Configuration conf = new Configuration(false);
+    conf.setInt(TezConfiguration.TEZ_AM_MAX_TASK_FAILURES_PER_NODE, 2);
+    conf.setBoolean(TezConfiguration.TEZ_AM_NODE_BLACKLISTING_ENABLED, true);
+    conf.setInt(TezConfiguration.TEZ_AM_NODE_BLACKLISTING_IGNORE_THRESHOLD, 33);
 
-    ContainerId cId1 = mock(ContainerId.class);
-    ContainerId cId2 = mock(ContainerId.class);
+    TestEventHandler handler = new TestEventHandler();
+    AMNodeTracker amNodeTracker = new AMNodeTracker(handler, appContext);
+    doReturn(amNodeTracker).when(appContext).getNodeTracker();
+    AMContainerMap amContainerMap = mock(AMContainerMap.class);
+    TaskSchedulerEventHandler taskSchedulerEventHandler =
+        mock(TaskSchedulerEventHandler.class);
+    dispatcher.register(AMNodeEventType.class, amNodeTracker);
+    dispatcher.register(AMContainerEventType.class, amContainerMap);
+    dispatcher.register(AMSchedulerEventType.class, taskSchedulerEventHandler);
+    amNodeTracker.init(conf);
+    amNodeTracker.start();
 
-    amNodeTracker.handle(new AMNodeEventContainerAllocated(nodeId, 0, cId1));
-    amNodeTracker.handle(new AMNodeEventContainerAllocated(nodeId, 0, cId2));
+    _testSingleNodeNotBlacklisted(amNodeTracker, handler, 1);
+  }
 
-    TezTaskAttemptID ta1 = mock(TezTaskAttemptID.class);
-    TezTaskAttemptID ta2 = mock(TezTaskAttemptID.class);
+  @Test (timeout = 5000)
+  public void testSingleNodeNotBlacklistedAlternateScheduler2() {
+    AppContext appContext = mock(AppContext.class);
+    Configuration conf = new Configuration(false);
+    conf.setInt(TezConfiguration.TEZ_AM_MAX_TASK_FAILURES_PER_NODE, 2);
+    conf.setBoolean(TezConfiguration.TEZ_AM_NODE_BLACKLISTING_ENABLED, true);
+    conf.setInt(TezConfiguration.TEZ_AM_NODE_BLACKLISTING_IGNORE_THRESHOLD, 33);
 
-    amNodeTracker.handle(new AMNodeEventTaskAttemptEnded(nodeId, 0, cId1, ta1, true));
-    dispatcher.await();
-    assertEquals(1, node.numFailedTAs);
-    assertEquals(AMNodeState.ACTIVE, node.getState());
+    TestEventHandler handler = new TestEventHandler();
+    AMNodeTracker amNodeTracker = new AMNodeTracker(handler, appContext);
+    doReturn(amNodeTracker).when(appContext).getNodeTracker();
+    AMContainerMap amContainerMap = mock(AMContainerMap.class);
+    TaskSchedulerEventHandler taskSchedulerEventHandler =
+        mock(TaskSchedulerEventHandler.class);
+    dispatcher.register(AMNodeEventType.class, amNodeTracker);
+    dispatcher.register(AMContainerEventType.class, amContainerMap);
+    dispatcher.register(AMSchedulerEventType.class, taskSchedulerEventHandler);
+    amNodeTracker.init(conf);
+    amNodeTracker.start();
 
-    amNodeTracker.handle(new AMNodeEventTaskAttemptEnded(nodeId, 0, cId2, ta2, true));
-    dispatcher.await();
-    assertEquals(2, node.numFailedTAs);
-    assertEquals(1, handler.events.size());
-    assertEquals(AMNodeEventType.N_IGNORE_BLACKLISTING_ENABLED, handler.events.get(0).getType());
-    assertEquals(AMNodeState.FORCED_ACTIVE, node.getState());
+    // Register multiple nodes from a scheduler which isn't being tested.
+    // This should not affect the blacklisting behaviour
+    for (int i = 0 ; i < 10 ; i++) {
+      amNodeTracker.nodeSeen(NodeId.newInstance("fakenode" + i, 3333), 0);
+    }
+
+    _testSingleNodeNotBlacklisted(amNodeTracker, handler, 1);
+    // No impact on blacklisting for the alternate source
+    assertFalse(amNodeTracker.isBlacklistingIgnored(0));
   }
 
   @Test(timeout=10000)
@@ -186,50 +267,142 @@ public class TestAMNodeTracker {
     dispatcher.register(AMSchedulerEventType.class, taskSchedulerEventHandler);
     amNodeTracker.init(conf);
     amNodeTracker.start();
+    try {
+      _testNodeSelfBlacklist(amNodeTracker, handler, 0);
+    } finally {
+      amNodeTracker.stop();
+    }
+  }
 
-    amNodeTracker.handle(new AMNodeEventNodeCountUpdated(4, 0));
+  @Test(timeout=10000)
+  public void testNodeSelfBlacklistAlternateScheduler1() {
+    AppContext appContext = mock(AppContext.class);
+    Configuration conf = new Configuration(false);
+    conf.setInt(TezConfiguration.TEZ_AM_MAX_TASK_FAILURES_PER_NODE, 2);
+    TestEventHandler handler = new TestEventHandler();
+    AMNodeTracker amNodeTracker = new AMNodeTracker(handler, appContext);
+    doReturn(amNodeTracker).when(appContext).getNodeTracker();
+    AMContainerMap amContainerMap = mock(AMContainerMap.class);
+    TaskSchedulerEventHandler taskSchedulerEventHandler =
+        mock(TaskSchedulerEventHandler.class);
+    dispatcher.register(AMNodeEventType.class, amNodeTracker);
+    dispatcher.register(AMContainerEventType.class, amContainerMap);
+    dispatcher.register(AMSchedulerEventType.class, taskSchedulerEventHandler);
+    amNodeTracker.init(conf);
+    amNodeTracker.start();
+    try {
+      _testNodeSelfBlacklist(amNodeTracker, handler, 1);
+    } finally {
+      amNodeTracker.stop();
+    }
+  }
+
+  @Test(timeout=10000)
+  public void testNodeSelfBlacklistAlternateScheduler2() {
+    AppContext appContext = mock(AppContext.class);
+    Configuration conf = new Configuration(false);
+    conf.setInt(TezConfiguration.TEZ_AM_MAX_TASK_FAILURES_PER_NODE, 2);
+    TestEventHandler handler = new TestEventHandler();
+    AMNodeTracker amNodeTracker = new AMNodeTracker(handler, appContext);
+    doReturn(amNodeTracker).when(appContext).getNodeTracker();
+    AMContainerMap amContainerMap = mock(AMContainerMap.class);
+    TaskSchedulerEventHandler taskSchedulerEventHandler =
+        mock(TaskSchedulerEventHandler.class);
+    dispatcher.register(AMNodeEventType.class, amNodeTracker);
+    dispatcher.register(AMContainerEventType.class, amContainerMap);
+    dispatcher.register(AMSchedulerEventType.class, taskSchedulerEventHandler);
+    amNodeTracker.init(conf);
+    amNodeTracker.start();
+    try {
+      // Register multiple nodes from a scheduler which isn't being tested.
+      // This should not affect the blacklisting behaviour
+      for (int i = 0 ; i < 100 ; i++) {
+        amNodeTracker.nodeSeen(NodeId.newInstance("fakenode" + i, 3333), 0);
+      }
+      _testNodeSelfBlacklist(amNodeTracker, handler, 1);
+      assertFalse(amNodeTracker.isBlacklistingIgnored(0));
+    } finally {
+      amNodeTracker.stop();
+    }
+  }
+
+  private void _testSingleNodeNotBlacklisted(AMNodeTracker amNodeTracker,
+                                             TestEventHandler handler, int schedulerId) {
+    amNodeTracker.handle(new AMNodeEventNodeCountUpdated(1, schedulerId));
+    NodeId nodeId = NodeId.newInstance("host1", 1234);
+    amNodeTracker.nodeSeen(nodeId, schedulerId);
+
+    AMNodeImpl node = (AMNodeImpl) amNodeTracker.get(nodeId, schedulerId);
+
+    ContainerId cId1 = mock(ContainerId.class);
+    ContainerId cId2 = mock(ContainerId.class);
+
+    amNodeTracker.handle(new AMNodeEventContainerAllocated(nodeId, schedulerId, cId1));
+    amNodeTracker.handle(new AMNodeEventContainerAllocated(nodeId, schedulerId, cId2));
+
+    TezTaskAttemptID ta1 = mock(TezTaskAttemptID.class);
+    TezTaskAttemptID ta2 = mock(TezTaskAttemptID.class);
+
+    amNodeTracker.handle(new AMNodeEventTaskAttemptEnded(nodeId, schedulerId, cId1, ta1, true));
+    dispatcher.await();
+    assertEquals(1, node.numFailedTAs);
+    assertEquals(AMNodeState.ACTIVE, node.getState());
+
+    amNodeTracker.handle(new AMNodeEventTaskAttemptEnded(nodeId, schedulerId, cId2, ta2, true));
+    dispatcher.await();
+    assertEquals(2, node.numFailedTAs);
+    assertEquals(1, handler.events.size());
+    assertEquals(AMNodeEventType.N_IGNORE_BLACKLISTING_ENABLED, handler.events.get(0).getType());
+    assertEquals(AMNodeState.FORCED_ACTIVE, node.getState());
+    // Blacklisting should be ignored since the node should have been blacklisted, but has not been
+    // as a result of being a single node for the source
+    assertTrue(amNodeTracker.isBlacklistingIgnored(schedulerId));
+  }
+
+  private void _testNodeSelfBlacklist(AMNodeTracker amNodeTracker, TestEventHandler handler, int schedulerId) {
+    amNodeTracker.handle(new AMNodeEventNodeCountUpdated(4, schedulerId));
     NodeId nodeId = NodeId.newInstance("host1", 1234);
     NodeId nodeId2 = NodeId.newInstance("host2", 1234);
     NodeId nodeId3 = NodeId.newInstance("host3", 1234);
     NodeId nodeId4 = NodeId.newInstance("host4", 1234);
-    amNodeTracker.nodeSeen(nodeId, 0);
-    amNodeTracker.nodeSeen(nodeId2, 0);
-    amNodeTracker.nodeSeen(nodeId3, 0);
-    amNodeTracker.nodeSeen(nodeId4, 0);
-    AMNodeImpl node = (AMNodeImpl) amNodeTracker.get(nodeId, 0);
-    
+    amNodeTracker.nodeSeen(nodeId, schedulerId);
+    amNodeTracker.nodeSeen(nodeId2, schedulerId);
+    amNodeTracker.nodeSeen(nodeId3, schedulerId);
+    amNodeTracker.nodeSeen(nodeId4, schedulerId);
+    AMNodeImpl node = (AMNodeImpl) amNodeTracker.get(nodeId, schedulerId);
+
     ContainerId cId1 = mock(ContainerId.class);
     ContainerId cId2 = mock(ContainerId.class);
     ContainerId cId3 = mock(ContainerId.class);
-    
-    amNodeTracker.handle(new AMNodeEventContainerAllocated(nodeId, 0, cId1));
-    amNodeTracker.handle(new AMNodeEventContainerAllocated(nodeId, 0, cId2));
-    amNodeTracker.handle(new AMNodeEventContainerAllocated(nodeId, 0, cId3));
+
+    amNodeTracker.handle(new AMNodeEventContainerAllocated(nodeId, schedulerId, cId1));
+    amNodeTracker.handle(new AMNodeEventContainerAllocated(nodeId, schedulerId, cId2));
+    amNodeTracker.handle(new AMNodeEventContainerAllocated(nodeId, schedulerId, cId3));
     assertEquals(3, node.containers.size());
-    
+
     TezTaskAttemptID ta1 = mock(TezTaskAttemptID.class);
     TezTaskAttemptID ta2 = mock(TezTaskAttemptID.class);
     TezTaskAttemptID ta3 = mock(TezTaskAttemptID.class);
-    
-    amNodeTracker.handle(new AMNodeEventTaskAttemptSucceeded(nodeId, 0, cId1, ta1));
+
+    amNodeTracker.handle(new AMNodeEventTaskAttemptSucceeded(nodeId, schedulerId, cId1, ta1));
     assertEquals(1, node.numSuccessfulTAs);
-    
-    amNodeTracker.handle(new AMNodeEventTaskAttemptEnded(nodeId, 0, cId2, ta2, true));
+
+    amNodeTracker.handle(new AMNodeEventTaskAttemptEnded(nodeId, schedulerId, cId2, ta2, true));
     assertEquals(1, node.numSuccessfulTAs);
     assertEquals(1, node.numFailedTAs);
     assertEquals(AMNodeState.ACTIVE, node.getState());
     // duplicate should not affect anything
-    amNodeTracker.handle(new AMNodeEventTaskAttemptEnded(nodeId, 0, cId2, ta2, true));
+    amNodeTracker.handle(new AMNodeEventTaskAttemptEnded(nodeId, schedulerId, cId2, ta2, true));
     assertEquals(1, node.numSuccessfulTAs);
     assertEquals(1, node.numFailedTAs);
     assertEquals(AMNodeState.ACTIVE, node.getState());
-    
-    amNodeTracker.handle(new AMNodeEventTaskAttemptEnded(nodeId, 0, cId3, ta3, true));
+
+    amNodeTracker.handle(new AMNodeEventTaskAttemptEnded(nodeId, schedulerId, cId3, ta3, true));
     dispatcher.await();
     assertEquals(1, node.numSuccessfulTAs);
     assertEquals(2, node.numFailedTAs);
     assertEquals(AMNodeState.BLACKLISTED, node.getState());
-    
+
     assertEquals(4, handler.events.size());
     assertEquals(AMContainerEventType.C_NODE_FAILED, handler.events.get(0).getType());
     assertEquals(cId1, ((AMContainerEventNodeFailed)handler.events.get(0)).getContainerId());
@@ -246,20 +419,20 @@ public class TestAMNodeTracker {
     ContainerId cId5 = mock(ContainerId.class);
     TezTaskAttemptID ta4 = mock(TezTaskAttemptID.class);
     TezTaskAttemptID ta5 = mock(TezTaskAttemptID.class);
-    AMNodeImpl node2 = (AMNodeImpl) amNodeTracker.get(nodeId2, 0);
-    amNodeTracker.handle(new AMNodeEventContainerAllocated(nodeId2, 0, cId4));
-    amNodeTracker.handle(new AMNodeEventContainerAllocated(nodeId2, 0, cId5));
-    
-    amNodeTracker.handle(new AMNodeEventTaskAttemptEnded(nodeId2, 0, cId4, ta4, true));
+    AMNodeImpl node2 = (AMNodeImpl) amNodeTracker.get(nodeId2, schedulerId);
+    amNodeTracker.handle(new AMNodeEventContainerAllocated(nodeId2, schedulerId, cId4));
+    amNodeTracker.handle(new AMNodeEventContainerAllocated(nodeId2, schedulerId, cId5));
+
+    amNodeTracker.handle(new AMNodeEventTaskAttemptEnded(nodeId2, schedulerId, cId4, ta4, true));
     assertEquals(1, node2.numFailedTAs);
     assertEquals(AMNodeState.ACTIVE, node2.getState());
-    
+
     handler.events.clear();
-    amNodeTracker.handle(new AMNodeEventTaskAttemptEnded(nodeId2, 0, cId5, ta5, true));
+    amNodeTracker.handle(new AMNodeEventTaskAttemptEnded(nodeId2, schedulerId, cId5, ta5, true));
     dispatcher.await();
     assertEquals(2, node2.numFailedTAs);
     assertEquals(AMNodeState.FORCED_ACTIVE, node2.getState());
-    AMNodeImpl node3 = (AMNodeImpl) amNodeTracker.get(nodeId3, 0);
+    AMNodeImpl node3 = (AMNodeImpl) amNodeTracker.get(nodeId3, schedulerId);
     assertEquals(AMNodeState.FORCED_ACTIVE, node3.getState());
     assertEquals(5, handler.events.size());
 
@@ -286,7 +459,7 @@ public class TestAMNodeTracker {
     // Increase the number of nodes. BLACKLISTING should be re-enabled.
     // Node 1 and Node 2 should go into BLACKLISTED state
     handler.events.clear();
-    amNodeTracker.handle(new AMNodeEventNodeCountUpdated(8, 0));
+    amNodeTracker.handle(new AMNodeEventNodeCountUpdated(8, schedulerId));
     dispatcher.await();
     LOG.info(("Completed waiting for dispatcher to process all pending events"));
     assertEquals(AMNodeState.BLACKLISTED, node.getState());
@@ -317,7 +490,7 @@ public class TestAMNodeTracker {
     assertEquals(4, numIgnoreBlacklistingDisabledEvents);
     assertEquals(2, numBlacklistedEvents);
     assertEquals(2, numNodeFailedEvents);
-    
+
     amNodeTracker.stop();
   }
 
@@ -336,6 +509,4 @@ public class TestAMNodeTracker {
     doReturn(healthReportTime).when(nodeReport).getLastHealthReportTime();
     return nodeReport;
   }
-
-  // TODO TEZ-2003. Add tests for multiple sources.
 }


[09/50] [abbrv] tez git commit: TEZ-2433. Fixes after rebase 05/08. (sseth)

Posted by ss...@apache.org.
TEZ-2433. Fixes after rebase 05/08. (sseth)


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

Branch: refs/heads/TEZ-2003
Commit: 2b66886a546c22bb65b68dc1a2f11cb54a4b3c64
Parents: 5e32809
Author: Siddharth Seth <ss...@apache.org>
Authored: Fri May 8 18:43:16 2015 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Fri Aug 14 13:46:43 2015 -0700

----------------------------------------------------------------------
 TEZ-2003-CHANGES.txt                            |  1 +
 .../tez/dag/api/TaskHeartbeatResponse.java      | 10 ++++++--
 .../dag/app/TaskAttemptListenerImpTezDag.java   | 27 ++++++++++----------
 .../tez/dag/app/TezTaskCommunicatorImpl.java    |  9 +++----
 .../app/TestTaskAttemptListenerImplTezDag.java  | 10 +++-----
 .../library/common/shuffle/TestFetcher.java     |  8 ++----
 6 files changed, 31 insertions(+), 34 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/2b66886a/TEZ-2003-CHANGES.txt
----------------------------------------------------------------------
diff --git a/TEZ-2003-CHANGES.txt b/TEZ-2003-CHANGES.txt
index 9b2339f..ad167ab 100644
--- a/TEZ-2003-CHANGES.txt
+++ b/TEZ-2003-CHANGES.txt
@@ -22,5 +22,6 @@ ALL CHANGES:
   TEZ-2388. Send dag identifier as part of the fetcher request string.
   TEZ-2414. LogicalIOProcessorRuntimeTask, RuntimeTask, TezTaskRunner should handle interrupts & carry out necessary cleanups.
   TEZ-2420. TaskRunner returning before executing the task.
+  TEZ-2433. Fixes after rebase 05/08
 
 INCOMPATIBLE CHANGES:

http://git-wip-us.apache.org/repos/asf/tez/blob/2b66886a/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
index c82a743..b826e76 100644
--- 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
@@ -22,11 +22,13 @@ import org.apache.tez.runtime.api.impl.TezEvent;
 public class TaskHeartbeatResponse {
 
   private final boolean shouldDie;
-  private List<TezEvent> events;
+  private final int nextFromEventId;
+  private final List<TezEvent> events;
 
-  public TaskHeartbeatResponse(boolean shouldDie, List<TezEvent> events) {
+  public TaskHeartbeatResponse(boolean shouldDie, List<TezEvent> events, int nextFromEventId) {
     this.shouldDie = shouldDie;
     this.events = events;
+    this.nextFromEventId = nextFromEventId;
   }
 
   public boolean isShouldDie() {
@@ -36,4 +38,8 @@ public class TaskHeartbeatResponse {
   public List<TezEvent> getEvents() {
     return events;
   }
+
+  public int getNextFromEventId() {
+    return nextFromEventId;
+  }
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/2b66886a/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 cbaed99..db78fa9 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
@@ -78,7 +78,7 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
   protected final TaskHeartbeatHandler taskHeartbeatHandler;
   protected final ContainerHeartbeatHandler containerHeartbeatHandler;
 
-  private final TaskHeartbeatResponse RESPONSE_SHOULD_DIE = new TaskHeartbeatResponse(true, null);
+  private final TaskHeartbeatResponse RESPONSE_SHOULD_DIE = new TaskHeartbeatResponse(true, null, 0);
 
   private final ConcurrentMap<TezTaskAttemptID, ContainerId> registeredAttempts =
       new ConcurrentHashMap<TezTaskAttemptID, ContainerId>();
@@ -194,7 +194,7 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
     // So - avoiding synchronization.
 
     pingContainerHeartbeatHandler(containerId);
-    List<TezEvent> outEvents = null;
+    TaskAttemptEventInfo eventInfo = new TaskAttemptEventInfo(0, null);
     TezTaskAttemptID taskAttemptID = request.getTaskAttemptId();
     if (taskAttemptID != null) {
       ContainerId containerIdFromMap = registeredAttempts.get(taskAttemptID);
@@ -216,12 +216,17 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
       }
 
       List<TezEvent> otherEvents = new ArrayList<TezEvent>();
+      // route TASK_STATUS_UPDATE_EVENT directly to TaskAttempt and route other events
+      // (DATA_MOVEMENT_EVENT, TASK_ATTEMPT_COMPLETED_EVENT, TASK_ATTEMPT_FAILED_EVENT)
+      // to VertexImpl to ensure the events ordering
+      //  1. DataMovementEvent is logged as RecoveryEvent before TaskAttemptFinishedEvent
+      //  2. TaskStatusEvent is handled before TaskAttemptFinishedEvent
       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));
+        if (eventType == EventType.TASK_STATUS_UPDATE_EVENT) {
+          TaskAttemptEvent taskAttemptEvent = new TaskAttemptEventStatusUpdate(taskAttemptID,
+              (TaskStatusUpdateEvent) tezEvent.getEvent());
+          context.getEventHandler().handle(taskAttemptEvent);
         } else {
           otherEvents.add(tezEvent);
         }
@@ -232,14 +237,13 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
             new VertexEventRouteEvent(vertexId, Collections.unmodifiableList(otherEvents)));
       }
       taskHeartbeatHandler.pinged(taskAttemptID);
-      outEvents = context
+      eventInfo = context
           .getCurrentDAG()
           .getVertex(taskAttemptID.getTaskID().getVertexID())
-          .getTask(taskAttemptID.getTaskID())
           .getTaskAttemptTezEvents(taskAttemptID, request.getStartIndex(),
               request.getMaxEvents());
     }
-    return new TaskHeartbeatResponse(false, outEvents);
+    return new TaskHeartbeatResponse(false, eventInfo.getEvents(), eventInfo.getNextFromEventId());
   }
   public void taskAlive(TezTaskAttemptID taskAttemptId) {
     taskHeartbeatHandler.pinged(taskAttemptId);
@@ -435,9 +439,4 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
           + ", ContainerId not known for this attempt");
     }
   }
-
-
-  public TaskCommunicator getTaskCommunicator() {
-    return taskCommunicators[0];
-  }
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/2b66886a/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 6200a5b..accde2c 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
@@ -364,13 +364,10 @@ public class TezTaskCommunicatorImpl extends TaskCommunicator {
             request.getMaxEvents());
         tResponse = taskCommunicatorContext.heartbeat(tRequest);
       }
-      TezHeartbeatResponse response;
-      if (tResponse == null) {
-        response = new TezHeartbeatResponse();
-      } else {
-        response = new TezHeartbeatResponse(tResponse.getEvents());
-      }
+      TezHeartbeatResponse response = new TezHeartbeatResponse();
       response.setLastRequestId(requestId);
+      response.setEvents(tResponse.getEvents());
+      response.setNextFromEventId(tResponse.getNextFromEventId());
       containerInfo.lastRequestId = requestId;
       containerInfo.lastResponse = response;
       return response;

http://git-wip-us.apache.org/repos/asf/tez/blob/2b66886a/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 2208220..34b9792 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
@@ -48,6 +48,7 @@ import org.apache.tez.common.ContainerTask;
 import org.apache.tez.common.security.JobTokenSecretManager;
 import org.apache.tez.dag.api.TezConfiguration;
 import org.apache.tez.dag.api.TaskHeartbeatRequest;
+import org.apache.tez.dag.api.TaskHeartbeatResponse;
 import org.apache.tez.dag.api.TezException;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.tez.common.TezTaskUmbilicalProtocol;
@@ -70,8 +71,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.apache.tez.runtime.api.impl.TezHeartbeatResponse;
 import org.junit.Before;
 import org.junit.Test;
 import org.mockito.ArgumentCaptor;
@@ -260,10 +259,9 @@ public class TestTaskAttemptListenerImplTezDag {
   public void testTaskHeartbeatResponse() throws Exception {
     List<TezEvent> events = new ArrayList<TezEvent>();
     List<TezEvent> eventsToSend = new ArrayList<TezEvent>();
-    TezHeartbeatResponse response = generateHeartbeat(events, 0, 1, 2, eventsToSend);
+    TaskHeartbeatResponse response = generateHeartbeat(events, 0, 1, 2, eventsToSend);
     
     assertEquals(2, response.getNextFromEventId());
-    assertEquals(1, response.getLastRequestId());
     assertEquals(eventsToSend, response.getEvents());
   }
 
@@ -320,7 +318,7 @@ public class TestTaskAttemptListenerImplTezDag {
     return succeedToAllocate;
   }
 
-  private TezHeartbeatResponse generateHeartbeat(List<TezEvent> events,
+  private TaskHeartbeatResponse generateHeartbeat(List<TezEvent> events,
       int fromEventId, int maxEvents, int nextFromEventId,
       List<TezEvent> sendEvents) throws IOException, TezException {
     ContainerId containerId = createContainerId(appId, 1);
@@ -335,7 +333,7 @@ public class TestTaskAttemptListenerImplTezDag {
     taskAttemptListener.registerTaskAttempt(amContainerTask, containerId, 0);
 
     TaskHeartbeatRequest request = mock(TaskHeartbeatRequest.class);
-
+    doReturn(containerId.toString()).when(request).getContainerIdentifier();
     doReturn(containerId.toString()).when(request).getContainerIdentifier();
     doReturn(taskAttemptID).when(request).getTaskAttemptId();
     doReturn(events).when(request).getEvents();

http://git-wip-us.apache.org/repos/asf/tez/blob/2b66886a/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 85e3540..08efb3e 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
@@ -31,7 +31,6 @@ import static org.mockito.Mockito.spy;
 import static org.mockito.Mockito.verify;
 
 import java.io.IOException;
-import java.nio.ByteBuffer;
 import java.util.Arrays;
 import java.util.Iterator;
 import java.util.LinkedList;
@@ -43,11 +42,8 @@ import com.google.common.collect.Lists;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.LocalFileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.DataOutputBuffer;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.tez.dag.api.TezConfiguration;
-import org.apache.tez.runtime.api.ExecutionContext;
-import org.apache.tez.runtime.api.InputContext;
 import org.apache.tez.runtime.library.api.TezRuntimeConfiguration;
 import org.apache.tez.runtime.library.common.InputAttemptIdentifier;
 import org.apache.tez.runtime.library.common.InputIdentifier;
@@ -93,7 +89,7 @@ public class TestFetcher {
 
     // when enabled and hostname does not match use http fetch.
     builder = new Fetcher.FetcherBuilder(fetcherCallback, null, null,
-        ApplicationId.newInstance(0, 1), null, "fetcherTest", conf, ENABLE_LOCAL_FETCH, HOST,
+        ApplicationId.newInstance(0, 1), -1, null, "fetcherTest", conf, ENABLE_LOCAL_FETCH, HOST,
         PORT, false);
     builder.assignWork(HOST + "_OTHER", PORT, 0, Arrays.asList(srcAttempts));
     fetcher = spy(builder.build());
@@ -109,7 +105,7 @@ public class TestFetcher {
 
     // when enabled and port does not match use http fetch.
     builder = new Fetcher.FetcherBuilder(fetcherCallback, null, null,
-        ApplicationId.newInstance(0, 1), null, "fetcherTest", conf, ENABLE_LOCAL_FETCH, HOST, PORT, false);
+        ApplicationId.newInstance(0, 1), -1, null, "fetcherTest", conf, ENABLE_LOCAL_FETCH, HOST, PORT, false);
     builder.assignWork(HOST, PORT + 1, 0, Arrays.asList(srcAttempts));
     fetcher = spy(builder.build());
 


[45/50] [abbrv] tez git commit: TEZ-2698. rebase 08/05 (sseth)

Posted by ss...@apache.org.
TEZ-2698. rebase 08/05 (sseth)


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

Branch: refs/heads/TEZ-2003
Commit: 73ce6c578188abd6d9c19b210c1f94e821b7fabe
Parents: 267fe73
Author: Siddharth Seth <ss...@apache.org>
Authored: Thu Aug 6 02:20:32 2015 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Fri Aug 14 13:47:10 2015 -0700

----------------------------------------------------------------------
 TEZ-2003-CHANGES.txt                                           | 1 +
 .../test/java/org/apache/tez/client/TestTezClientUtils.java    | 2 +-
 .../org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java   | 1 +
 .../java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java   | 5 ++++-
 .../src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java | 6 +++---
 .../apache/tez/dag/app/TestTaskAttemptListenerImplTezDag.java  | 4 +---
 .../apache/tez/runtime/library/common/shuffle/TestFetcher.java | 2 +-
 7 files changed, 12 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/73ce6c57/TEZ-2003-CHANGES.txt
----------------------------------------------------------------------
diff --git a/TEZ-2003-CHANGES.txt b/TEZ-2003-CHANGES.txt
index f921739..b133ea3 100644
--- a/TEZ-2003-CHANGES.txt
+++ b/TEZ-2003-CHANGES.txt
@@ -43,5 +43,6 @@ ALL CHANGES:
   TEZ-2657. Add tests for client side changes - specifying plugins, etc.
   TEZ-2626. Fix log lines with DEBUG in messages, consolidate TEZ-2003 TODOs.
   TEZ-2126. Add unit tests for verifying multiple schedulers, launchers, communicators.
+  TEZ-2698. rebase 08/05
 
 INCOMPATIBLE CHANGES:

http://git-wip-us.apache.org/repos/asf/tez/blob/73ce6c57/tez-api/src/test/java/org/apache/tez/client/TestTezClientUtils.java
----------------------------------------------------------------------
diff --git a/tez-api/src/test/java/org/apache/tez/client/TestTezClientUtils.java b/tez-api/src/test/java/org/apache/tez/client/TestTezClientUtils.java
index 8f40bbd..d1033b2 100644
--- a/tez-api/src/test/java/org/apache/tez/client/TestTezClientUtils.java
+++ b/tez-api/src/test/java/org/apache/tez/client/TestTezClientUtils.java
@@ -225,7 +225,7 @@ public class TestTezClientUtils {
         appId, null, "dagname",
         amConf, m,
         credentials, false,
-        new TezApiVersionInfo(), null);
+        new TezApiVersionInfo(), null, null);
     assertEquals(testpriority, appcontext.getPriority().getPriority());
   }
 

http://git-wip-us.apache.org/repos/asf/tez/blob/73ce6c57/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 7b97738..6c1dad9 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
@@ -32,6 +32,7 @@ import org.apache.commons.collections4.ListUtils;
 import org.apache.tez.dag.api.NamedEntityDescriptor;
 import org.apache.tez.dag.api.TezConstants;
 import org.apache.tez.dag.api.UserPayload;
+import org.apache.tez.dag.app.dag.event.TaskAttemptEventType;
 import org.apache.tez.serviceplugins.api.ContainerEndReason;
 import org.apache.tez.dag.app.dag.event.TaskAttemptEvent;
 import org.apache.tez.dag.app.dag.event.TaskAttemptEventStatusUpdate;

http://git-wip-us.apache.org/repos/asf/tez/blob/73ce6c57/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 fb6d5e7..d3f1c44 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
@@ -163,7 +163,10 @@ public class TezTaskCommunicatorImpl extends TaskCommunicator {
       }
 
       server.start();
-      this.address = NetUtils.getConnectAddress(server);
+      InetSocketAddress serverBindAddress = NetUtils.getConnectAddress(server);
+      this.address = NetUtils.createSocketAddrForHost(
+          serverBindAddress.getAddress().getCanonicalHostName(),
+          serverBindAddress.getPort());
       LOG.info("Instantiated TezTaskCommunicator RPC at " + this.address);
     } catch (IOException e) {
       throw new TezUncheckedException(e);

http://git-wip-us.apache.org/repos/asf/tez/blob/73ce6c57/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 2e6e568..b04b461 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
@@ -401,7 +401,7 @@ public class MockDAGAppMaster extends DAGAppMaster {
               List<TezEvent> events = Lists.newArrayListWithCapacity(
                                       cData.taskSpec.getOutputs().size() + 1);
               if (cData.numUpdates == 0 && eventsDelegate != null) {
-                eventsDelegate.getEvents(cData.taskSpec, events, getContext().getClock().getTime());
+                eventsDelegate.getEvents(cData.taskSpec, events, MockDAGAppMaster.this.getContext().getClock().getTime());
               }
               TezCounters counters = null;
               if (countersDelegate != null) {
@@ -416,7 +416,7 @@ 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),
-                  getContext().getClock().getTime()));
+                  MockDAGAppMaster.this.getContext().getClock().getTime()));
 //              TezHeartbeatRequest request = new TezHeartbeatRequest(cData.numUpdates, events,
 //                  cData.cIdStr, cData.taId, cData.nextFromEventId, 50000);
               TaskHeartbeatRequest request =
@@ -431,7 +431,7 @@ public class MockDAGAppMaster extends DAGAppMaster {
               List<TezEvent> events = Collections.singletonList(new TezEvent(
                   new TaskAttemptCompletedEvent(), new EventMetaData(
                       EventProducerConsumerType.SYSTEM, cData.vName, "", cData.taId),
-                  getContext().getClock().getTime()));
+                  MockDAGAppMaster.this.getContext().getClock().getTime()));
               TaskHeartbeatRequest request =
                   new TaskHeartbeatRequest(cData.cIdStr, cData.taId, events, cData.nextFromEventId, cData.nextPreRoutedFromEventId,
                       10000);

http://git-wip-us.apache.org/repos/asf/tez/blob/73ce6c57/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 639c487..4d404b9 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
@@ -89,7 +89,7 @@ import org.junit.Test;
 import org.mockito.ArgumentCaptor;
 
 @SuppressWarnings("unchecked")
-// TODO TEZ-2003 Rename to TestTezTaskCommunicator
+// TODO TEZ-2003 (post) TEZ-2696 Rename to TestTezTaskCommunicator
 public class TestTaskAttemptListenerImplTezDag {
   private ApplicationId appId;
   private ApplicationAttemptId appAttemptId;
@@ -153,8 +153,6 @@ public class TestTaskAttemptListenerImplTezDag {
         Lists.newArrayList(
             new NamedEntityDescriptor(TezConstants.getTezYarnServicePluginName(), null)
                 .setUserPayload(defaultPayload)));
-    TezTaskCommunicatorImpl taskCommunicator = (TezTaskCommunicatorImpl)taskAttemptListener.getTaskCommunicator();
-    TezTaskUmbilicalProtocol tezUmbilical = taskCommunicator.getUmbilical();
 
     taskSpec = mock(TaskSpec.class);
     doReturn(taskAttemptID).when(taskSpec).getTaskAttemptID();

http://git-wip-us.apache.org/repos/asf/tez/blob/73ce6c57/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 08efb3e..42b957b 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
@@ -273,7 +273,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, PORT, false);
+        ApplicationId.newInstance(0, 1), 1, null, "fetcherTest", conf, true, HOST, PORT, false);
     builder.assignWork(HOST, PORT, partition, Arrays.asList(srcAttempts));
     Fetcher fetcher = spy(builder.build());
     fetcher.populateRemainingMap(new LinkedList<InputAttemptIdentifier>(Arrays.asList(srcAttempts)));


[12/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/5e66ee78
Tree: http://git-wip-us.apache.org/repos/asf/tez/tree/5e66ee78
Diff: http://git-wip-us.apache.org/repos/asf/tez/diff/5e66ee78

Branch: refs/heads/TEZ-2003
Commit: 5e66ee78a9d55681a564c4f34306a83a1add4694
Parents: 98eeb77
Author: Siddharth Seth <ss...@apache.org>
Authored: Thu Apr 9 13:33:48 2015 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Fri Aug 14 13:46:43 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/5e66ee78/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/5e66ee78/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/5e66ee78/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/5e66ee78/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 a6994d2..386e4af 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;
 
@@ -34,13 +36,13 @@ import org.apache.tez.dag.app.dag.event.TaskAttemptEvent;
 import org.apache.tez.dag.app.dag.event.TaskAttemptEventStatusUpdate;
 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;
@@ -66,14 +68,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);
@@ -123,7 +123,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
   }
@@ -144,13 +144,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
@@ -158,7 +158,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) {
@@ -170,18 +170,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
@@ -251,30 +239,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,
@@ -287,7 +265,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,
@@ -300,6 +277,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.
@@ -309,7 +291,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/5e66ee78/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/5e66ee78/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 a4a707b..fa2749a 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;
@@ -254,6 +255,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/5e66ee78/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/5e66ee78/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 3d44ba6..ef2df78 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
@@ -702,6 +702,11 @@ public class DAGImpl implements org.apache.tez.dag.app.dag.DAG,
   }
 
   @Override
+  public StateChangeNotifier getStateChangeNotifier() {
+    return entityUpdateTracker;
+  }
+
+  @Override
   public TezCounters getAllCounters() {
 
     readLock.lock();


[05/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/cc090621
Tree: http://git-wip-us.apache.org/repos/asf/tez/tree/cc090621
Diff: http://git-wip-us.apache.org/repos/asf/tez/diff/cc090621

Branch: refs/heads/TEZ-2003
Commit: cc0906214d09400a34fed331c7f92fad3ae922db
Parents: 1155d59
Author: Siddharth Seth <ss...@apache.org>
Authored: Mon Feb 23 21:59:39 2015 -0800
Committer: Siddharth Seth <ss...@apache.org>
Committed: Fri Aug 14 13:46:42 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/cc090621/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/cc090621/docs/pom.xml
----------------------------------------------------------------------
diff --git a/docs/pom.xml b/docs/pom.xml
index f80b5c1..def3273 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/cc090621/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 2ddb113..1ced4e7 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/cc090621/tez-api/pom.xml
----------------------------------------------------------------------
diff --git a/tez-api/pom.xml b/tez-api/pom.xml
index bb09c6d..93947ab 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/cc090621/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/cc090621/tez-dag/pom.xml
----------------------------------------------------------------------
diff --git a/tez-dag/pom.xml b/tez-dag/pom.xml
index 355ca74..00efdd6 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/cc090621/tez-dist/pom.xml
----------------------------------------------------------------------
diff --git a/tez-dist/pom.xml b/tez-dist/pom.xml
index b7cc8b2..89f7338 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/cc090621/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/cc090621/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/cc090621/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/cc090621/tez-plugins/pom.xml
----------------------------------------------------------------------
diff --git a/tez-plugins/pom.xml b/tez-plugins/pom.xml
index f5ec3b4..fe6f110 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/cc090621/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/cc090621/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/cc090621/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/cc090621/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/cc090621/tez-runtime-library/pom.xml
----------------------------------------------------------------------
diff --git a/tez-runtime-library/pom.xml b/tez-runtime-library/pom.xml
index 25f1cc1..c0924d4 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/cc090621/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/cc090621/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>


[27/50] [abbrv] tez git commit: TEZ-2621. rebase 07/14. (sseth)

Posted by ss...@apache.org.
TEZ-2621. rebase 07/14. (sseth)


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

Branch: refs/heads/TEZ-2003
Commit: 2baf6063fdd4e5f119332b82afa9a55a2a0b6967
Parents: 06f6dcf
Author: Siddharth Seth <ss...@apache.org>
Authored: Tue Jul 14 17:26:25 2015 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Fri Aug 14 13:46:44 2015 -0700

----------------------------------------------------------------------
 TEZ-2003-CHANGES.txt                            |  1 +
 .../app/TestTaskAttemptListenerImplTezDag.java  | 25 +++++++++++++++++---
 .../tez/dag/app/dag/impl/TestTaskAttempt.java   |  7 ++++--
 3 files changed, 28 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/2baf6063/TEZ-2003-CHANGES.txt
----------------------------------------------------------------------
diff --git a/TEZ-2003-CHANGES.txt b/TEZ-2003-CHANGES.txt
index 1e8abcf..590fe7f 100644
--- a/TEZ-2003-CHANGES.txt
+++ b/TEZ-2003-CHANGES.txt
@@ -32,5 +32,6 @@ ALL CHANGES:
   TEZ-2502. Fix for TezTaskRunner2 not killing tasks properly in all situations.
   TEZ-2508. rebase 06/01
   TEZ-2526. Fix version for tez-history-parser.
+  TEZ-2621. rebase 07/14
 
 INCOMPATIBLE CHANGES:

http://git-wip-us.apache.org/repos/asf/tez/blob/2baf6063/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 68d3baf..7f0362d 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
@@ -35,7 +35,9 @@ import java.util.Map;
 import java.util.Random;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.Text;
 import org.apache.hadoop.security.Credentials;
+import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
@@ -45,7 +47,9 @@ import org.apache.hadoop.yarn.event.Event;
 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.JobTokenIdentifier;
 import org.apache.tez.common.security.JobTokenSecretManager;
+import org.apache.tez.common.security.TokenCache;
 import org.apache.tez.dag.api.TezConfiguration;
 import org.apache.tez.dag.api.ContainerEndReason;
 import org.apache.tez.dag.api.TaskAttemptEndReason;
@@ -284,11 +288,18 @@ public class TestTaskAttemptListenerImplTezDag {
     }
   }
 
+  // TODO TEZ-2003 Move this into TestTezTaskCommunicator. Potentially other tests as well.
   @Test (timeout= 5000)
   public void testPortRange_NotSpecified() {
     Configuration conf = new Configuration();
+    JobTokenIdentifier identifier = new JobTokenIdentifier(new Text(
+        "fakeIdentifier"));
+    Token<JobTokenIdentifier> sessionToken = new Token<JobTokenIdentifier>(identifier,
+        new JobTokenSecretManager());
+    sessionToken.setService(identifier.getJobId());
+    TokenCache.setSessionToken(sessionToken, credentials);
     taskAttemptListener = new TaskAttemptListenerImpTezDag(appContext,
-        mock(TaskHeartbeatHandler.class), mock(ContainerHeartbeatHandler.class), null);
+        mock(TaskHeartbeatHandler.class), mock(ContainerHeartbeatHandler.class), null, null, false);
     // no exception happen, should started properly
     taskAttemptListener.init(conf);
     taskAttemptListener.start();
@@ -298,12 +309,20 @@ public class TestTaskAttemptListenerImplTezDag {
     boolean succeedToAllocate = true;
     try {
       Configuration conf = new Configuration();
+      
+      JobTokenIdentifier identifier = new JobTokenIdentifier(new Text(
+          "fakeIdentifier"));
+      Token<JobTokenIdentifier> sessionToken = new Token<JobTokenIdentifier>(identifier,
+          new JobTokenSecretManager());
+      sessionToken.setService(identifier.getJobId());
+      TokenCache.setSessionToken(sessionToken, credentials);
+
       conf.set(TezConfiguration.TEZ_AM_TASK_AM_PORT_RANGE, port + "-" + port);
       taskAttemptListener = new TaskAttemptListenerImpTezDag(appContext,
-          mock(TaskHeartbeatHandler.class), mock(ContainerHeartbeatHandler.class), null);
+          mock(TaskHeartbeatHandler.class), mock(ContainerHeartbeatHandler.class), null, null, false);
       taskAttemptListener.init(conf);
       taskAttemptListener.start();
-      int resultedPort = taskAttemptListener.getAddress().getPort();
+      int resultedPort = taskAttemptListener.getTaskCommunicator(0).getAddress().getPort();
       assertEquals(port, resultedPort);
     } catch (Exception e) {
       succeedToAllocate = false;

http://git-wip-us.apache.org/repos/asf/tez/blob/2baf6063/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 d6fc46e..2bf1c85 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
@@ -21,6 +21,7 @@ package org.apache.tez.dag.app.dag.impl;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.fail;
+import static org.mockito.Matchers.any;
 import static org.mockito.Mockito.doReturn;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.spy;
@@ -1270,8 +1271,10 @@ public class TestTaskAttempt {
 
     MockEventHandler eventHandler = spy(new MockEventHandler());
     TaskAttemptListener taListener = mock(TaskAttemptListener.class);
-    when(taListener.getAddress()).thenReturn(
+    TaskCommunicator mockTaskComm = mock(TaskCommunicator.class);
+    when(mockTaskComm.getAddress()).thenReturn(
         new InetSocketAddress("localhost", 0));
+    when(taListener.getTaskCommunicator(any(int.class))).thenReturn(mockTaskComm);
 
     Configuration taskConf = new Configuration();
     taskConf.setClass("fs.file.impl", StubbedFS.class, FileSystem.class);
@@ -1292,7 +1295,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();


[47/50] [abbrv] tez git commit: TEZ-2707. Fix comments from reviews - part 2. (sseth)

Posted by ss...@apache.org.
TEZ-2707. Fix comments from reviews - part 2. (sseth)


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

Branch: refs/heads/TEZ-2003
Commit: 815c7cdca1ea5f5fc21484f591672589279d3eda
Parents: fee059b
Author: Siddharth Seth <ss...@apache.org>
Authored: Tue Aug 11 16:52:32 2015 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Fri Aug 14 13:49:07 2015 -0700

----------------------------------------------------------------------
 TEZ-2003-CHANGES.txt                            |  1 +
 .../org/apache/tez/dag/app/DAGAppMaster.java    |  2 +-
 .../dag/app/TaskAttemptListenerImpTezDag.java   |  4 ++
 .../tez/dag/app/dag/impl/TaskAttemptImpl.java   |  6 +-
 .../apache/tez/dag/app/rm/AMSchedulerEvent.java | 10 ++-
 .../rm/AMSchedulerEventDeallocateContainer.java | 10 +--
 .../rm/AMSchedulerEventNodeBlacklistUpdate.java |  8 +--
 .../tez/dag/app/rm/AMSchedulerEventTAEnded.java |  8 +--
 .../app/rm/AMSchedulerEventTALaunchRequest.java |  8 +--
 .../apache/tez/dag/app/rm/node/AMNodeEvent.java | 10 +--
 .../apache/tez/dag/app/rm/node/AMNodeImpl.java  | 14 ++---
 .../tez/dag/app/rm/node/AMNodeTracker.java      | 32 +++++-----
 .../apache/tez/dag/app/MockDAGAppMaster.java    |  2 -
 .../tez/dag/app/dag/impl/TestTaskAttempt.java   | 65 ++++++--------------
 .../tez/dag/app/dag/impl/TestVertexImpl2.java   | 56 ++++++++++++++---
 15 files changed, 114 insertions(+), 122 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/815c7cdc/TEZ-2003-CHANGES.txt
----------------------------------------------------------------------
diff --git a/TEZ-2003-CHANGES.txt b/TEZ-2003-CHANGES.txt
index fd3374e..adb800b 100644
--- a/TEZ-2003-CHANGES.txt
+++ b/TEZ-2003-CHANGES.txt
@@ -46,5 +46,6 @@ ALL CHANGES:
   TEZ-2698. rebase 08/05
   TEZ-2675. Add javadocs for new pluggable components, fix problems reported by jenkins
   TEZ-2678. Fix comments from reviews - part 1.
+  TEZ-2707. Fix comments from reviews - part 2.
 
 INCOMPATIBLE CHANGES:

http://git-wip-us.apache.org/repos/asf/tez/blob/815c7cdc/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 ed4f520..fdc48b3 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
@@ -2416,7 +2416,7 @@ public class DAGAppMaster extends AbstractService {
 
 
   @VisibleForTesting
-  static void parsePlugin(List<NamedEntityDescriptor> resultList,
+  public static void parsePlugin(List<NamedEntityDescriptor> resultList,
       BiMap<String, Integer> pluginMap, List<TezNamedEntityDescriptorProto> namedEntityDescriptorProtos,
       boolean tezYarnEnabled, boolean uberEnabled, UserPayload defaultPayload) {
 

http://git-wip-us.apache.org/repos/asf/tez/blob/815c7cdc/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 2f6e93c..185193f 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
@@ -225,6 +225,7 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
             " events: " + (inEvents != null ? inEvents.size() : -1));
       }
 
+      long currTime = context.getClock().getTime();
       List<TezEvent> otherEvents = new ArrayList<TezEvent>();
       // route TASK_STATUS_UPDATE_EVENT directly to TaskAttempt and route other events
       // (DATA_MOVEMENT_EVENT, TASK_ATTEMPT_COMPLETED_EVENT, TASK_ATTEMPT_FAILED_EVENT)
@@ -232,6 +233,9 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
       //  1. DataMovementEvent is logged as RecoveryEvent before TaskAttemptFinishedEvent
       //  2. TaskStatusEvent is handled before TaskAttemptFinishedEvent
       for (TezEvent tezEvent : ListUtils.emptyIfNull(inEvents)) {
+        // for now, set the event time on the AM when it is received.
+        // this avoids any time disparity between machines.
+        tezEvent.setEventReceivedTime(currTime);
         final EventType eventType = tezEvent.getEventType();
         if (eventType == EventType.TASK_STATUS_UPDATE_EVENT) {
           TaskAttemptEvent taskAttemptEvent = new TaskAttemptEventStatusUpdate(taskAttemptID,

http://git-wip-us.apache.org/repos/asf/tez/blob/815c7cdc/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 c6d8a7e..1c4102d 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
@@ -134,7 +134,6 @@ 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;
@@ -440,6 +439,7 @@ public class TaskAttemptImpl implements TaskAttempt,
     this.vertex = this.task.getVertex();
     this.creationCausalTA = schedulingCausalTA;
     this.creationTime = clock.getTime();
+    this.schedulingCausalTA = schedulingCausalTA;
 
     this.reportedStatus = new TaskAttemptStatus(this.attemptId);
     initTaskAttemptStatus(reportedStatus);
@@ -703,7 +703,7 @@ public class TaskAttemptImpl implements TaskAttempt,
   public long getScheduleTime() {
     readLock.lock();
     try {
-      return scheduleTime;
+      return scheduledTime;
     } finally {
       readLock.unlock();
     }
@@ -1071,7 +1071,7 @@ public class TaskAttemptImpl implements TaskAttempt,
     public TaskAttemptStateInternal transition(TaskAttemptImpl ta, TaskAttemptEvent event) {
       TaskAttemptEventSchedule scheduleEvent = (TaskAttemptEventSchedule) event;
 
-      ta.scheduleTime = ta.clock.getTime();
+      ta.scheduledTime = 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/815c7cdc/tez-dag/src/main/java/org/apache/tez/dag/app/rm/AMSchedulerEvent.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/AMSchedulerEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/AMSchedulerEvent.java
index af0bed0..dd9d951 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/AMSchedulerEvent.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/AMSchedulerEvent.java
@@ -22,8 +22,14 @@ import org.apache.hadoop.yarn.event.AbstractEvent;
 
 public class AMSchedulerEvent extends AbstractEvent<AMSchedulerEventType> {
 
-  // TODO Not a very useful class...
-  public AMSchedulerEvent(AMSchedulerEventType type) {
+  private final int schedulerId;
+
+  public AMSchedulerEvent(AMSchedulerEventType type, int schedulerId) {
     super(type);
+    this.schedulerId = schedulerId;
+  }
+
+  public int getSchedulerId() {
+    return this.schedulerId;
   }
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/815c7cdc/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 5270aa2..d1ca99e 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,20 +23,14 @@ 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, int schedulerId) {
-    super(AMSchedulerEventType.S_CONTAINER_DEALLOCATE);
+    super(AMSchedulerEventType.S_CONTAINER_DEALLOCATE, schedulerId);
     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/815c7cdc/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 679705a..d22c0ec 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,20 +23,14 @@ 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, int schedulerId) {
     super((add ? AMSchedulerEventType.S_NODE_BLACKLISTED
-        : AMSchedulerEventType.S_NODE_UNBLACKLISTED));
+        : AMSchedulerEventType.S_NODE_UNBLACKLISTED), schedulerId);
     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/815c7cdc/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 ccc5465..f7fee3a 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
@@ -30,17 +30,15 @@ public class AMSchedulerEventTAEnded extends AMSchedulerEvent {
   private final TaskAttemptState state;
   private final TaskAttemptEndReason taskAttemptEndReason;
   private final String diagnostics;
-  private final int schedulerId;
 
   public AMSchedulerEventTAEnded(TaskAttempt attempt, ContainerId containerId,
       TaskAttemptState state, TaskAttemptEndReason taskAttemptEndReason, String diagnostics, int schedulerId) {
-    super(AMSchedulerEventType.S_TA_ENDED);
+    super(AMSchedulerEventType.S_TA_ENDED, schedulerId);
     this.attempt = attempt;
     this.containerId = containerId;
     this.state = state;
     this.taskAttemptEndReason = taskAttemptEndReason;
     this.diagnostics = diagnostics;
-    this.schedulerId = schedulerId;
   }
 
   public TezTaskAttemptID getAttemptID() {
@@ -59,10 +57,6 @@ public class AMSchedulerEventTAEnded extends AMSchedulerEvent {
     return this.containerId;
   }
 
-  public int getSchedulerId() {
-    return schedulerId;
-  }
-
   public TaskAttemptEndReason getTaskAttemptEndReason() {
     return taskAttemptEndReason;
   }

http://git-wip-us.apache.org/repos/asf/tez/blob/815c7cdc/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 c59193c..0424c97 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,7 +38,6 @@ 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;
 
@@ -48,7 +47,7 @@ public class AMSchedulerEventTALaunchRequest extends AMSchedulerEvent {
       TaskLocationHint locationHint, int priority,
       ContainerContext containerContext,
       int schedulerId, int launcherId, int taskCommId) {
-    super(AMSchedulerEventType.S_TA_LAUNCH_REQUEST);
+    super(AMSchedulerEventType.S_TA_LAUNCH_REQUEST, schedulerId);
     this.attemptId = attemptId;
     this.capability = capability;
     this.remoteTaskSpec = remoteTaskSpec;
@@ -56,7 +55,6 @@ public class AMSchedulerEventTALaunchRequest extends AMSchedulerEvent {
     this.locationHint = locationHint;
     this.priority = priority;
     this.containerContext = containerContext;
-    this.schedulerId = schedulerId;
     this.launcherId = launcherId;
     this.taskCommId = taskCommId;
   }
@@ -89,10 +87,6 @@ public class AMSchedulerEventTALaunchRequest extends AMSchedulerEvent {
     return this.containerContext;
   }
 
-  public int getSchedulerId() {
-    return schedulerId;
-  }
-
   public int getLauncherId() {
     return launcherId;
   }

http://git-wip-us.apache.org/repos/asf/tez/blob/815c7cdc/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeEvent.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeEvent.java
index 85bc513..1a975b0 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeEvent.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeEvent.java
@@ -24,19 +24,19 @@ import org.apache.hadoop.yarn.event.AbstractEvent;
 public class AMNodeEvent extends AbstractEvent<AMNodeEventType> {
 
   private final NodeId nodeId;
-  private final int sourceId; // Effectively the schedulerId
+  private final int schedulerId;
 
-  public AMNodeEvent(NodeId nodeId, int sourceId, AMNodeEventType type) {
+  public AMNodeEvent(NodeId nodeId, int schedulerId, AMNodeEventType type) {
     super(type);
     this.nodeId = nodeId;
-    this.sourceId = sourceId;
+    this.schedulerId = schedulerId;
   }
 
   public NodeId getNodeId() {
     return this.nodeId;
   }
 
-  public int getSourceId() {
-    return sourceId;
+  public int getSchedulerId() {
+    return schedulerId;
   }
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/815c7cdc/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 88b36cb1f..18d5978 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
@@ -54,7 +54,7 @@ public class AMNodeImpl implements AMNode {
   private final ReadLock readLock;
   private final WriteLock writeLock;
   private final NodeId nodeId;
-  private final int sourceId;
+  private final int schedulerId;
   private final AppContext appContext;
   private final int maxTaskFailuresPerNode;
   private boolean blacklistingEnabled;
@@ -173,14 +173,14 @@ public class AMNodeImpl implements AMNode {
 
 
   @SuppressWarnings("rawtypes")
-  public AMNodeImpl(NodeId nodeId, int sourceId, int maxTaskFailuresPerNode,
+  public AMNodeImpl(NodeId nodeId, int schedulerId, int maxTaskFailuresPerNode,
       EventHandler eventHandler, boolean blacklistingEnabled,
       AppContext appContext) {
     ReentrantReadWriteLock rwLock = new ReentrantReadWriteLock();
     this.readLock = rwLock.readLock();
     this.writeLock = rwLock.writeLock();
     this.nodeId = nodeId;
-    this.sourceId = sourceId;
+    this.schedulerId = schedulerId;
     this.appContext = appContext;
     this.eventHandler = eventHandler;
     this.blacklistingEnabled = blacklistingEnabled;
@@ -249,7 +249,7 @@ public class AMNodeImpl implements AMNode {
 
   /* Blacklist the node with the AMNodeTracker and check if the node should be blacklisted */
   protected boolean registerBadNodeAndShouldBlacklist() {
-    return appContext.getNodeTracker().registerBadNodeAndShouldBlacklist(this, sourceId);
+    return appContext.getNodeTracker().registerBadNodeAndShouldBlacklist(this, schedulerId);
   }
 
   protected void blacklistSelf() {
@@ -259,8 +259,7 @@ public class AMNodeImpl implements AMNode {
     // these containers are not useful anymore
     pastContainers.addAll(containers);
     containers.clear();
-    // TODO TEZ-2124 node tracking per ext source
-    sendEvent(new AMSchedulerEventNodeBlacklistUpdate(getNodeId(), true, 0));
+    sendEvent(new AMSchedulerEventNodeBlacklistUpdate(getNodeId(), true, schedulerId));
   }
 
   @SuppressWarnings("unchecked")
@@ -366,8 +365,7 @@ public class AMNodeImpl implements AMNode {
     public void transition(AMNodeImpl node, AMNodeEvent nEvent) {
       node.ignoreBlacklisting = ignore;
       if (node.getState() == AMNodeState.BLACKLISTED) {
-        // TODO TEZ-2124 node tracking per ext source
-        node.sendEvent(new AMSchedulerEventNodeBlacklistUpdate(node.getNodeId(), false, 0));
+        node.sendEvent(new AMSchedulerEventNodeBlacklistUpdate(node.getNodeId(), false, node.schedulerId));
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/tez/blob/815c7cdc/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeTracker.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeTracker.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeTracker.java
index 32e515b..751276e 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeTracker.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeTracker.java
@@ -82,14 +82,14 @@ public class AMNodeTracker extends AbstractService implements
     }
   }
 
-  public void nodeSeen(NodeId nodeId, int sourceId) {
-    PerSourceNodeTracker nodeTracker = getAndCreateIfNeededPerSourceTracker(sourceId);
+  public void nodeSeen(NodeId nodeId, int schedulerId) {
+    PerSourceNodeTracker nodeTracker = getAndCreateIfNeededPerSourceTracker(schedulerId);
     nodeTracker.nodeSeen(nodeId);
   }
 
 
-  boolean registerBadNodeAndShouldBlacklist(AMNode amNode, int sourceId) {
-    return perSourceNodeTrackers.get(sourceId).registerBadNodeAndShouldBlacklist(amNode);
+  boolean registerBadNodeAndShouldBlacklist(AMNode amNode, int schedulerId) {
+    return perSourceNodeTrackers.get(schedulerId).registerBadNodeAndShouldBlacklist(amNode);
   }
 
   public void handle(AMNodeEvent rEvent) {
@@ -101,42 +101,42 @@ public class AMNodeTracker extends AbstractService implements
       case N_IGNORE_BLACKLISTING_ENABLED:
       case N_IGNORE_BLACKLISTING_DISABLED:
         // All of these will only be seen after a node has been registered.
-        perSourceNodeTrackers.get(rEvent.getSourceId()).handle(rEvent);
+        perSourceNodeTrackers.get(rEvent.getSchedulerId()).handle(rEvent);
         break;
       case N_TURNED_UNHEALTHY:
       case N_TURNED_HEALTHY:
       case N_NODE_COUNT_UPDATED:
         // These events can be seen without a node having been marked as 'seen' before
-        getAndCreateIfNeededPerSourceTracker(rEvent.getSourceId()).handle(rEvent);
+        getAndCreateIfNeededPerSourceTracker(rEvent.getSchedulerId()).handle(rEvent);
         break;
     }
   }
 
-  public AMNode get(NodeId nodeId, int sourceId) {
-    return perSourceNodeTrackers.get(sourceId).get(nodeId);
+  public AMNode get(NodeId nodeId, int schedulerId) {
+    return perSourceNodeTrackers.get(schedulerId).get(nodeId);
   }
 
-  public int getNumNodes(int sourceId) {
-    return perSourceNodeTrackers.get(sourceId).getNumNodes();
+  public int getNumNodes(int schedulerId) {
+    return perSourceNodeTrackers.get(schedulerId).getNumNodes();
   }
 
   @Private
   @VisibleForTesting
-  public boolean isBlacklistingIgnored(int sourceId) {
-    return perSourceNodeTrackers.get(sourceId).isBlacklistingIgnored();
+  public boolean isBlacklistingIgnored(int schedulerId) {
+    return perSourceNodeTrackers.get(schedulerId).isBlacklistingIgnored();
   }
 
   public void dagComplete(DAG dag) {
     // TODO TEZ-2337 Maybe reset failures from previous DAGs
   }
 
-  private PerSourceNodeTracker getAndCreateIfNeededPerSourceTracker(int sourceId) {
-    PerSourceNodeTracker nodeTracker = perSourceNodeTrackers.get(sourceId);
+  private PerSourceNodeTracker getAndCreateIfNeededPerSourceTracker(int schedulerId) {
+    PerSourceNodeTracker nodeTracker = perSourceNodeTrackers.get(schedulerId);
     if (nodeTracker == null) {
       nodeTracker =
-          new PerSourceNodeTracker(sourceId, eventHandler, appContext, maxTaskFailuresPerNode,
+          new PerSourceNodeTracker(schedulerId, eventHandler, appContext, maxTaskFailuresPerNode,
               nodeBlacklistingEnabled, blacklistDisablePercent);
-      PerSourceNodeTracker old = perSourceNodeTrackers.putIfAbsent(sourceId, nodeTracker);
+      PerSourceNodeTracker old = perSourceNodeTrackers.putIfAbsent(schedulerId, nodeTracker);
       nodeTracker = old != null ? old : nodeTracker;
     }
     return nodeTracker;

http://git-wip-us.apache.org/repos/asf/tez/blob/815c7cdc/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 b04b461..fe3e4ef 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
@@ -417,8 +417,6 @@ public class MockDAGAppMaster extends DAGAppMaster {
               events.add(new TezEvent(new TaskStatusUpdateEvent(counters, progress, stats), new EventMetaData(
                   EventProducerConsumerType.SYSTEM, cData.vName, "", cData.taId),
                   MockDAGAppMaster.this.getContext().getClock().getTime()));
-//              TezHeartbeatRequest request = new TezHeartbeatRequest(cData.numUpdates, events,
-//                  cData.cIdStr, cData.taId, cData.nextFromEventId, 50000);
               TaskHeartbeatRequest request =
                   new TaskHeartbeatRequest(cData.cIdStr, cData.taId, events, cData.nextFromEventId, cData.nextPreRoutedFromEventId,
                       50000);

http://git-wip-us.apache.org/repos/asf/tez/blob/815c7cdc/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 947ea93..04bb2df 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
@@ -21,7 +21,6 @@ package org.apache.tez.dag.app.dag.impl;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.fail;
-import static org.mockito.Matchers.any;
 import static org.mockito.Mockito.doReturn;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.spy;
@@ -286,10 +285,7 @@ public class TestTaskAttempt {
     TezTaskAttemptID taskAttemptID = TezTaskAttemptID.getInstance(taskID, 0);
 
     MockEventHandler eventHandler = new MockEventHandler();
-    TaskAttemptListener taListener = mock(TaskAttemptListener.class);
-    TaskCommunicator taskComm = mock(TaskCommunicator.class);
-    doReturn(new InetSocketAddress("localhost", 0)).when(taskComm).getAddress();
-    doReturn(taskComm).when(taListener).getTaskCommunicator(0);
+    TaskAttemptListener taListener = createMockTaskAttemptListener();
 
     Configuration taskConf = new Configuration();
     taskConf.setClass("fs.file.impl", StubbedFS.class, FileSystem.class);
@@ -338,10 +334,7 @@ public class TestTaskAttempt {
     TezTaskID taskID = TezTaskID.getInstance(vertexID, 1);
 
     MockEventHandler eventHandler = spy(new MockEventHandler());
-    TaskAttemptListener taListener = mock(TaskAttemptListener.class);
-    TaskCommunicator taskComm = mock(TaskCommunicator.class);
-    doReturn(new InetSocketAddress("localhost", 0)).when(taskComm).getAddress();
-    doReturn(taskComm).when(taListener).getTaskCommunicator(0);
+    TaskAttemptListener taListener = createMockTaskAttemptListener();
 
     Configuration taskConf = new Configuration();
     taskConf.setClass("fs.file.impl", StubbedFS.class, FileSystem.class);
@@ -441,10 +434,7 @@ public class TestTaskAttempt {
     TezTaskID taskID = TezTaskID.getInstance(vertexID, 1);
 
     MockEventHandler eventHandler = new MockEventHandler();
-    TaskAttemptListener taListener = mock(TaskAttemptListener.class);
-    TaskCommunicator taskComm = mock(TaskCommunicator.class);
-    doReturn(new InetSocketAddress("localhost", 0)).when(taskComm).getAddress();
-    doReturn(taskComm).when(taListener).getTaskCommunicator(0);
+    TaskAttemptListener taListener = createMockTaskAttemptListener();
 
     Configuration taskConf = new Configuration();
     taskConf.setClass("fs.file.impl", StubbedFS.class, FileSystem.class);
@@ -508,10 +498,7 @@ public class TestTaskAttempt {
     TezTaskID taskID = TezTaskID.getInstance(vertexID, 1);
 
     MockEventHandler eventHandler = spy(new MockEventHandler());
-    TaskAttemptListener taListener = mock(TaskAttemptListener.class);
-    TaskCommunicator taskComm = mock(TaskCommunicator.class);
-    doReturn(new InetSocketAddress("localhost", 0)).when(taskComm).getAddress();
-    doReturn(taskComm).when(taListener).getTaskCommunicator(0);
+    TaskAttemptListener taListener = createMockTaskAttemptListener();
 
     Configuration taskConf = new Configuration();
     taskConf.setClass("fs.file.impl", StubbedFS.class, FileSystem.class);
@@ -602,10 +589,7 @@ public class TestTaskAttempt {
     TezTaskID taskID = TezTaskID.getInstance(vertexID, 1);
 
     MockEventHandler eventHandler = spy(new MockEventHandler());
-    TaskAttemptListener taListener = mock(TaskAttemptListener.class);
-    TaskCommunicator taskComm = mock(TaskCommunicator.class);
-    doReturn(new InetSocketAddress("localhost", 0)).when(taskComm).getAddress();
-    doReturn(taskComm).when(taListener).getTaskCommunicator(0);
+    TaskAttemptListener taListener = createMockTaskAttemptListener();
 
     Configuration taskConf = new Configuration();
     taskConf.setClass("fs.file.impl", StubbedFS.class, FileSystem.class);
@@ -735,10 +719,7 @@ public class TestTaskAttempt {
     TezTaskID taskID = TezTaskID.getInstance(vertexID, 1);
 
     MockEventHandler eventHandler = spy(new MockEventHandler());
-    TaskAttemptListener taListener = mock(TaskAttemptListener.class);
-    TaskCommunicator taskComm = mock(TaskCommunicator.class);
-    doReturn(new InetSocketAddress("localhost", 0)).when(taskComm).getAddress();
-    doReturn(taskComm).when(taListener).getTaskCommunicator(0);
+    TaskAttemptListener taListener = createMockTaskAttemptListener();
 
     Configuration taskConf = new Configuration();
     taskConf.setClass("fs.file.impl", StubbedFS.class, FileSystem.class);
@@ -829,10 +810,7 @@ public class TestTaskAttempt {
     TezTaskID taskID = TezTaskID.getInstance(vertexID, 1);
 
     MockEventHandler eventHandler = spy(new MockEventHandler());
-    TaskAttemptListener taListener = mock(TaskAttemptListener.class);
-    TaskCommunicator taskComm = mock(TaskCommunicator.class);
-    doReturn(new InetSocketAddress("localhost", 0)).when(taskComm).getAddress();
-    doReturn(taskComm).when(taListener).getTaskCommunicator(0);
+    TaskAttemptListener taListener = createMockTaskAttemptListener();
 
     Configuration taskConf = new Configuration();
     taskConf.setClass("fs.file.impl", StubbedFS.class, FileSystem.class);
@@ -926,10 +904,7 @@ public class TestTaskAttempt {
     TezTaskID taskID = TezTaskID.getInstance(vertexID, 1);
 
     MockEventHandler eventHandler = spy(new MockEventHandler());
-    TaskAttemptListener taListener = mock(TaskAttemptListener.class);
-    TaskCommunicator taskComm = mock(TaskCommunicator.class);
-    doReturn(new InetSocketAddress("localhost", 0)).when(taskComm).getAddress();
-    doReturn(taskComm).when(taListener).getTaskCommunicator(0);
+    TaskAttemptListener taListener = createMockTaskAttemptListener();
 
     Configuration taskConf = new Configuration();
     taskConf.setClass("fs.file.impl", StubbedFS.class, FileSystem.class);
@@ -1031,10 +1006,7 @@ public class TestTaskAttempt {
     TezTaskID taskID = TezTaskID.getInstance(vertexID, 1);
 
     MockEventHandler eventHandler = spy(new MockEventHandler());
-    TaskAttemptListener taListener = mock(TaskAttemptListener.class);
-    TaskCommunicator taskComm = mock(TaskCommunicator.class);
-    doReturn(new InetSocketAddress("localhost", 0)).when(taskComm).getAddress();
-    doReturn(taskComm).when(taListener).getTaskCommunicator(0);
+    TaskAttemptListener taListener = createMockTaskAttemptListener();
 
     Configuration taskConf = new Configuration();
     taskConf.setClass("fs.file.impl", StubbedFS.class, FileSystem.class);
@@ -1133,10 +1105,7 @@ public class TestTaskAttempt {
 
     MockEventHandler mockEh = new MockEventHandler();
     MockEventHandler eventHandler = spy(mockEh);
-    TaskAttemptListener taListener = mock(TaskAttemptListener.class);
-    TaskCommunicator taskComm = mock(TaskCommunicator.class);
-    doReturn(new InetSocketAddress("localhost", 0)).when(taskComm).getAddress();
-    doReturn(taskComm).when(taListener).getTaskCommunicator(0);
+    TaskAttemptListener taListener = createMockTaskAttemptListener();
 
     Configuration taskConf = new Configuration();
     taskConf.setClass("fs.file.impl", StubbedFS.class, FileSystem.class);
@@ -1280,11 +1249,7 @@ public class TestTaskAttempt {
     TezTaskID taskID = TezTaskID.getInstance(vertexID, 1);
 
     MockEventHandler eventHandler = spy(new MockEventHandler());
-    TaskAttemptListener taListener = mock(TaskAttemptListener.class);
-    TaskCommunicator mockTaskComm = mock(TaskCommunicator.class);
-    when(mockTaskComm.getAddress()).thenReturn(
-        new InetSocketAddress("localhost", 0));
-    when(taListener.getTaskCommunicator(any(int.class))).thenReturn(mockTaskComm);
+    TaskAttemptListener taListener = createMockTaskAttemptListener();
 
     Configuration taskConf = new Configuration();
     taskConf.setClass("fs.file.impl", StubbedFS.class, FileSystem.class);
@@ -1412,4 +1377,12 @@ public class TestTaskAttempt {
     return new ContainerContext(new HashMap<String, LocalResource>(),
         new Credentials(), new HashMap<String, String>(), "");
   }
+
+  private TaskAttemptListener createMockTaskAttemptListener() {
+    TaskAttemptListener taListener = mock(TaskAttemptListener.class);
+    TaskCommunicator taskComm = mock(TaskCommunicator.class);
+    doReturn(new InetSocketAddress("localhost", 0)).when(taskComm).getAddress();
+    doReturn(taskComm).when(taListener).getTaskCommunicator(0);
+    return taListener;
+  }
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/815c7cdc/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexImpl2.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexImpl2.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexImpl2.java
index 352ad87..0e34f68 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexImpl2.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexImpl2.java
@@ -24,36 +24,42 @@ import static org.junit.Assert.assertTrue;
 import static org.mockito.Mockito.doReturn;
 import static org.mockito.Mockito.mock;
 
+import java.io.IOException;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
 
 import com.google.common.collect.BiMap;
 import com.google.common.collect.HashBiMap;
+import com.google.common.collect.Lists;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.yarn.api.ApplicationConstants;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.util.Clock;
+import org.apache.tez.common.TezUtils;
 import org.apache.tez.dag.api.DagTypeConverters;
+import org.apache.tez.dag.api.NamedEntityDescriptor;
 import org.apache.tez.dag.api.TaskLocationHint;
 import org.apache.tez.dag.api.TezConfiguration;
 import org.apache.tez.dag.api.TezConstants;
-import org.apache.tez.dag.api.Vertex;
+import org.apache.tez.dag.api.TezUncheckedException;
+import org.apache.tez.dag.api.UserPayload;
 import org.apache.tez.dag.api.Vertex.VertexExecutionContext;
 import org.apache.tez.dag.api.VertexLocationHint;
 import org.apache.tez.dag.api.records.DAGProtos;
+import org.apache.tez.dag.api.records.DAGProtos.TezNamedEntityDescriptorProto;
 import org.apache.tez.dag.api.records.DAGProtos.VertexPlan;
 import org.apache.tez.dag.app.AppContext;
 import org.apache.tez.dag.app.ContainerContext;
+import org.apache.tez.dag.app.DAGAppMaster;
 import org.apache.tez.dag.app.TaskAttemptListener;
 import org.apache.tez.dag.app.TaskHeartbeatHandler;
 import org.apache.tez.dag.app.dag.DAG;
 import org.apache.tez.dag.app.dag.StateChangeNotifier;
 import org.apache.tez.dag.records.TezVertexID;
 import org.apache.tez.dag.utils.TaskSpecificLaunchCmdOption;
-import org.apache.tez.runtime.api.ExecutionContext;
 import org.junit.Test;
 
 /**
@@ -363,16 +369,46 @@ public class TestVertexImpl2 {
       this.vertexName = "testvertex";
       this.vertexExecutionContext = vertexExecutionContext;
       this.defaultExecutionContext = defaultDagExecitionContext;
-      if (numPlugins == 0) {
-        this.taskSchedulers.put(TezConstants.getTezYarnServicePluginName(), 0);
-        this.containerLaunchers.put(TezConstants.getTezYarnServicePluginName(), 0);
-        this.taskSchedulers.put(TezConstants.getTezYarnServicePluginName(), 0);
-      } else {
+      if (numPlugins == 0) { // Add default container plugins only
+        UserPayload defaultPayload;
+        try {
+          defaultPayload = TezUtils.createUserPayloadFromConf(new Configuration(false));
+        } catch (IOException e) {
+          throw new TezUncheckedException(e);
+        }
+        DAGAppMaster.parsePlugin(Lists.<NamedEntityDescriptor>newLinkedList(), taskSchedulers, null,
+            true, false, defaultPayload);
+        DAGAppMaster
+            .parsePlugin(Lists.<NamedEntityDescriptor>newLinkedList(), containerLaunchers, null,
+                true, false, defaultPayload);
+        DAGAppMaster.parsePlugin(Lists.<NamedEntityDescriptor>newLinkedList(), taskComms, null,
+            true, false, defaultPayload);
+      } else { // Add N plugins, no YARN defaults
+        List<TezNamedEntityDescriptorProto> schedulerList = new LinkedList<>();
+        List<TezNamedEntityDescriptorProto> launcherList = new LinkedList<>();
+        List<TezNamedEntityDescriptorProto> taskCommList = new LinkedList<>();
         for (int i = 0; i < numPlugins; i++) {
-          this.taskSchedulers.put(append(TASK_SCHEDULER_NAME_BASE, i), i);
-          this.containerLaunchers.put(append(CONTAINER_LAUNCHER_NAME_BASE, i), i);
-          this.taskComms.put(append(TASK_COMM_NAME_BASE, i), i);
+          schedulerList.add(TezNamedEntityDescriptorProto.newBuilder()
+              .setName(append(TASK_SCHEDULER_NAME_BASE, i)).setEntityDescriptor(
+                  DAGProtos.TezEntityDescriptorProto.newBuilder()
+                      .setClassName(append(TASK_SCHEDULER_NAME_BASE, i))).build());
+          launcherList.add(TezNamedEntityDescriptorProto.newBuilder()
+              .setName(append(CONTAINER_LAUNCHER_NAME_BASE, i)).setEntityDescriptor(
+                  DAGProtos.TezEntityDescriptorProto.newBuilder()
+                      .setClassName(append(CONTAINER_LAUNCHER_NAME_BASE, i))).build());
+          taskCommList.add(
+              TezNamedEntityDescriptorProto.newBuilder().setName(append(TASK_COMM_NAME_BASE, i))
+                  .setEntityDescriptor(
+                      DAGProtos.TezEntityDescriptorProto.newBuilder()
+                          .setClassName(append(TASK_COMM_NAME_BASE, i))).build());
         }
+        DAGAppMaster.parsePlugin(Lists.<NamedEntityDescriptor>newLinkedList(), taskSchedulers,
+            schedulerList, false, false, null);
+        DAGAppMaster.parsePlugin(Lists.<NamedEntityDescriptor>newLinkedList(), containerLaunchers,
+            launcherList, false, false, null);
+        DAGAppMaster
+            .parsePlugin(Lists.<NamedEntityDescriptor>newLinkedList(), taskComms, taskCommList,
+                false, false, null);
       }
 
       this.appContext = createDefaultMockAppContext();


[17/50] [abbrv] tez git commit: TEZ-2124. Change Node tracking to work per external container source. (sseth)

Posted by ss...@apache.org.
TEZ-2124. Change Node tracking to work per external container source. (sseth)


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

Branch: refs/heads/TEZ-2003
Commit: a5dfca2bb28b6921a95836630e2a2d02cb9318c2
Parents: 2baf606
Author: Siddharth Seth <ss...@apache.org>
Authored: Thu Jul 16 14:18:35 2015 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Fri Aug 14 13:46:44 2015 -0700

----------------------------------------------------------------------
 TEZ-2003-CHANGES.txt                            |   1 +
 .../apache/tez/dag/app/dag/impl/DAGImpl.java    |   2 +
 .../apache/tez/dag/app/dag/impl/TaskImpl.java   |   3 +-
 .../dag/app/launcher/ContainerLauncherImpl.java |   5 +-
 .../app/rm/TaskSchedulerAppCallbackImpl.java    |  89 +++++++++
 .../dag/app/rm/TaskSchedulerEventHandler.java   |  71 +++----
 .../apache/tez/dag/app/rm/node/AMNodeEvent.java |   8 +-
 .../rm/node/AMNodeEventContainerAllocated.java  |   4 +-
 .../rm/node/AMNodeEventNodeCountUpdated.java    |   4 +-
 .../app/rm/node/AMNodeEventStateChanged.java    |   4 +-
 .../rm/node/AMNodeEventTaskAttemptEnded.java    |   4 +-
 .../node/AMNodeEventTaskAttemptSucceeded.java   |   4 +-
 .../apache/tez/dag/app/rm/node/AMNodeImpl.java  |   6 +-
 .../tez/dag/app/rm/node/AMNodeTracker.java      | 162 +++++-----------
 .../dag/app/rm/node/PerSourceNodeTracker.java   | 187 +++++++++++++++++++
 .../apache/tez/dag/app/MockDAGAppMaster.java    |   2 +-
 .../tez/dag/app/TestMockDAGAppMaster.java       |   2 +-
 .../tez/dag/app/rm/TestContainerReuse.java      |  62 +++---
 .../app/rm/TestTaskSchedulerEventHandler.java   |  11 +-
 .../dag/app/rm/TestTaskSchedulerHelpers.java    |   2 +-
 .../tez/dag/app/rm/node/TestAMNodeTracker.java  |  64 ++++---
 21 files changed, 462 insertions(+), 235 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/a5dfca2b/TEZ-2003-CHANGES.txt
----------------------------------------------------------------------
diff --git a/TEZ-2003-CHANGES.txt b/TEZ-2003-CHANGES.txt
index 590fe7f..604947c 100644
--- a/TEZ-2003-CHANGES.txt
+++ b/TEZ-2003-CHANGES.txt
@@ -33,5 +33,6 @@ ALL CHANGES:
   TEZ-2508. rebase 06/01
   TEZ-2526. Fix version for tez-history-parser.
   TEZ-2621. rebase 07/14
+  TEZ-2124. Change Node tracking to work per external container source.
 
 INCOMPATIBLE CHANGES:

http://git-wip-us.apache.org/repos/asf/tez/blob/a5dfca2b/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 e37fc2f..ec2ef66 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
@@ -1440,9 +1440,11 @@ public class DAGImpl implements org.apache.tez.dag.app.dag.DAG,
       VertexImpl v = createVertex(this, vertexName, i);
       addVertex(v);
     }
+
     // check task resources, only check it in non-local mode
     if (!appContext.isLocal()) {
       for (Vertex v : vertexMap.values()) {
+        // TODO TEZ-2003 (post) Ideally, this should be per source.
         if (v.getTaskResource().compareTo(appContext.getClusterInfo().getMaxContainerCapability()) > 0) {
           String msg = "Vertex's TaskResource is beyond the cluster container capability," +
               "Vertex=" + v.getLogIdentifier() +", Requested TaskResource=" + v.getTaskResource()

http://git-wip-us.apache.org/repos/asf/tez/blob/a5dfca2b/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 93b4c3f..1b55295 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
@@ -1396,7 +1396,8 @@ public class TaskImpl implements Task, EventHandler<TaskEvent> {
         if (amContainer != null) {
           // inform the node about failure
           task.eventHandler.handle(
-              new AMNodeEventTaskAttemptEnded(amContainer.getContainer().getNodeId(), 
+              new AMNodeEventTaskAttemptEnded(amContainer.getContainer().getNodeId(),
+                  task.getVertex().getTaskSchedulerIdentifier(),
                   containerId, failedAttemptId, true));
         }
       }

http://git-wip-us.apache.org/repos/asf/tez/blob/a5dfca2b/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 a1eb2a7..a12fb04 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,6 +30,7 @@ import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 
 import org.apache.commons.lang.exception.ExceptionUtils;
+import org.apache.tez.dag.api.TezConstants;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -292,7 +293,9 @@ public class ContainerLauncherImpl extends AbstractService implements
 
             // nodes where containers will run at *this* point of time. This is
             // *not* the cluster size and doesn't need to be.
-            int numNodes = context.getNodeTracker().getNumNodes();
+            int yarnSourceIndex =
+                context.getTaskScheduerIdentifier(TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT);
+            int numNodes = context.getNodeTracker().getNumNodes(yarnSourceIndex);
             int idealPoolSize = Math.min(limitOnPoolSize, numNodes);
 
             if (poolSize < idealPoolSize) {

http://git-wip-us.apache.org/repos/asf/tez/blob/a5dfca2b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerAppCallbackImpl.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerAppCallbackImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerAppCallbackImpl.java
new file mode 100644
index 0000000..ea37e94
--- /dev/null
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerAppCallbackImpl.java
@@ -0,0 +1,89 @@
+/*
+ * 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.nio.ByteBuffer;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
+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.NodeReport;
+import org.apache.hadoop.yarn.api.records.Resource;
+
+public class TaskSchedulerAppCallbackImpl implements TaskSchedulerService.TaskSchedulerAppCallback{
+
+  private final TaskSchedulerEventHandler tseh;
+  private final int schedulerId;
+
+  public TaskSchedulerAppCallbackImpl(TaskSchedulerEventHandler tseh, int schedulerId) {
+    this.tseh = tseh;
+    this.schedulerId = schedulerId;
+  }
+
+  @Override
+  public void taskAllocated(Object task, Object appCookie, Container container) {
+    tseh.taskAllocated(schedulerId, task, appCookie, container);
+  }
+
+  @Override
+  public void containerCompleted(Object taskLastAllocated, ContainerStatus containerStatus) {
+    tseh.containerCompleted(schedulerId, taskLastAllocated, containerStatus);
+  }
+
+  @Override
+  public void containerBeingReleased(ContainerId containerId) {
+    tseh.containerBeingReleased(schedulerId, containerId);
+  }
+
+  @Override
+  public void nodesUpdated(List<NodeReport> updatedNodes) {
+    tseh.nodesUpdated(schedulerId, updatedNodes);
+  }
+
+  @Override
+  public void appShutdownRequested() {
+    tseh.appShutdownRequested(schedulerId);
+  }
+
+  @Override
+  public void setApplicationRegistrationData(Resource maxContainerCapability,
+                                             Map<ApplicationAccessType, String> appAcls,
+                                             ByteBuffer clientAMSecretKey) {
+    tseh.setApplicationRegistrationData(schedulerId, maxContainerCapability, appAcls, clientAMSecretKey);
+  }
+
+  @Override
+  public void onError(Throwable t) {
+    tseh.onError(schedulerId, t);
+  }
+
+  @Override
+  public float getProgress() {
+    return tseh.getProgress(schedulerId);
+  }
+
+  @Override
+  public void preemptContainer(ContainerId containerId) {
+    tseh.preemptContainer(schedulerId, containerId);
+  }
+
+  @Override
+  public AppFinalStatus getFinalAppStatus() {
+    return tseh.getFinalAppStatus();
+  }
+}

http://git-wip-us.apache.org/repos/asf/tez/blob/a5dfca2b/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 69763b3..1ad0059 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
@@ -29,6 +29,7 @@ import java.util.concurrent.LinkedBlockingQueue;
 import java.util.concurrent.atomic.AtomicBoolean;
 
 import com.google.common.annotations.VisibleForTesting;
+import org.apache.tez.dag.app.rm.TaskSchedulerService.TaskSchedulerAppCallback.AppFinalStatus;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -81,8 +82,7 @@ import org.apache.tez.dag.records.TaskAttemptTerminationCause;
 import com.google.common.base.Preconditions;
 
 
-public class TaskSchedulerEventHandler extends AbstractService
-                                         implements TaskSchedulerAppCallback,
+public class TaskSchedulerEventHandler extends AbstractService implements
                                                EventHandler<AMSchedulerEvent> {
   static final Logger LOG = LoggerFactory.getLogger(TaskSchedulerEventHandler.class);
 
@@ -315,7 +315,7 @@ public class TaskSchedulerEventHandler extends AbstractService
       // stopped.
       // AMNodeImpl blacklisting logic does not account for KILLED attempts.
       sendEvent(new AMNodeEventTaskAttemptEnded(appContext.getAllContainers().
-          get(attemptContainerId).getContainer().getNodeId(), attemptContainerId,
+          get(attemptContainerId).getContainer().getNodeId(), event.getSchedulerId(), attemptContainerId,
           attempt.getID(), event.getState() == TaskAttemptState.FAILED));
     }
   }
@@ -330,7 +330,7 @@ public class TaskSchedulerEventHandler extends AbstractService
       sendEvent(new AMContainerEventTASucceeded(usedContainerId,
           event.getAttemptID()));
       sendEvent(new AMNodeEventTaskAttemptSucceeded(appContext.getAllContainers().
-          get(usedContainerId).getContainer().getNodeId(), usedContainerId,
+          get(usedContainerId).getContainer().getNodeId(), event.getSchedulerId(), usedContainerId,
           event.getAttemptID()));
     }
 
@@ -392,14 +392,16 @@ public class TaskSchedulerEventHandler extends AbstractService
   private TaskSchedulerService createTaskScheduler(String host, int port, String trackingUrl,
                                                    AppContext appContext,
                                                    String schedulerClassName,
-                                                   long customAppIdIdentifier) {
+                                                   long customAppIdIdentifier,
+                                                   int schedulerId) {
+    TaskSchedulerAppCallback appCallback = new TaskSchedulerAppCallbackImpl(this, schedulerId);
     if (schedulerClassName.equals(TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT)) {
       LOG.info("Creating TaskScheduler: YarnTaskSchedulerService");
-      return new YarnTaskSchedulerService(this, this.containerSignatureMatcher,
+      return new YarnTaskSchedulerService(appCallback, 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,
+      return new LocalTaskSchedulerService(appCallback, this.containerSignatureMatcher,
           host, port, trackingUrl, customAppIdIdentifier, appContext);
     } else {
       LOG.info("Creating custom TaskScheduler: " + schedulerClassName);
@@ -411,7 +413,7 @@ public class TaskSchedulerEventHandler extends AbstractService
             .getConstructor(TaskSchedulerAppCallback.class, AppContext.class, String.class,
                 int.class, String.class, long.class, Configuration.class);
         ctor.setAccessible(true);
-        return ctor.newInstance(this, appContext, host, port, trackingUrl, customAppIdIdentifier,
+        return ctor.newInstance(appCallback, appContext, host, port, trackingUrl, customAppIdIdentifier,
             getConfig());
       } catch (NoSuchMethodException e) {
         throw new TezUncheckedException(e);
@@ -441,7 +443,7 @@ public class TaskSchedulerEventHandler extends AbstractService
       LOG.info("ClusterIdentifier for TaskScheduler [" + i + ":" + taskSchedulerClasses[i] + "]=" +
           customAppIdIdentifier);
       taskSchedulers[i] = createTaskScheduler(host, port,
-          trackingUrl, appContext, taskSchedulerClasses[i], customAppIdIdentifier);
+          trackingUrl, appContext, taskSchedulerClasses[i], customAppIdIdentifier, i);
     }
   }
 
@@ -521,20 +523,21 @@ public class TaskSchedulerEventHandler extends AbstractService
     }
   }
 
-  // TaskSchedulerAppCallback methods
-  @Override
-  public synchronized void taskAllocated(Object task,
+  // TODO TEZ-2003 Consolidate TaskSchedulerAppCallback methods once these methods are moved into context
+
+  // TaskSchedulerAppCallback methods with schedulerId, where relevant
+  public synchronized void taskAllocated(int schedulerId, Object task,
                                            Object appCookie,
                                            Container container) {
     AMSchedulerEventTALaunchRequest event =
         (AMSchedulerEventTALaunchRequest) appCookie;
     ContainerId containerId = container.getId();
     if (appContext.getAllContainers()
-        .addContainerIfNew(container, event.getSchedulerId(), event.getLauncherId(),
+        .addContainerIfNew(container, schedulerId, event.getLauncherId(),
             event.getTaskCommId())) {
-      appContext.getNodeTracker().nodeSeen(container.getNodeId());
+      appContext.getNodeTracker().nodeSeen(container.getNodeId(), schedulerId);
       sendEvent(new AMNodeEventContainerAllocated(container
-          .getNodeId(), container.getId()));
+          .getNodeId(), schedulerId, container.getId()));
     }
 
 
@@ -554,8 +557,8 @@ public class TaskSchedulerEventHandler extends AbstractService
             .getContainerContext().getCredentials(), event.getPriority()));
   }
 
-  @Override
-  public synchronized void containerCompleted(Object task, ContainerStatus containerStatus) {
+  public synchronized void containerCompleted(int schedulerId, Object task, ContainerStatus containerStatus) {
+    // SchedulerId isn't used here since no node updates are sent out
     // Inform the Containers about completion.
     AMContainer amContainer = appContext.getAllContainers().get(containerStatus.getContainerId());
     if (amContainer != null) {
@@ -578,8 +581,8 @@ public class TaskSchedulerEventHandler extends AbstractService
     }
   }
 
-  @Override
-  public synchronized void containerBeingReleased(ContainerId containerId) {
+  public synchronized void containerBeingReleased(int schedulerId, ContainerId containerId) {
+    // SchedulerId isn't used here since no node updates are sent out
     AMContainer amContainer = appContext.getAllContainers().get(containerId);
     if (amContainer != null) {
       sendEvent(new AMContainerEventStopRequest(containerId));
@@ -587,28 +590,27 @@ public class TaskSchedulerEventHandler extends AbstractService
   }
 
   @SuppressWarnings("unchecked")
-  @Override
-  public synchronized void nodesUpdated(List<NodeReport> updatedNodes) {
+  public synchronized void nodesUpdated(int schedulerId, List<NodeReport> updatedNodes) {
     for (NodeReport nr : updatedNodes) {
       // Scheduler will find out from the node, if at all.
       // Relying on the RM to not allocate containers on an unhealthy node.
-      eventHandler.handle(new AMNodeEventStateChanged(nr));
+      eventHandler.handle(new AMNodeEventStateChanged(nr, schedulerId));
     }
   }
 
-  @Override
-  public synchronized void appShutdownRequested() {
+  public synchronized void appShutdownRequested(int schedulerId) {
     // This can happen if the RM has been restarted. If it is in that state,
     // this application must clean itself up.
-    LOG.info("App shutdown requested by scheduler");
+    LOG.info("App shutdown requested by scheduler {}", schedulerId);
     sendEvent(new DAGAppMasterEvent(DAGAppMasterEventType.AM_REBOOT));
   }
 
-  @Override
   public synchronized void setApplicationRegistrationData(
+      int schedulerId,
       Resource maxContainerCapability,
       Map<ApplicationAccessType, String> appAcls, 
       ByteBuffer clientAMSecretKey) {
+    // TODO TEZ-2003 (post) Ideally clusterInfo should be available per source rather than a global view.
     this.appContext.getClusterInfo().setMaxContainerCapability(
         maxContainerCapability);
     this.appAcls = appAcls;
@@ -619,7 +621,6 @@ public class TaskSchedulerEventHandler extends AbstractService
   // TaskScheduler uses a separate thread for it's callbacks. Since this method
   // returns a value which is required, the TaskScheduler wait for the call to
   // complete and can hence lead to a deadlock if called from within a TSEH lock.
-  @Override
   public AppFinalStatus getFinalAppStatus() {
     FinalApplicationStatus finishState = FinalApplicationStatus.UNDEFINED;
     StringBuffer sb = new StringBuffer();
@@ -661,24 +662,25 @@ public class TaskSchedulerEventHandler extends AbstractService
   // TaskScheduler uses a separate thread for it's callbacks. Since this method
   // returns a value which is required, the TaskScheduler wait for the call to
   // complete and can hence lead to a deadlock if called from within a TSEH lock.
-  @Override
-  public float getProgress() {
+  public float getProgress(int schedulerId) {
     // 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.
+
+    // Doubles as a mechanism to update node counts periodically. Hence schedulerId required.
+
     // 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;
-      sendEvent(new AMNodeEventNodeCountUpdated(cachedNodeCount));
+      sendEvent(new AMNodeEventNodeCountUpdated(cachedNodeCount, schedulerId));
     }
     return dagAppMaster.getProgress();
   }
 
-  @Override
-  public void onError(Throwable t) {
-    LOG.info("Error reported by scheduler", t);
+  public void onError(int schedulerId, Throwable t) {
+    LOG.info("Error reported by scheduler {} - {}", schedulerId, t);
     sendEvent(new DAGAppMasterEventSchedulingServiceError(t));
   }
 
@@ -693,8 +695,7 @@ public class TaskSchedulerEventHandler extends AbstractService
     // the context has updated information.
   }
 
-  @Override
-  public void preemptContainer(ContainerId containerId) {
+  public void preemptContainer(int schedulerId, ContainerId 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);

http://git-wip-us.apache.org/repos/asf/tez/blob/a5dfca2b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeEvent.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeEvent.java
index a623cda..85bc513 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeEvent.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeEvent.java
@@ -24,13 +24,19 @@ import org.apache.hadoop.yarn.event.AbstractEvent;
 public class AMNodeEvent extends AbstractEvent<AMNodeEventType> {
 
   private final NodeId nodeId;
+  private final int sourceId; // Effectively the schedulerId
 
-  public AMNodeEvent(NodeId nodeId, AMNodeEventType type) {
+  public AMNodeEvent(NodeId nodeId, int sourceId, AMNodeEventType type) {
     super(type);
     this.nodeId = nodeId;
+    this.sourceId = sourceId;
   }
 
   public NodeId getNodeId() {
     return this.nodeId;
   }
+
+  public int getSourceId() {
+    return sourceId;
+  }
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/a5dfca2b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeEventContainerAllocated.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeEventContainerAllocated.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeEventContainerAllocated.java
index 0770969..e250f42 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeEventContainerAllocated.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeEventContainerAllocated.java
@@ -24,8 +24,8 @@ public class AMNodeEventContainerAllocated extends AMNodeEvent {
 
   private final ContainerId containerId;
 
-  public AMNodeEventContainerAllocated(NodeId nodeId, ContainerId containerId) {
-    super(nodeId, AMNodeEventType.N_CONTAINER_ALLOCATED);
+  public AMNodeEventContainerAllocated(NodeId nodeId, int sourceId, ContainerId containerId) {
+    super(nodeId, sourceId, AMNodeEventType.N_CONTAINER_ALLOCATED);
     this.containerId = containerId;
   }
 

http://git-wip-us.apache.org/repos/asf/tez/blob/a5dfca2b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeEventNodeCountUpdated.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeEventNodeCountUpdated.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeEventNodeCountUpdated.java
index 86ca1fc..3b35daf 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeEventNodeCountUpdated.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeEventNodeCountUpdated.java
@@ -22,8 +22,8 @@ public class AMNodeEventNodeCountUpdated extends AMNodeEvent {
 
   private final int count;
   
-  public AMNodeEventNodeCountUpdated(int nodeCount) {
-    super(null, AMNodeEventType.N_NODE_COUNT_UPDATED);
+  public AMNodeEventNodeCountUpdated(int nodeCount, int sourceId) {
+    super(null, sourceId, AMNodeEventType.N_NODE_COUNT_UPDATED);
     this.count = nodeCount;
   }
   

http://git-wip-us.apache.org/repos/asf/tez/blob/a5dfca2b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeEventStateChanged.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeEventStateChanged.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeEventStateChanged.java
index ca4e5bd..b371ddd 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeEventStateChanged.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeEventStateChanged.java
@@ -23,8 +23,8 @@ public class AMNodeEventStateChanged extends AMNodeEvent {
 
   private NodeReport nodeReport;
 
-  public AMNodeEventStateChanged(NodeReport nodeReport) {
-    super(nodeReport.getNodeId(), 
+  public AMNodeEventStateChanged(NodeReport nodeReport, int sourceId) {
+    super(nodeReport.getNodeId(), sourceId,
           (nodeReport.getNodeState().isUnusable() ? 
               AMNodeEventType.N_TURNED_UNHEALTHY :
               AMNodeEventType.N_TURNED_HEALTHY));

http://git-wip-us.apache.org/repos/asf/tez/blob/a5dfca2b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeEventTaskAttemptEnded.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeEventTaskAttemptEnded.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeEventTaskAttemptEnded.java
index c823236..4a4cb61 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeEventTaskAttemptEnded.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeEventTaskAttemptEnded.java
@@ -27,9 +27,9 @@ public class AMNodeEventTaskAttemptEnded extends AMNodeEvent {
   private final ContainerId containerId;
   private final TezTaskAttemptID taskAttemptId;
   
-  public AMNodeEventTaskAttemptEnded(NodeId nodeId, ContainerId containerId,
+  public AMNodeEventTaskAttemptEnded(NodeId nodeId, int sourceId, ContainerId containerId,
       TezTaskAttemptID taskAttemptId, boolean failed) {
-    super(nodeId, AMNodeEventType.N_TA_ENDED);
+    super(nodeId, sourceId, AMNodeEventType.N_TA_ENDED);
     this.failed = failed;
     this.containerId = containerId;
     this.taskAttemptId = taskAttemptId;

http://git-wip-us.apache.org/repos/asf/tez/blob/a5dfca2b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeEventTaskAttemptSucceeded.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeEventTaskAttemptSucceeded.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeEventTaskAttemptSucceeded.java
index b07d594..2b8cb7d 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeEventTaskAttemptSucceeded.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeEventTaskAttemptSucceeded.java
@@ -27,9 +27,9 @@ public class AMNodeEventTaskAttemptSucceeded extends AMNodeEvent {
   private final ContainerId containerId;
   private final TezTaskAttemptID taskAttemptId;
 
-  public AMNodeEventTaskAttemptSucceeded(NodeId nodeId,
+  public AMNodeEventTaskAttemptSucceeded(NodeId nodeId, int sourceId,
       ContainerId containerId, TezTaskAttemptID taskAttemptId) {
-    super(nodeId, AMNodeEventType.N_TA_SUCCEEDED);
+    super(nodeId, sourceId, AMNodeEventType.N_TA_SUCCEEDED);
     this.containerId = containerId;
     this.taskAttemptId = taskAttemptId;
   }

http://git-wip-us.apache.org/repos/asf/tez/blob/a5dfca2b/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 0d8e4cd..88b36cb1f 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
@@ -54,6 +54,7 @@ public class AMNodeImpl implements AMNode {
   private final ReadLock readLock;
   private final WriteLock writeLock;
   private final NodeId nodeId;
+  private final int sourceId;
   private final AppContext appContext;
   private final int maxTaskFailuresPerNode;
   private boolean blacklistingEnabled;
@@ -172,13 +173,14 @@ public class AMNodeImpl implements AMNode {
 
 
   @SuppressWarnings("rawtypes")
-  public AMNodeImpl(NodeId nodeId, int maxTaskFailuresPerNode,
+  public AMNodeImpl(NodeId nodeId, int sourceId, int maxTaskFailuresPerNode,
       EventHandler eventHandler, boolean blacklistingEnabled,
       AppContext appContext) {
     ReentrantReadWriteLock rwLock = new ReentrantReadWriteLock();
     this.readLock = rwLock.readLock();
     this.writeLock = rwLock.writeLock();
     this.nodeId = nodeId;
+    this.sourceId = sourceId;
     this.appContext = appContext;
     this.eventHandler = eventHandler;
     this.blacklistingEnabled = blacklistingEnabled;
@@ -247,7 +249,7 @@ public class AMNodeImpl implements AMNode {
 
   /* Blacklist the node with the AMNodeTracker and check if the node should be blacklisted */
   protected boolean registerBadNodeAndShouldBlacklist() {
-    return appContext.getNodeTracker().registerBadNodeAndShouldBlacklist(this);
+    return appContext.getNodeTracker().registerBadNodeAndShouldBlacklist(this, sourceId);
   }
 
   protected void blacklistSelf() {

http://git-wip-us.apache.org/repos/asf/tez/blob/a5dfca2b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeTracker.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeTracker.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeTracker.java
index 102cbe9..0668ff2 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeTracker.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeTracker.java
@@ -18,9 +18,8 @@
 
 package org.apache.tez.dag.app.rm.node;
 
-import java.util.HashSet;
-import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
 
 import org.apache.tez.dag.app.dag.DAG;
 import org.slf4j.Logger;
@@ -29,7 +28,6 @@ import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.service.AbstractService;
 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.TezConfiguration;
 import org.apache.tez.dag.api.TezUncheckedException;
@@ -42,23 +40,21 @@ public class AMNodeTracker extends AbstractService implements
   
   static final Logger LOG = LoggerFactory.getLogger(AMNodeTracker.class);
   
-  private final ConcurrentHashMap<NodeId, AMNode> nodeMap;
-  private final ConcurrentHashMap<String, Set<NodeId>> blacklistMap;
+  private final ConcurrentMap<Integer, PerSourceNodeTracker> perSourceNodeTrackers;
+
   @SuppressWarnings("rawtypes")
   private final EventHandler eventHandler;
   private final AppContext appContext;
-  private int numClusterNodes;
-  private boolean ignoreBlacklisting = false;
+
+  // Not final since it's setup in serviceInit
   private int maxTaskFailuresPerNode;
   private boolean nodeBlacklistingEnabled;
   private int blacklistDisablePercent;
-  float currentIgnoreBlacklistingCountThreshold = 0;
-  
+
   @SuppressWarnings("rawtypes")
   public AMNodeTracker(EventHandler eventHandler, AppContext appContext) {
     super("AMNodeMap");
-    this.nodeMap = new ConcurrentHashMap<NodeId, AMNode>();
-    this.blacklistMap = new ConcurrentHashMap<String, Set<NodeId>>();
+    this.perSourceNodeTrackers = new ConcurrentHashMap<>();
     this.eventHandler = eventHandler;
     this.appContext = appContext;
   }
@@ -76,7 +72,7 @@ public class AMNodeTracker extends AbstractService implements
           TezConfiguration.TEZ_AM_NODE_BLACKLISTING_IGNORE_THRESHOLD_DEFAULT);
 
     LOG.info("blacklistDisablePercent is " + blacklistDisablePercent +
-        ", blacklistingEnabled: " + nodeBlacklistingEnabled + 
+        ", blacklistingEnabled: " + nodeBlacklistingEnabled +
         ", maxTaskFailuresPerNode: " + maxTaskFailuresPerNode);
 
     if (blacklistDisablePercent < -1 || blacklistDisablePercent > 100) {
@@ -85,130 +81,66 @@ public class AMNodeTracker extends AbstractService implements
           + ". Should be an integer between 0 and 100 or -1 to disabled");
     }
   }
-  
-  public void nodeSeen(NodeId nodeId) {
-    if (nodeMap.putIfAbsent(nodeId, new AMNodeImpl(nodeId, maxTaskFailuresPerNode,
-        eventHandler, nodeBlacklistingEnabled, appContext)) == null) {
-      LOG.info("Adding new node: " + nodeId);
-    }
-  }
 
-  private void addToBlackList(NodeId nodeId) {
-    String host = nodeId.getHost();
-
-    if (!blacklistMap.containsKey(host)) {
-      blacklistMap.putIfAbsent(host, new HashSet<NodeId>());
-    }
-    Set<NodeId> nodes = blacklistMap.get(host);
-
-    if (!nodes.contains(nodeId)) {
-      nodes.add(nodeId);
-    }
+  public void nodeSeen(NodeId nodeId, int sourceId) {
+    PerSourceNodeTracker nodeTracker = getAndCreateIfNeededPerSourceTracker(sourceId);
+    nodeTracker.nodeSeen(nodeId);
   }
 
-  boolean registerBadNodeAndShouldBlacklist(AMNode amNode) {
-    if (nodeBlacklistingEnabled) {
-      addToBlackList(amNode.getNodeId());
-      computeIgnoreBlacklisting();
-      return !ignoreBlacklisting;
-    } else {
-      return false;
-    }
+
+  boolean registerBadNodeAndShouldBlacklist(AMNode amNode, int sourceId) {
+    return perSourceNodeTrackers.get(sourceId).registerBadNodeAndShouldBlacklist(amNode);
   }
 
   public void handle(AMNodeEvent rEvent) {
     // No synchronization required until there's multiple dispatchers.
-    NodeId nodeId = rEvent.getNodeId();
     switch (rEvent.getType()) {
-    case N_NODE_COUNT_UPDATED:
-      AMNodeEventNodeCountUpdated event = (AMNodeEventNodeCountUpdated) rEvent;
-      numClusterNodes = event.getNodeCount();
-      LOG.info("Num cluster nodes = " + numClusterNodes);
-      recomputeCurrentIgnoreBlacklistingThreshold();
-      computeIgnoreBlacklisting();
-      break;
-    case N_TURNED_UNHEALTHY:
-    case N_TURNED_HEALTHY:
-      AMNode amNode = nodeMap.get(nodeId);
-      if (amNode == null) {
-        LOG.info("Ignoring RM Health Update for unknown node: " + nodeId);
-      } else {
-        amNode.handle(rEvent);
-      }
-      break;
-    default:
-      nodeMap.get(nodeId).handle(rEvent);
+      case N_CONTAINER_ALLOCATED:
+      case N_TA_SUCCEEDED:
+      case N_TA_ENDED:
+      case N_IGNORE_BLACKLISTING_ENABLED:
+      case N_IGNORE_BLACKLISTING_DISABLED:
+        // All of these will only be seen after a node has been registered.
+        perSourceNodeTrackers.get(rEvent.getSourceId()).handle(rEvent);
+        break;
+      case N_TURNED_UNHEALTHY:
+      case N_TURNED_HEALTHY:
+      case N_NODE_COUNT_UPDATED:
+        // These events can be seen without a node having been marked as 'seen' before
+        getAndCreateIfNeededPerSourceTracker(rEvent.getSourceId()).handle(rEvent);
+        break;
     }
   }
 
-  private void recomputeCurrentIgnoreBlacklistingThreshold() {
-    if (nodeBlacklistingEnabled && blacklistDisablePercent != -1) {
-      currentIgnoreBlacklistingCountThreshold =
-          (float) numClusterNodes * blacklistDisablePercent / 100;
-    }
+  public AMNode get(NodeId nodeId, int sourceId) {
+    return perSourceNodeTrackers.get(sourceId).get(nodeId);
   }
 
-  // May be incorrect if there's multiple NodeManagers running on a single host.
-  // knownNodeCount is based on node managers, not hosts. blacklisting is
-  // currently based on hosts.
-  protected void computeIgnoreBlacklisting() {
-
-    boolean stateChanged = false;
-
-    if (!nodeBlacklistingEnabled || blacklistDisablePercent == -1 || blacklistMap.size() == 0) {
-      return;
-    }
-    if (blacklistMap.size() >= currentIgnoreBlacklistingCountThreshold) {
-      if (ignoreBlacklisting == false) {
-        ignoreBlacklisting = true;
-        LOG.info("Ignore Blacklisting set to true. Known: " + numClusterNodes
-            + ", Blacklisted: " + blacklistMap.size());
-        stateChanged = true;
-      }
-    } else {
-      if (ignoreBlacklisting == true) {
-        ignoreBlacklisting = false;
-        LOG.info("Ignore blacklisting set to false. Known: "
-            + numClusterNodes + ", Blacklisted: " + blacklistMap.size());
-        stateChanged = true;
-      }
-    }
-
-    if (stateChanged) {
-      sendIngoreBlacklistingStateToNodes();
-    }
-  }
-
-  private void sendIngoreBlacklistingStateToNodes() {
-    AMNodeEventType eventType =
-        ignoreBlacklisting ? AMNodeEventType.N_IGNORE_BLACKLISTING_ENABLED
-        : AMNodeEventType.N_IGNORE_BLACKLISTING_DISABLED;
-    for (NodeId nodeId : nodeMap.keySet()) {
-      sendEvent(new AMNodeEvent(nodeId, eventType));
-    }
-  }
-
-  public AMNode get(NodeId nodeId) {
-    return nodeMap.get(nodeId);
-  }
-
-  @SuppressWarnings("unchecked")
-  private void sendEvent(Event<?> event) {
-    this.eventHandler.handle(event);
-  }
-
-  public int getNumNodes() {
-    return nodeMap.size();
+  public int getNumNodes(int sourceId) {
+    return perSourceNodeTrackers.get(sourceId).getNumNodes();
   }
 
   @Private
   @VisibleForTesting
-  public boolean isBlacklistingIgnored() {
-    return this.ignoreBlacklisting;
+  public boolean isBlacklistingIgnored(int sourceId) {
+    return perSourceNodeTrackers.get(sourceId).isBlacklistingIgnored();
   }
 
   public void dagComplete(DAG dag) {
     // TODO TEZ-2337 Maybe reset failures from previous DAGs
   }
 
+  private PerSourceNodeTracker getAndCreateIfNeededPerSourceTracker(int sourceId) {
+    PerSourceNodeTracker nodeTracker = perSourceNodeTrackers.get(sourceId);
+    if (nodeTracker == null) {
+      nodeTracker =
+          new PerSourceNodeTracker(sourceId, eventHandler, appContext, maxTaskFailuresPerNode,
+              nodeBlacklistingEnabled, blacklistDisablePercent);
+      PerSourceNodeTracker old = perSourceNodeTrackers.putIfAbsent(sourceId, nodeTracker);
+      nodeTracker = old != null ? old : nodeTracker;
+    }
+    return nodeTracker;
+  }
+
+
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/a5dfca2b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/PerSourceNodeTracker.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/PerSourceNodeTracker.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/PerSourceNodeTracker.java
new file mode 100644
index 0000000..3264708
--- /dev/null
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/PerSourceNodeTracker.java
@@ -0,0 +1,187 @@
+/*
+ * 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.node;
+
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.classification.InterfaceAudience;
+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.app.AppContext;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class PerSourceNodeTracker {
+
+  static final Logger LOG = LoggerFactory.getLogger(PerSourceNodeTracker.class);
+
+  private final int sourceId;
+  private final ConcurrentHashMap<NodeId, AMNode> nodeMap;
+  private final ConcurrentHashMap<String, Set<NodeId>> blacklistMap;
+
+  @SuppressWarnings("rawtypes")
+  private final EventHandler eventHandler;
+  private final AppContext appContext;
+
+  private final int maxTaskFailuresPerNode;
+  private final boolean nodeBlacklistingEnabled;
+  private final int blacklistDisablePercent;
+
+  private int numClusterNodes;
+  float currentIgnoreBlacklistingCountThreshold = 0;
+  private boolean ignoreBlacklisting = false;
+
+  @SuppressWarnings("rawtypes")
+  public PerSourceNodeTracker(int sourceId, EventHandler eventHandler, AppContext appContext,
+                              int maxTaskFailuresPerNode, boolean nodeBlacklistingEnabled,
+                              int blacklistDisablePercent) {
+    this.sourceId = sourceId;
+    this.nodeMap = new ConcurrentHashMap<>();
+    this.blacklistMap = new ConcurrentHashMap<>();
+    this.eventHandler = eventHandler;
+    this.appContext = appContext;
+
+    this.maxTaskFailuresPerNode = maxTaskFailuresPerNode;
+    this.nodeBlacklistingEnabled = nodeBlacklistingEnabled;
+    this.blacklistDisablePercent = blacklistDisablePercent;
+  }
+
+
+
+  public void nodeSeen(NodeId nodeId) {
+    if (nodeMap.putIfAbsent(nodeId, new AMNodeImpl(nodeId, sourceId, maxTaskFailuresPerNode,
+        eventHandler, nodeBlacklistingEnabled, appContext)) == null) {
+      LOG.info("Adding new node {} to nodeTracker {}", nodeId, sourceId);
+    }
+  }
+
+  public AMNode get(NodeId nodeId) {
+    return nodeMap.get(nodeId);
+  }
+
+  public int getNumNodes() {
+    return nodeMap.size();
+  }
+
+  public void handle(AMNodeEvent rEvent) {
+    // No synchronization required until there's multiple dispatchers.
+    NodeId nodeId = rEvent.getNodeId();
+    switch (rEvent.getType()) {
+      case N_NODE_COUNT_UPDATED:
+        AMNodeEventNodeCountUpdated event = (AMNodeEventNodeCountUpdated) rEvent;
+        numClusterNodes = event.getNodeCount();
+        LOG.info("Num cluster nodes = " + numClusterNodes);
+        recomputeCurrentIgnoreBlacklistingThreshold();
+        computeIgnoreBlacklisting();
+        break;
+      case N_TURNED_UNHEALTHY:
+      case N_TURNED_HEALTHY:
+        AMNode amNode = nodeMap.get(nodeId);
+        if (amNode == null) {
+          LOG.info("Ignoring RM Health Update for unknown node: " + nodeId);
+        } else {
+          amNode.handle(rEvent);
+        }
+        break;
+      default:
+        nodeMap.get(nodeId).handle(rEvent);
+    }
+  }
+
+  boolean registerBadNodeAndShouldBlacklist(AMNode amNode) {
+    if (nodeBlacklistingEnabled) {
+      addToBlackList(amNode.getNodeId());
+      computeIgnoreBlacklisting();
+      return !ignoreBlacklisting;
+    } else {
+      return false;
+    }
+  }
+
+  @InterfaceAudience.Private
+  @VisibleForTesting
+  public boolean isBlacklistingIgnored() {
+    return this.ignoreBlacklisting;
+  }
+
+  private void recomputeCurrentIgnoreBlacklistingThreshold() {
+    if (nodeBlacklistingEnabled && blacklistDisablePercent != -1) {
+      currentIgnoreBlacklistingCountThreshold =
+          (float) numClusterNodes * blacklistDisablePercent / 100;
+    }
+  }
+
+  // May be incorrect if there's multiple NodeManagers running on a single host.
+  // knownNodeCount is based on node managers, not hosts. blacklisting is
+  // currently based on hosts.
+  protected void computeIgnoreBlacklisting() {
+
+    boolean stateChanged = false;
+
+    if (!nodeBlacklistingEnabled || blacklistDisablePercent == -1 || blacklistMap.size() == 0) {
+      return;
+    }
+    if (blacklistMap.size() >= currentIgnoreBlacklistingCountThreshold) {
+      if (ignoreBlacklisting == false) {
+        ignoreBlacklisting = true;
+        LOG.info("Ignore Blacklisting set to true. Known: " + numClusterNodes
+            + ", Blacklisted: " + blacklistMap.size());
+        stateChanged = true;
+      }
+    } else {
+      if (ignoreBlacklisting == true) {
+        ignoreBlacklisting = false;
+        LOG.info("Ignore blacklisting set to false. Known: "
+            + numClusterNodes + ", Blacklisted: " + blacklistMap.size());
+        stateChanged = true;
+      }
+    }
+
+    if (stateChanged) {
+      sendIngoreBlacklistingStateToNodes();
+    }
+  }
+
+  private void addToBlackList(NodeId nodeId) {
+    String host = nodeId.getHost();
+
+    if (!blacklistMap.containsKey(host)) {
+      blacklistMap.putIfAbsent(host, new HashSet<NodeId>());
+    }
+    Set<NodeId> nodes = blacklistMap.get(host);
+
+    if (!nodes.contains(nodeId)) {
+      nodes.add(nodeId);
+    }
+  }
+
+  private void sendIngoreBlacklistingStateToNodes() {
+    AMNodeEventType eventType =
+        ignoreBlacklisting ? AMNodeEventType.N_IGNORE_BLACKLISTING_ENABLED
+            : AMNodeEventType.N_IGNORE_BLACKLISTING_DISABLED;
+    for (NodeId nodeId : nodeMap.keySet()) {
+      sendEvent(new AMNodeEvent(nodeId, sourceId, eventType));
+    }
+  }
+
+  @SuppressWarnings("unchecked")
+  private void sendEvent(Event<?> event) {
+    this.eventHandler.handle(event);
+  }
+}

http://git-wip-us.apache.org/repos/asf/tez/blob/a5dfca2b/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 9882954..0f35bba 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
@@ -257,7 +257,7 @@ public class MockDAGAppMaster extends DAGAppMaster {
     }
     
     public void preemptContainer(ContainerData cData) {
-      getTaskSchedulerEventHandler().containerCompleted(null, 
+      getTaskSchedulerEventHandler().containerCompleted(0, null,
           ContainerStatus.newInstance(cData.cId, null, "Preempted", ContainerExitStatus.PREEMPTED));
       cData.clear();
     }

http://git-wip-us.apache.org/repos/asf/tez/blob/a5dfca2b/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 42d4b0b..7584b4c 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
@@ -200,7 +200,7 @@ public class TestMockDAGAppMaster {
     mockLauncher.waitTillContainersLaunched();
     ContainerData cData = mockLauncher.getContainers().values().iterator().next();
     DAGImpl dagImpl = (DAGImpl) mockApp.getContext().getCurrentDAG();
-    mockApp.getTaskSchedulerEventHandler().preemptContainer(cData.cId);
+    mockApp.getTaskSchedulerEventHandler().preemptContainer(0, cData.cId);
     
     mockLauncher.startScheduling(true);
     dagClient.waitForCompletion();

http://git-wip-us.apache.org/repos/asf/tez/blob/a5dfca2b/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 080c20f..62edac9 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
@@ -216,9 +216,9 @@ public class TestContainerReuse {
     TestTaskSchedulerHelpers.waitForDelayedDrainNotify(drainNotifier);
     drainableAppCallback.drain();
     verify(taskSchedulerEventHandler).taskAllocated(
-      eq(ta11), any(Object.class), eq(containerHost1));
+      eq(0), eq(ta11), any(Object.class), eq(containerHost1));
     verify(taskSchedulerEventHandler).taskAllocated(
-      eq(ta21), any(Object.class), eq(containerHost2));
+      eq(0), eq(ta21), any(Object.class), eq(containerHost2));
 
     // Adding the event later so that task1 assigned to containerHost1
     // is deterministic.
@@ -230,7 +230,7 @@ public class TestContainerReuse {
     drainableAppCallback.drain();
     verify(taskScheduler).deallocateTask(eq(ta11), eq(true), eq((TaskAttemptEndReason)null));
     verify(taskSchedulerEventHandler, times(1)).taskAllocated(
-      eq(ta31), any(Object.class), eq(containerHost1));
+      eq(0), eq(ta31), any(Object.class), eq(containerHost1));
     verify(rmClient, times(0)).releaseAssignedContainer(
       eq(containerHost1.getId()));
     eventHandler.verifyNoInvocations(AMContainerEventStopRequest.class);
@@ -245,7 +245,7 @@ public class TestContainerReuse {
     while (System.currentTimeMillis() < currentTs + 5000l) {
       try {
         verify(taskSchedulerEventHandler,
-          times(1)).containerBeingReleased(eq(containerHost2.getId()));
+          times(1)).containerBeingReleased(eq(0), eq(containerHost2.getId()));
         exception = null;
         break;
       } catch (Throwable e) {
@@ -351,8 +351,8 @@ public class TestContainerReuse {
     taskScheduler.onContainersAllocated(Lists.newArrayList(containerHost1, containerHost2));
     TestTaskSchedulerHelpers.waitForDelayedDrainNotify(drainNotifier);
     drainableAppCallback.drain();
-    verify(taskSchedulerEventHandler).taskAllocated(eq(ta11), any(Object.class), eq(containerHost1));
-    verify(taskSchedulerEventHandler).taskAllocated(eq(ta21), any(Object.class), eq(containerHost2));
+    verify(taskSchedulerEventHandler).taskAllocated(eq(0), eq(ta11), any(Object.class), eq(containerHost1));
+    verify(taskSchedulerEventHandler).taskAllocated(eq(0), eq(ta21), any(Object.class), eq(containerHost2));
 
     // Adding the event later so that task1 assigned to containerHost1 is deterministic.
     taskSchedulerEventHandler.handleEvent(lrTa31);
@@ -363,7 +363,7 @@ public class TestContainerReuse {
     drainableAppCallback.drain();
     verify(taskScheduler).deallocateTask(eq(ta21), eq(true), eq((TaskAttemptEndReason)null));
     verify(taskSchedulerEventHandler, times(0)).taskAllocated(
-      eq(ta31), any(Object.class), eq(containerHost2));
+        eq(0), eq(ta31), any(Object.class), eq(containerHost2));
     verify(rmClient, times(1)).releaseAssignedContainer(
       eq(containerHost2.getId()));
     eventHandler.verifyInvocation(AMContainerEventStopRequest.class);
@@ -459,13 +459,13 @@ public class TestContainerReuse {
     taskScheduler.onContainersAllocated(Collections.singletonList(container1));
     TestTaskSchedulerHelpers.waitForDelayedDrainNotify(drainNotifier);
     drainableAppCallback.drain();
-    verify(taskSchedulerEventHandler).taskAllocated(eq(ta11), any(Object.class), eq(container1));
+    verify(taskSchedulerEventHandler).taskAllocated(eq(0), 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, null, 0));
     drainableAppCallback.drain();
     verify(taskScheduler).deallocateTask(eq(ta11), eq(true), eq((TaskAttemptEndReason)null));
-    verify(taskSchedulerEventHandler).taskAllocated(eq(ta12), any(Object.class), eq(container1));
+    verify(taskSchedulerEventHandler).taskAllocated(eq(0), eq(ta12), any(Object.class), eq(container1));
     verify(rmClient, times(0)).releaseAssignedContainer(eq(container1.getId()));
     eventHandler.verifyNoInvocations(AMContainerEventStopRequest.class);
     eventHandler.reset();
@@ -475,7 +475,7 @@ public class TestContainerReuse {
     taskSchedulerEventHandler.handleEvent(new AMSchedulerEventTAEnded(ta12, container1.getId(), TaskAttemptState.SUCCEEDED, null, 0));
     drainableAppCallback.drain();
     verify(taskScheduler).deallocateTask(eq(ta12), eq(true), eq((TaskAttemptEndReason)null));
-    verify(taskSchedulerEventHandler).taskAllocated(eq(ta13), any(Object.class), eq(container1));
+    verify(taskSchedulerEventHandler).taskAllocated(eq(0), eq(ta13), any(Object.class), eq(container1));
     verify(rmClient, times(0)).releaseAssignedContainer(eq(container1.getId()));
     eventHandler.verifyNoInvocations(AMContainerEventStopRequest.class);
     eventHandler.reset();
@@ -483,7 +483,7 @@ public class TestContainerReuse {
     // Verify no re-use if a previous task fails.
     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(taskSchedulerEventHandler, times(0)).taskAllocated(eq(0), eq(ta14), any(Object.class), eq(container1));
     verify(taskScheduler).deallocateTask(eq(ta13), eq(false), eq((TaskAttemptEndReason)null));
     verify(rmClient).releaseAssignedContainer(eq(container1.getId()));
     eventHandler.verifyInvocation(AMContainerEventStopRequest.class);
@@ -496,7 +496,7 @@ public class TestContainerReuse {
     taskScheduler.onContainersAllocated(Collections.singletonList(container2));
     TestTaskSchedulerHelpers.waitForDelayedDrainNotify(drainNotifier);
     drainableAppCallback.drain();
-    verify(taskSchedulerEventHandler).taskAllocated(eq(ta14), any(Object.class), eq(container2));
+    verify(taskSchedulerEventHandler).taskAllocated(eq(0), 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, null, 0));
@@ -606,14 +606,14 @@ public class TestContainerReuse {
     taskScheduler.onContainersAllocated(Collections.singletonList(container1));
     TestTaskSchedulerHelpers.waitForDelayedDrainNotify(drainNotifier);
     drainableAppCallback.drain();
-    verify(taskSchedulerEventHandler).taskAllocated(eq(ta11), any(Object.class), eq(container1));
+    verify(taskSchedulerEventHandler).taskAllocated(eq(0), eq(ta11), any(Object.class), eq(container1));
 
     // First task had profiling on. This container can not be reused further.
     taskSchedulerEventHandler.handleEvent(
         new AMSchedulerEventTAEnded(ta11, container1.getId(), TaskAttemptState.SUCCEEDED, null, 0));
     drainableAppCallback.drain();
     verify(taskScheduler).deallocateTask(eq(ta11), eq(true), eq((TaskAttemptEndReason)null));
-    verify(taskSchedulerEventHandler, times(0)).taskAllocated(eq(ta12), any(Object.class),
+    verify(taskSchedulerEventHandler, times(0)).taskAllocated(eq(0), eq(ta12), any(Object.class),
         eq(container1));
     verify(rmClient, times(1)).releaseAssignedContainer(eq(container1.getId()));
     eventHandler.verifyInvocation(AMContainerEventStopRequest.class);
@@ -652,14 +652,14 @@ public class TestContainerReuse {
     taskScheduler.onContainersAllocated(Collections.singletonList(container2));
     TestTaskSchedulerHelpers.waitForDelayedDrainNotify(drainNotifier);
     drainableAppCallback.drain();
-    verify(taskSchedulerEventHandler).taskAllocated(eq(ta13), any(Object.class), eq(container2));
+    verify(taskSchedulerEventHandler).taskAllocated(eq(0), eq(ta13), any(Object.class), eq(container2));
 
     // 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, null, 0));
     drainableAppCallback.drain();
     verify(taskScheduler).deallocateTask(eq(ta13), eq(true), eq((TaskAttemptEndReason)null));
-    verify(taskSchedulerEventHandler, times(0)).taskAllocated(eq(ta14), any(Object.class),
+    verify(taskSchedulerEventHandler, times(0)).taskAllocated(eq(0), eq(ta14), any(Object.class),
         eq(container2));
     verify(rmClient, times(1)).releaseAssignedContainer(eq(container2.getId()));
     eventHandler.verifyInvocation(AMContainerEventStopRequest.class);
@@ -698,13 +698,13 @@ public class TestContainerReuse {
     taskScheduler.onContainersAllocated(Collections.singletonList(container3));
     TestTaskSchedulerHelpers.waitForDelayedDrainNotify(drainNotifier);
     drainableAppCallback.drain();
-    verify(taskSchedulerEventHandler).taskAllocated(eq(ta15), any(Object.class), eq(container3));
+    verify(taskSchedulerEventHandler).taskAllocated(eq(0), 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, null, 0));
     drainableAppCallback.drain();
     verify(taskScheduler).deallocateTask(eq(ta15), eq(true), eq((TaskAttemptEndReason)null));
-    verify(taskSchedulerEventHandler).taskAllocated(eq(ta16), any(Object.class), eq(container3));
+    verify(taskSchedulerEventHandler).taskAllocated(eq(0), eq(ta16), any(Object.class), eq(container3));
     eventHandler.reset();
 
     taskScheduler.close();
@@ -804,7 +804,7 @@ public class TestContainerReuse {
     TestTaskSchedulerHelpers.waitForDelayedDrainNotify(drainNotifier);
     drainableAppCallback.drain();
     verify(taskSchedulerEventHandler).taskAllocated(
-      eq(ta11), any(Object.class), eq(container1));
+        eq(0), eq(ta11), any(Object.class), eq(container1));
 
     // Send launch request for task2 (vertex2)
     taskSchedulerEventHandler.handleEvent(lrEvent12);
@@ -818,7 +818,7 @@ public class TestContainerReuse {
     drainableAppCallback.drain();
     verify(taskScheduler).deallocateTask(eq(ta11), eq(true), eq((TaskAttemptEndReason)null));
     verify(taskSchedulerEventHandler, times(0)).taskAllocated(
-      eq(ta12), any(Object.class), eq(container1));
+        eq(0), eq(ta12), any(Object.class), eq(container1));
     verify(rmClient, times(0)).releaseAssignedContainer(eq(container1.getId()));
     eventHandler.verifyNoInvocations(AMContainerEventStopRequest.class);
     eventHandler.reset();
@@ -826,7 +826,7 @@ public class TestContainerReuse {
     LOG.info("Sleeping to ensure that the scheduling loop runs");
     Thread.sleep(3000l);
     verify(taskSchedulerEventHandler).taskAllocated(
-      eq(ta12), any(Object.class), eq(container1));
+        eq(0), eq(ta12), any(Object.class), eq(container1));
 
     // TA12 completed.
     taskSchedulerEventHandler.handleEvent(
@@ -940,7 +940,7 @@ public class TestContainerReuse {
     TestTaskSchedulerHelpers.waitForDelayedDrainNotify(drainNotifier);
     drainableAppCallback.drain();
     verify(taskSchedulerEventHandler).taskAllocated(
-      eq(ta11), any(Object.class), eq(container1));
+        eq(0), eq(ta11), any(Object.class), eq(container1));
 
     // Send launch request for task2 (vertex2)
     taskSchedulerEventHandler.handleEvent(lrEvent21);
@@ -953,7 +953,7 @@ public class TestContainerReuse {
     drainableAppCallback.drain();
     verify(taskScheduler).deallocateTask(eq(ta11), eq(true), eq((TaskAttemptEndReason)null));
     verify(taskSchedulerEventHandler).taskAllocated(
-      eq(ta21), any(Object.class), eq(container1));
+        eq(0), eq(ta21), any(Object.class), eq(container1));
     verify(rmClient, times(0)).releaseAssignedContainer(eq(container1.getId()));
 
     // Task 2 completes.
@@ -1063,7 +1063,7 @@ public class TestContainerReuse {
     taskScheduler.onContainersAllocated(Collections.singletonList(container1));
     TestTaskSchedulerHelpers.waitForDelayedDrainNotify(drainNotifier);
     drainableAppCallback.drain();
-    verify(taskSchedulerEventHandler).taskAllocated(eq(ta111), any(Object.class), eq(container1));
+    verify(taskSchedulerEventHandler).taskAllocated(eq(0), eq(ta111), any(Object.class), eq(container1));
     assignEvent = (AMContainerEventAssignTA) eventHandler.verifyInvocation(AMContainerEventAssignTA.class);
     assertEquals(1, assignEvent.getRemoteTaskLocalResources().size());
     
@@ -1071,7 +1071,7 @@ public class TestContainerReuse {
     taskSchedulerEventHandler.handleEvent(new AMSchedulerEventTAEnded(ta111, container1.getId(), TaskAttemptState.SUCCEEDED, null, 0));
     drainableAppCallback.drain();
     verify(taskScheduler).deallocateTask(eq(ta111), eq(true), eq((TaskAttemptEndReason)null));
-    verify(taskSchedulerEventHandler).taskAllocated(eq(ta112), any(Object.class), eq(container1));
+    verify(taskSchedulerEventHandler).taskAllocated(eq(0), eq(ta112), any(Object.class), eq(container1));
     verify(rmClient, times(0)).releaseAssignedContainer(eq(container1.getId()));
     eventHandler.verifyNoInvocations(AMContainerEventStopRequest.class);
     assignEvent = (AMContainerEventAssignTA) eventHandler.verifyInvocation(AMContainerEventAssignTA.class);
@@ -1114,7 +1114,7 @@ public class TestContainerReuse {
     // TODO This is terrible, need a better way to ensure the scheduling loop has run
     LOG.info("Sleeping to ensure that the scheduling loop runs");
     Thread.sleep(6000l);
-    verify(taskSchedulerEventHandler).taskAllocated(eq(ta211), any(Object.class), eq(container1));
+    verify(taskSchedulerEventHandler).taskAllocated(eq(0), eq(ta211), any(Object.class), eq(container1));
     verify(rmClient, times(0)).releaseAssignedContainer(eq(container1.getId()));
     eventHandler.verifyNoInvocations(AMContainerEventStopRequest.class);
     assignEvent = (AMContainerEventAssignTA) eventHandler.verifyInvocation(AMContainerEventAssignTA.class);
@@ -1124,7 +1124,7 @@ public class TestContainerReuse {
     taskSchedulerEventHandler.handleEvent(new AMSchedulerEventTAEnded(ta211, container1.getId(), TaskAttemptState.SUCCEEDED, null, 0));
     drainableAppCallback.drain();
     verify(taskScheduler).deallocateTask(eq(ta211), eq(true), eq((TaskAttemptEndReason)null));
-    verify(taskSchedulerEventHandler).taskAllocated(eq(ta212), any(Object.class), eq(container1));
+    verify(taskSchedulerEventHandler).taskAllocated(eq(0), eq(ta212), any(Object.class), eq(container1));
     verify(rmClient, times(0)).releaseAssignedContainer(eq(container1.getId()));
     eventHandler.verifyNoInvocations(AMContainerEventStopRequest.class);
     assignEvent = (AMContainerEventAssignTA) eventHandler.verifyInvocation(AMContainerEventAssignTA.class);
@@ -1237,7 +1237,7 @@ public class TestContainerReuse {
     taskScheduler.onContainersAllocated(Collections.singletonList(container1));
     TestTaskSchedulerHelpers.waitForDelayedDrainNotify(drainNotifier);
     drainableAppCallback.drain();
-    verify(taskSchedulerEventHandler).taskAllocated(eq(ta111), any(Object.class), eq(container1));
+    verify(taskSchedulerEventHandler).taskAllocated(eq(0), eq(ta111), any(Object.class), eq(container1));
     assignEvent = (AMContainerEventAssignTA) eventHandler.verifyInvocation(AMContainerEventAssignTA.class);
     assertEquals(1, assignEvent.getRemoteTaskLocalResources().size());
 
@@ -1245,7 +1245,7 @@ public class TestContainerReuse {
     taskSchedulerEventHandler.handleEvent(new AMSchedulerEventTAEnded(ta111, container1.getId(), TaskAttemptState.SUCCEEDED, null, 0));
     drainableAppCallback.drain();
     verify(taskScheduler).deallocateTask(eq(ta111), eq(true), eq((TaskAttemptEndReason)null));
-    verify(taskSchedulerEventHandler).taskAllocated(eq(ta112), any(Object.class), eq(container1));
+    verify(taskSchedulerEventHandler).taskAllocated(eq(0), eq(ta112), any(Object.class), eq(container1));
     verify(rmClient, times(0)).releaseAssignedContainer(eq(container1.getId()));
     eventHandler.verifyNoInvocations(AMContainerEventStopRequest.class);
     assignEvent = (AMContainerEventAssignTA) eventHandler.verifyInvocation(AMContainerEventAssignTA.class);
@@ -1290,7 +1290,7 @@ public class TestContainerReuse {
 
     Thread.sleep(6000l);
     verify(rmClient, times(1)).releaseAssignedContainer(eq(container1.getId()));
-    verify(taskSchedulerEventHandler).taskAllocated(eq(ta211), any(Object.class), eq(container2));
+    verify(taskSchedulerEventHandler).taskAllocated(eq(0), eq(ta211), any(Object.class), eq(container2));
     eventHandler.reset();
 
     taskScheduler.close();
@@ -1369,7 +1369,7 @@ public class TestContainerReuse {
     drainNotifier.set(false);
     taskScheduler.onContainersAllocated(Collections.singletonList(container1));
     drainableAppCallback.drain();
-    verify(taskSchedulerEventHandler, times(0)).taskAllocated(eq(ta11),
+    verify(taskSchedulerEventHandler, times(0)).taskAllocated(eq(0), eq(ta11),
         any(Object.class), eq(container1));
     taskScheduler.close();
     taskSchedulerEventHandler.close();

http://git-wip-us.apache.org/repos/asf/tez/blob/a5dfca2b/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 daf1db6..005692e 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
@@ -163,7 +163,7 @@ public class TestTaskSchedulerEventHandler {
     AMSchedulerEventTALaunchRequest lr =
         new AMSchedulerEventTALaunchRequest(mockAttemptId, resource, null, mockTaskAttempt, locHint,
             priority, containerContext, 0, 0, 0);
-    schedulerHandler.taskAllocated(mockTaskAttempt, lr, container);
+    schedulerHandler.taskAllocated(0, mockTaskAttempt, lr, container);
     assertEquals(2, mockEventHandler.events.size());
     assertTrue(mockEventHandler.events.get(1) instanceof AMContainerEventAssignTA);
     AMContainerEventAssignTA assignEvent =
@@ -227,7 +227,7 @@ public class TestTaskSchedulerEventHandler {
     when(mockStatus.getContainerId()).thenReturn(mockCId);
     when(mockStatus.getDiagnostics()).thenReturn(diagnostics);
     when(mockStatus.getExitStatus()).thenReturn(ContainerExitStatus.PREEMPTED);
-    schedulerHandler.containerCompleted(mockTask, mockStatus);
+    schedulerHandler.containerCompleted(0, mockTask, mockStatus);
     assertEquals(1, mockEventHandler.events.size());
     Event event = mockEventHandler.events.get(0);
     assertEquals(AMContainerEventType.C_COMPLETED, event.getType());
@@ -257,7 +257,7 @@ public class TestTaskSchedulerEventHandler {
     ContainerId mockCId = mock(ContainerId.class);
     verify(mockTaskScheduler, times(0)).deallocateContainer((ContainerId)any());
     when(mockAMContainerMap.get(mockCId)).thenReturn(mockAmContainer);
-    schedulerHandler.preemptContainer(mockCId);
+    schedulerHandler.preemptContainer(0, mockCId);
     verify(mockTaskScheduler, times(1)).deallocateContainer(mockCId);
     assertEquals(1, mockEventHandler.events.size());
     Event event = mockEventHandler.events.get(0);
@@ -290,7 +290,7 @@ public class TestTaskSchedulerEventHandler {
     when(mockStatus.getContainerId()).thenReturn(mockCId);
     when(mockStatus.getDiagnostics()).thenReturn(diagnostics);
     when(mockStatus.getExitStatus()).thenReturn(ContainerExitStatus.DISKS_FAILED);
-    schedulerHandler.containerCompleted(mockTask, mockStatus);
+    schedulerHandler.containerCompleted(0, mockTask, mockStatus);
     assertEquals(1, mockEventHandler.events.size());
     Event event = mockEventHandler.events.get(0);
     assertEquals(AMContainerEventType.C_COMPLETED, event.getType());
@@ -325,7 +325,7 @@ public class TestTaskSchedulerEventHandler {
     // use -104 rather than ContainerExitStatus.KILLED_EXCEEDED_PMEM because
     // ContainerExitStatus.KILLED_EXCEEDED_PMEM is only available after hadoop-2.5
     when(mockStatus.getExitStatus()).thenReturn(-104);
-    schedulerHandler.containerCompleted(mockTask, mockStatus);
+    schedulerHandler.containerCompleted(0, mockTask, mockStatus);
     assertEquals(1, mockEventHandler.events.size());
     Event event = mockEventHandler.events.get(0);
     assertEquals(AMContainerEventType.C_COMPLETED, event.getType());
@@ -383,4 +383,5 @@ public class TestTaskSchedulerEventHandler {
 
   }
 
+  // TODO TEZ-2003. Add tests with multiple schedulers, and ensuring that events go out with correct IDs.
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/a5dfca2b/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 ffab769..04610ab 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
@@ -134,7 +134,7 @@ class TestTaskSchedulerHelpers {
 
     @Override
     public void instantiateScheduelrs(String host, int port, String trackingUrl, AppContext appContext) {
-      taskSchedulers[0] = new TaskSchedulerWithDrainableAppCallback(this,
+      taskSchedulers[0] = new TaskSchedulerWithDrainableAppCallback(new TaskSchedulerAppCallbackImpl(this, 0),
           containerSignatureMatcher, host, port, trackingUrl, amrmClientAsync,
           appContext);
     }

http://git-wip-us.apache.org/repos/asf/tez/blob/a5dfca2b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/node/TestAMNodeTracker.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/node/TestAMNodeTracker.java b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/node/TestAMNodeTracker.java
index d907ea0..84d2e1f 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/node/TestAMNodeTracker.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/node/TestAMNodeTracker.java
@@ -93,12 +93,12 @@ public class TestAMNodeTracker {
     amNodeTracker.start();
 
     NodeId nodeId = NodeId.newInstance("host1", 2342);
-    amNodeTracker.nodeSeen(nodeId);
+    amNodeTracker.nodeSeen(nodeId, 0);
 
     NodeReport nodeReport = generateNodeReport(nodeId, NodeState.UNHEALTHY);
-    amNodeTracker.handle(new AMNodeEventStateChanged(nodeReport));
+    amNodeTracker.handle(new AMNodeEventStateChanged(nodeReport, 0));
     dispatcher.await();
-    assertEquals(AMNodeState.UNHEALTHY, amNodeTracker.get(nodeId).getState());
+    assertEquals(AMNodeState.UNHEALTHY, amNodeTracker.get(nodeId, 0).getState());
     amNodeTracker.stop();
   }
 
@@ -114,7 +114,7 @@ public class TestAMNodeTracker {
     NodeId nodeId = NodeId.newInstance("unknownhost", 2342);
 
     NodeReport nodeReport = generateNodeReport(nodeId, NodeState.UNHEALTHY);
-    amNodeTracker.handle(new AMNodeEventStateChanged(nodeReport));
+    amNodeTracker.handle(new AMNodeEventStateChanged(nodeReport, 0));
     dispatcher.await();
 
     amNodeTracker.stop();
@@ -142,27 +142,27 @@ public class TestAMNodeTracker {
     amNodeTracker.init(conf);
     amNodeTracker.start();
 
-    amNodeTracker.handle(new AMNodeEventNodeCountUpdated(1));
+    amNodeTracker.handle(new AMNodeEventNodeCountUpdated(1, 0));
     NodeId nodeId = NodeId.newInstance("host1", 1234);
-    amNodeTracker.nodeSeen(nodeId);
+    amNodeTracker.nodeSeen(nodeId, 0);
 
-    AMNodeImpl node = (AMNodeImpl) amNodeTracker.get(nodeId);
+    AMNodeImpl node = (AMNodeImpl) amNodeTracker.get(nodeId, 0);
 
     ContainerId cId1 = mock(ContainerId.class);
     ContainerId cId2 = mock(ContainerId.class);
 
-    amNodeTracker.handle(new AMNodeEventContainerAllocated(nodeId, cId1));
-    amNodeTracker.handle(new AMNodeEventContainerAllocated(nodeId, cId2));
+    amNodeTracker.handle(new AMNodeEventContainerAllocated(nodeId, 0, cId1));
+    amNodeTracker.handle(new AMNodeEventContainerAllocated(nodeId, 0, cId2));
 
     TezTaskAttemptID ta1 = mock(TezTaskAttemptID.class);
     TezTaskAttemptID ta2 = mock(TezTaskAttemptID.class);
 
-    amNodeTracker.handle(new AMNodeEventTaskAttemptEnded(nodeId, cId1, ta1, true));
+    amNodeTracker.handle(new AMNodeEventTaskAttemptEnded(nodeId, 0, cId1, ta1, true));
     dispatcher.await();
     assertEquals(1, node.numFailedTAs);
     assertEquals(AMNodeState.ACTIVE, node.getState());
 
-    amNodeTracker.handle(new AMNodeEventTaskAttemptEnded(nodeId, cId2, ta2, true));
+    amNodeTracker.handle(new AMNodeEventTaskAttemptEnded(nodeId, 0, cId2, ta2, true));
     dispatcher.await();
     assertEquals(2, node.numFailedTAs);
     assertEquals(1, handler.events.size());
@@ -187,44 +187,44 @@ public class TestAMNodeTracker {
     amNodeTracker.init(conf);
     amNodeTracker.start();
 
-    amNodeTracker.handle(new AMNodeEventNodeCountUpdated(4));
+    amNodeTracker.handle(new AMNodeEventNodeCountUpdated(4, 0));
     NodeId nodeId = NodeId.newInstance("host1", 1234);
     NodeId nodeId2 = NodeId.newInstance("host2", 1234);
     NodeId nodeId3 = NodeId.newInstance("host3", 1234);
     NodeId nodeId4 = NodeId.newInstance("host4", 1234);
-    amNodeTracker.nodeSeen(nodeId);
-    amNodeTracker.nodeSeen(nodeId2);
-    amNodeTracker.nodeSeen(nodeId3);
-    amNodeTracker.nodeSeen(nodeId4);
-    AMNodeImpl node = (AMNodeImpl) amNodeTracker.get(nodeId);
+    amNodeTracker.nodeSeen(nodeId, 0);
+    amNodeTracker.nodeSeen(nodeId2, 0);
+    amNodeTracker.nodeSeen(nodeId3, 0);
+    amNodeTracker.nodeSeen(nodeId4, 0);
+    AMNodeImpl node = (AMNodeImpl) amNodeTracker.get(nodeId, 0);
     
     ContainerId cId1 = mock(ContainerId.class);
     ContainerId cId2 = mock(ContainerId.class);
     ContainerId cId3 = mock(ContainerId.class);
     
-    amNodeTracker.handle(new AMNodeEventContainerAllocated(nodeId, cId1));
-    amNodeTracker.handle(new AMNodeEventContainerAllocated(nodeId, cId2));
-    amNodeTracker.handle(new AMNodeEventContainerAllocated(nodeId, cId3));
+    amNodeTracker.handle(new AMNodeEventContainerAllocated(nodeId, 0, cId1));
+    amNodeTracker.handle(new AMNodeEventContainerAllocated(nodeId, 0, cId2));
+    amNodeTracker.handle(new AMNodeEventContainerAllocated(nodeId, 0, cId3));
     assertEquals(3, node.containers.size());
     
     TezTaskAttemptID ta1 = mock(TezTaskAttemptID.class);
     TezTaskAttemptID ta2 = mock(TezTaskAttemptID.class);
     TezTaskAttemptID ta3 = mock(TezTaskAttemptID.class);
     
-    amNodeTracker.handle(new AMNodeEventTaskAttemptSucceeded(nodeId, cId1, ta1));
+    amNodeTracker.handle(new AMNodeEventTaskAttemptSucceeded(nodeId, 0, cId1, ta1));
     assertEquals(1, node.numSuccessfulTAs);
     
-    amNodeTracker.handle(new AMNodeEventTaskAttemptEnded(nodeId, cId2, ta2, true));
+    amNodeTracker.handle(new AMNodeEventTaskAttemptEnded(nodeId, 0, cId2, ta2, true));
     assertEquals(1, node.numSuccessfulTAs);
     assertEquals(1, node.numFailedTAs);
     assertEquals(AMNodeState.ACTIVE, node.getState());
     // duplicate should not affect anything
-    amNodeTracker.handle(new AMNodeEventTaskAttemptEnded(nodeId, cId2, ta2, true));
+    amNodeTracker.handle(new AMNodeEventTaskAttemptEnded(nodeId, 0, cId2, ta2, true));
     assertEquals(1, node.numSuccessfulTAs);
     assertEquals(1, node.numFailedTAs);
     assertEquals(AMNodeState.ACTIVE, node.getState());
     
-    amNodeTracker.handle(new AMNodeEventTaskAttemptEnded(nodeId, cId3, ta3, true));
+    amNodeTracker.handle(new AMNodeEventTaskAttemptEnded(nodeId, 0, cId3, ta3, true));
     dispatcher.await();
     assertEquals(1, node.numSuccessfulTAs);
     assertEquals(2, node.numFailedTAs);
@@ -246,20 +246,20 @@ public class TestAMNodeTracker {
     ContainerId cId5 = mock(ContainerId.class);
     TezTaskAttemptID ta4 = mock(TezTaskAttemptID.class);
     TezTaskAttemptID ta5 = mock(TezTaskAttemptID.class);
-    AMNodeImpl node2 = (AMNodeImpl) amNodeTracker.get(nodeId2);
-    amNodeTracker.handle(new AMNodeEventContainerAllocated(nodeId2, cId4));
-    amNodeTracker.handle(new AMNodeEventContainerAllocated(nodeId2, cId5));
+    AMNodeImpl node2 = (AMNodeImpl) amNodeTracker.get(nodeId2, 0);
+    amNodeTracker.handle(new AMNodeEventContainerAllocated(nodeId2, 0, cId4));
+    amNodeTracker.handle(new AMNodeEventContainerAllocated(nodeId2, 0, cId5));
     
-    amNodeTracker.handle(new AMNodeEventTaskAttemptEnded(nodeId2, cId4, ta4, true));
+    amNodeTracker.handle(new AMNodeEventTaskAttemptEnded(nodeId2, 0, cId4, ta4, true));
     assertEquals(1, node2.numFailedTAs);
     assertEquals(AMNodeState.ACTIVE, node2.getState());
     
     handler.events.clear();
-    amNodeTracker.handle(new AMNodeEventTaskAttemptEnded(nodeId2, cId5, ta5, true));
+    amNodeTracker.handle(new AMNodeEventTaskAttemptEnded(nodeId2, 0, cId5, ta5, true));
     dispatcher.await();
     assertEquals(2, node2.numFailedTAs);
     assertEquals(AMNodeState.FORCED_ACTIVE, node2.getState());
-    AMNodeImpl node3 = (AMNodeImpl) amNodeTracker.get(nodeId3);
+    AMNodeImpl node3 = (AMNodeImpl) amNodeTracker.get(nodeId3, 0);
     assertEquals(AMNodeState.FORCED_ACTIVE, node3.getState());
     assertEquals(5, handler.events.size());
 
@@ -286,7 +286,7 @@ public class TestAMNodeTracker {
     // Increase the number of nodes. BLACKLISTING should be re-enabled.
     // Node 1 and Node 2 should go into BLACKLISTED state
     handler.events.clear();
-    amNodeTracker.handle(new AMNodeEventNodeCountUpdated(8));
+    amNodeTracker.handle(new AMNodeEventNodeCountUpdated(8, 0));
     dispatcher.await();
     LOG.info(("Completed waiting for dispatcher to process all pending events"));
     assertEquals(AMNodeState.BLACKLISTED, node.getState());
@@ -336,4 +336,6 @@ public class TestAMNodeTracker {
     doReturn(healthReportTime).when(nodeReport).getLastHealthReportTime();
     return nodeReport;
   }
+
+  // TODO TEZ-2003. Add tests for multiple sources.
 }


[40/50] [abbrv] tez git commit: TEZ-2678. Fix comments from reviews - part 1. (sseth)

Posted by ss...@apache.org.
http://git-wip-us.apache.org/repos/asf/tez/blob/fee059b9/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 8e8224a..0a02f9e 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
@@ -18,6 +18,7 @@
 
 package org.apache.tez.dag.app.rm;
 
+import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.assertEquals;
 import static org.mockito.Matchers.any;
@@ -38,6 +39,8 @@ import java.util.concurrent.ExecutionException;
 import java.util.concurrent.atomic.AtomicBoolean;
 
 import org.apache.tez.common.TezUtils;
+import org.apache.tez.serviceplugins.api.TaskScheduler;
+import org.mockito.ArgumentCaptor;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -213,10 +216,10 @@ public class TestContainerReuse {
     taskSchedulerEventHandler.handleEvent(lrTa31);
 
     taskSchedulerEventHandler.handleEvent(
-      new AMSchedulerEventTAEnded(
-        ta11, containerHost1.getId(), TaskAttemptState.SUCCEEDED, null, 0));
+        new AMSchedulerEventTAEnded(
+            ta11, containerHost1.getId(), TaskAttemptState.SUCCEEDED, null, null, 0));
     drainableAppCallback.drain();
-    verify(taskScheduler).deallocateTask(eq(ta11), eq(true), eq((TaskAttemptEndReason)null));
+    verifyDeAllocateTask(taskScheduler, ta11, true, null, null);
     verify(taskSchedulerEventHandler, times(1)).taskAllocated(
       eq(0), eq(ta31), any(Object.class), eq(containerHost1));
     verify(rmClient, times(0)).releaseAssignedContainer(
@@ -226,7 +229,7 @@ public class TestContainerReuse {
 
     taskSchedulerEventHandler.handleEvent(
       new AMSchedulerEventTAEnded(ta21, containerHost2.getId(),
-        TaskAttemptState.SUCCEEDED, null, 0));
+        TaskAttemptState.SUCCEEDED, null, null, 0));
 
     long currentTs = System.currentTimeMillis();
     Throwable exception = null;
@@ -332,16 +335,17 @@ public class TestContainerReuse {
     TestTaskSchedulerHelpers.waitForDelayedDrainNotify(drainNotifier);
     drainableAppCallback.drain();
     verify(taskSchedulerEventHandler).taskAllocated(eq(0), eq(ta11), any(Object.class), eq(containerHost1));
-    verify(taskSchedulerEventHandler).taskAllocated(eq(0), eq(ta21), any(Object.class), eq(containerHost2));
+    verify(taskSchedulerEventHandler).taskAllocated(eq(0), eq(ta21), any(Object.class),
+        eq(containerHost2));
 
     // Adding the event later so that task1 assigned to containerHost1 is deterministic.
     taskSchedulerEventHandler.handleEvent(lrTa31);
 
     taskSchedulerEventHandler.handleEvent(
         new AMSchedulerEventTAEnded(ta21, containerHost2.getId(),
-            TaskAttemptState.SUCCEEDED, null, 0));
+            TaskAttemptState.SUCCEEDED, null, null, 0));
     drainableAppCallback.drain();
-    verify(taskScheduler).deallocateTask(eq(ta21), eq(true), eq((TaskAttemptEndReason)null));
+    verifyDeAllocateTask(taskScheduler, ta21, true, null, null);
     verify(taskSchedulerEventHandler, times(0)).taskAllocated(
         eq(0), eq(ta31), any(Object.class), eq(containerHost2));
     verify(rmClient, times(1)).releaseAssignedContainer(
@@ -431,12 +435,15 @@ public class TestContainerReuse {
     taskScheduler.onContainersAllocated(Collections.singletonList(container1));
     TestTaskSchedulerHelpers.waitForDelayedDrainNotify(drainNotifier);
     drainableAppCallback.drain();
-    verify(taskSchedulerEventHandler).taskAllocated(eq(0), eq(ta11), any(Object.class), eq(container1));
+    verify(taskSchedulerEventHandler).taskAllocated(eq(0), 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, null, 0));
+    taskSchedulerEventHandler.handleEvent(
+        new AMSchedulerEventTAEnded(ta11, container1.getId(), TaskAttemptState.SUCCEEDED, null,
+            null, 0));
     drainableAppCallback.drain();
-    verify(taskScheduler).deallocateTask(eq(ta11), eq(true), eq((TaskAttemptEndReason)null));
+    verifyDeAllocateTask(taskScheduler, ta11, true, null, null);
     verify(taskSchedulerEventHandler).taskAllocated(eq(0), eq(ta12), any(Object.class), eq(container1));
     verify(rmClient, times(0)).releaseAssignedContainer(eq(container1.getId()));
     eventHandler.verifyNoInvocations(AMContainerEventStopRequest.class);
@@ -444,19 +451,25 @@ public class TestContainerReuse {
 
     // Task assigned to container completed successfully.
     // Verify reuse across hosts.
-    taskSchedulerEventHandler.handleEvent(new AMSchedulerEventTAEnded(ta12, container1.getId(), TaskAttemptState.SUCCEEDED, null, 0));
+    taskSchedulerEventHandler.handleEvent(
+        new AMSchedulerEventTAEnded(ta12, container1.getId(), TaskAttemptState.SUCCEEDED, null,
+            null, 0));
     drainableAppCallback.drain();
-    verify(taskScheduler).deallocateTask(eq(ta12), eq(true), eq((TaskAttemptEndReason)null));
-    verify(taskSchedulerEventHandler).taskAllocated(eq(0), eq(ta13), any(Object.class), eq(container1));
+    verifyDeAllocateTask(taskScheduler, ta12, true, null, null);
+    verify(taskSchedulerEventHandler).taskAllocated(eq(0), 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, null, 0));
+    taskSchedulerEventHandler.handleEvent(
+        new AMSchedulerEventTAEnded(ta13, container1.getId(), TaskAttemptState.FAILED, null,
+            "TIMEOUT", 0));
     drainableAppCallback.drain();
-    verify(taskSchedulerEventHandler, times(0)).taskAllocated(eq(0), eq(ta14), any(Object.class), eq(container1));
-    verify(taskScheduler).deallocateTask(eq(ta13), eq(false), eq((TaskAttemptEndReason)null));
+    verify(taskSchedulerEventHandler, times(0)).taskAllocated(eq(0), eq(ta14), any(Object.class),
+        eq(container1));
+    verifyDeAllocateTask(taskScheduler, ta13, false, null, "TIMEOUT");
     verify(rmClient).releaseAssignedContainer(eq(container1.getId()));
     eventHandler.verifyInvocation(AMContainerEventStopRequest.class);
     eventHandler.reset();
@@ -468,12 +481,15 @@ public class TestContainerReuse {
     taskScheduler.onContainersAllocated(Collections.singletonList(container2));
     TestTaskSchedulerHelpers.waitForDelayedDrainNotify(drainNotifier);
     drainableAppCallback.drain();
-    verify(taskSchedulerEventHandler).taskAllocated(eq(0), eq(ta14), any(Object.class), eq(container2));
+    verify(taskSchedulerEventHandler).taskAllocated(eq(0), 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, null, 0));
+    taskSchedulerEventHandler.handleEvent(
+        new AMSchedulerEventTAEnded(ta14, container2.getId(), TaskAttemptState.SUCCEEDED, null,
+            null, 0));
     drainableAppCallback.drain();
-    verify(taskScheduler).deallocateTask(eq(ta14), eq(true), eq((TaskAttemptEndReason)null));
+    verifyDeAllocateTask(taskScheduler, ta14, true, null, null);
     verify(rmClient).releaseAssignedContainer(eq(container2.getId()));
     eventHandler.verifyInvocation(AMContainerEventStopRequest.class);
     eventHandler.reset();
@@ -570,13 +586,15 @@ public class TestContainerReuse {
     taskScheduler.onContainersAllocated(Collections.singletonList(container1));
     TestTaskSchedulerHelpers.waitForDelayedDrainNotify(drainNotifier);
     drainableAppCallback.drain();
-    verify(taskSchedulerEventHandler).taskAllocated(eq(0), eq(ta11), any(Object.class), eq(container1));
+    verify(taskSchedulerEventHandler).taskAllocated(eq(0), eq(ta11), any(Object.class),
+        eq(container1));
 
     // First task had profiling on. This container can not be reused further.
     taskSchedulerEventHandler.handleEvent(
-        new AMSchedulerEventTAEnded(ta11, container1.getId(), TaskAttemptState.SUCCEEDED, null, 0));
+        new AMSchedulerEventTAEnded(ta11, container1.getId(), TaskAttemptState.SUCCEEDED, null,
+            null, 0));
     drainableAppCallback.drain();
-    verify(taskScheduler).deallocateTask(eq(ta11), eq(true), eq((TaskAttemptEndReason)null));
+    verifyDeAllocateTask(taskScheduler, ta11, true, null, null);
     verify(taskSchedulerEventHandler, times(0)).taskAllocated(eq(0), eq(ta12), any(Object.class),
         eq(container1));
     verify(rmClient, times(1)).releaseAssignedContainer(eq(container1.getId()));
@@ -620,9 +638,11 @@ 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, null, 0));
+    taskSchedulerEventHandler.handleEvent(
+        new AMSchedulerEventTAEnded(ta13, container2.getId(), TaskAttemptState.SUCCEEDED, null,
+            null, 0));
     drainableAppCallback.drain();
-    verify(taskScheduler).deallocateTask(eq(ta13), eq(true), eq((TaskAttemptEndReason)null));
+    verifyDeAllocateTask(taskScheduler, ta13, true, null, null);
     verify(taskSchedulerEventHandler, times(0)).taskAllocated(eq(0), eq(ta14), any(Object.class),
         eq(container2));
     verify(rmClient, times(1)).releaseAssignedContainer(eq(container2.getId()));
@@ -662,12 +682,15 @@ public class TestContainerReuse {
     taskScheduler.onContainersAllocated(Collections.singletonList(container3));
     TestTaskSchedulerHelpers.waitForDelayedDrainNotify(drainNotifier);
     drainableAppCallback.drain();
-    verify(taskSchedulerEventHandler).taskAllocated(eq(0), eq(ta15), any(Object.class), eq(container3));
+    verify(taskSchedulerEventHandler).taskAllocated(eq(0), 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, null, 0));
+    taskSchedulerEventHandler.handleEvent(
+        new AMSchedulerEventTAEnded(ta15, container3.getId(), TaskAttemptState.SUCCEEDED, null,
+            null, 0));
     drainableAppCallback.drain();
-    verify(taskScheduler).deallocateTask(eq(ta15), eq(true), eq((TaskAttemptEndReason)null));
+    verifyDeAllocateTask(taskScheduler, ta15, true, null, null);
     verify(taskSchedulerEventHandler).taskAllocated(eq(0), eq(ta16), any(Object.class), eq(container3));
     eventHandler.reset();
 
@@ -769,10 +792,10 @@ public class TestContainerReuse {
     // Container should not be immediately assigned to task 2
     // until delay expires.
     taskSchedulerEventHandler.handleEvent(
-      new AMSchedulerEventTAEnded(ta11, container1.getId(),
-        TaskAttemptState.SUCCEEDED, null, 0));
+        new AMSchedulerEventTAEnded(ta11, container1.getId(),
+            TaskAttemptState.SUCCEEDED, null, null, 0));
     drainableAppCallback.drain();
-    verify(taskScheduler).deallocateTask(eq(ta11), eq(true), eq((TaskAttemptEndReason)null));
+    verifyDeAllocateTask(taskScheduler, ta11, true, null, null);
     verify(taskSchedulerEventHandler, times(0)).taskAllocated(
         eq(0), eq(ta12), any(Object.class), eq(container1));
     verify(rmClient, times(0)).releaseAssignedContainer(eq(container1.getId()));
@@ -787,7 +810,7 @@ public class TestContainerReuse {
     // TA12 completed.
     taskSchedulerEventHandler.handleEvent(
       new AMSchedulerEventTAEnded(ta12, container1.getId(),
-        TaskAttemptState.SUCCEEDED, null, 0));
+        TaskAttemptState.SUCCEEDED, null, null, 0));
     drainableAppCallback.drain();
     LOG.info("Sleeping to ensure that the scheduling loop runs");
     Thread.sleep(3000l);
@@ -897,9 +920,9 @@ public class TestContainerReuse {
     // Container should  be assigned to task21.
     taskSchedulerEventHandler.handleEvent(
         new AMSchedulerEventTAEnded(ta11, container1.getId(),
-            TaskAttemptState.SUCCEEDED, null, 0));
+            TaskAttemptState.SUCCEEDED, null, null, 0));
     drainableAppCallback.drain();
-    verify(taskScheduler).deallocateTask(eq(ta11), eq(true), eq((TaskAttemptEndReason)null));
+    verifyDeAllocateTask(taskScheduler, ta11, true, null, null);
     verify(taskSchedulerEventHandler).taskAllocated(
         eq(0), eq(ta21), any(Object.class), eq(container1));
     verify(rmClient, times(0)).releaseAssignedContainer(eq(container1.getId()));
@@ -907,7 +930,7 @@ public class TestContainerReuse {
     // Task 2 completes.
     taskSchedulerEventHandler.handleEvent(
         new AMSchedulerEventTAEnded(ta21, container1.getId(),
-            TaskAttemptState.SUCCEEDED, null, 0));
+            TaskAttemptState.SUCCEEDED, null, null, 0));
     verify(rmClient, times(0)).releaseAssignedContainer(eq(container1.getId()));
 
     LOG.info("Sleeping to ensure that the scheduling loop runs");
@@ -1008,9 +1031,11 @@ 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, null, 0));
+    taskSchedulerEventHandler.handleEvent(
+        new AMSchedulerEventTAEnded(ta111, container1.getId(), TaskAttemptState.SUCCEEDED, null,
+            null, 0));
     drainableAppCallback.drain();
-    verify(taskScheduler).deallocateTask(eq(ta111), eq(true), eq((TaskAttemptEndReason)null));
+    verifyDeAllocateTask(taskScheduler, ta111, true, null, null);
     verify(taskSchedulerEventHandler).taskAllocated(eq(0), eq(ta112), any(Object.class), eq(container1));
     verify(rmClient, times(0)).releaseAssignedContainer(eq(container1.getId()));
     eventHandler.verifyNoInvocations(AMContainerEventStopRequest.class);
@@ -1020,9 +1045,11 @@ public class TestContainerReuse {
 
     // Task assigned to container completed successfully.
     // Verify reuse across hosts.
-    taskSchedulerEventHandler.handleEvent(new AMSchedulerEventTAEnded(ta112, container1.getId(), TaskAttemptState.SUCCEEDED, null, 0));
+    taskSchedulerEventHandler.handleEvent(
+        new AMSchedulerEventTAEnded(ta112, container1.getId(), TaskAttemptState.SUCCEEDED, null,
+            null, 0));
     drainableAppCallback.drain();
-    verify(taskScheduler).deallocateTask(eq(ta112), eq(true), eq((TaskAttemptEndReason)null));
+    verifyDeAllocateTask(taskScheduler, ta112, true, null, null);
     verify(rmClient, times(0)).releaseAssignedContainer(eq(container1.getId()));
     eventHandler.verifyNoInvocations(AMContainerEventStopRequest.class);
     eventHandler.reset();
@@ -1054,16 +1081,19 @@ public class TestContainerReuse {
     // TODO This is terrible, need a better way to ensure the scheduling loop has run
     LOG.info("Sleeping to ensure that the scheduling loop runs");
     Thread.sleep(6000l);
-    verify(taskSchedulerEventHandler).taskAllocated(eq(0), eq(ta211), any(Object.class), eq(container1));
+    verify(taskSchedulerEventHandler).taskAllocated(eq(0), eq(ta211), any(Object.class),
+        eq(container1));
     verify(rmClient, times(0)).releaseAssignedContainer(eq(container1.getId()));
     eventHandler.verifyNoInvocations(AMContainerEventStopRequest.class);
     assignEvent = (AMContainerEventAssignTA) eventHandler.verifyInvocation(AMContainerEventAssignTA.class);
     assertEquals(2, assignEvent.getRemoteTaskLocalResources().size());
     eventHandler.reset();
 
-    taskSchedulerEventHandler.handleEvent(new AMSchedulerEventTAEnded(ta211, container1.getId(), TaskAttemptState.SUCCEEDED, null, 0));
+    taskSchedulerEventHandler.handleEvent(
+        new AMSchedulerEventTAEnded(ta211, container1.getId(), TaskAttemptState.SUCCEEDED, null,
+            null, 0));
     drainableAppCallback.drain();
-    verify(taskScheduler).deallocateTask(eq(ta211), eq(true), eq((TaskAttemptEndReason)null));
+    verifyDeAllocateTask(taskScheduler, ta211, true, null, null);
     verify(taskSchedulerEventHandler).taskAllocated(eq(0), eq(ta212), any(Object.class), eq(container1));
     verify(rmClient, times(0)).releaseAssignedContainer(eq(container1.getId()));
     eventHandler.verifyNoInvocations(AMContainerEventStopRequest.class);
@@ -1174,9 +1204,11 @@ 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, null, 0));
+    taskSchedulerEventHandler.handleEvent(
+        new AMSchedulerEventTAEnded(ta111, container1.getId(), TaskAttemptState.SUCCEEDED, null,
+            null, 0));
     drainableAppCallback.drain();
-    verify(taskScheduler).deallocateTask(eq(ta111), eq(true), eq((TaskAttemptEndReason)null));
+    verifyDeAllocateTask(taskScheduler, ta111, true, null, null);
     verify(taskSchedulerEventHandler).taskAllocated(eq(0), eq(ta112), any(Object.class), eq(container1));
     verify(rmClient, times(0)).releaseAssignedContainer(eq(container1.getId()));
     eventHandler.verifyNoInvocations(AMContainerEventStopRequest.class);
@@ -1186,9 +1218,9 @@ public class TestContainerReuse {
 
     // Task assigned to container completed successfully.
     // Verify reuse across hosts.
-    taskSchedulerEventHandler.handleEvent(new AMSchedulerEventTAEnded(ta112, container1.getId(), TaskAttemptState.SUCCEEDED, null, 0));
+    taskSchedulerEventHandler.handleEvent(new AMSchedulerEventTAEnded(ta112, container1.getId(), TaskAttemptState.SUCCEEDED, null, null, 0));
     drainableAppCallback.drain();
-    verify(taskScheduler).deallocateTask(eq(ta112), eq(true), eq((TaskAttemptEndReason)null));
+    verifyDeAllocateTask(taskScheduler, ta112, true, null, null);
     verify(rmClient, times(0)).releaseAssignedContainer(eq(container1.getId()));
     eventHandler.verifyNoInvocations(AMContainerEventStopRequest.class);
     eventHandler.reset();
@@ -1301,6 +1333,7 @@ public class TestContainerReuse {
   }
 
   private Container createContainer(int id, String host, Resource resource, Priority priority) {
+    @SuppressWarnings("deprecation")
     ContainerId containerID = ContainerId.newInstance(
         ApplicationAttemptId.newInstance(ApplicationId.newInstance(1, 1), 1),
         id);
@@ -1368,4 +1401,17 @@ public class TestContainerReuse {
       return this.dagID;
     }
   }
+
+  private void verifyDeAllocateTask(TaskScheduler taskScheduler, Object ta, boolean taskSucceeded,
+                                    TaskAttemptEndReason endReason, String diagContains) {
+    ArgumentCaptor<String> argumentCaptor = ArgumentCaptor.forClass(String.class);
+    verify(taskScheduler)
+        .deallocateTask(eq(ta), eq(taskSucceeded), eq(endReason), argumentCaptor.capture());
+    assertEquals(1, argumentCaptor.getAllValues().size());
+    if (diagContains == null) {
+      assertNull(argumentCaptor.getValue());
+    } else {
+      assertTrue(argumentCaptor.getValue().contains(diagContains));
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/fee059b9/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 c637f5f..3b2de34 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
@@ -93,7 +93,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, null);
+    taskSchedulerService.deallocateTask(task, false, null, null);
     // start the RequestHandler, DeallocateTaskRequest has higher priority, so will be processed first
     taskSchedulerService.startRequestHandlerThread();
 
@@ -128,7 +128,7 @@ public class TestLocalTaskSchedulerService {
 
     MockAsyncDelegateRequestHandler requestHandler = taskSchedulerService.getRequestHandler();
     requestHandler.drainRequest(1);
-    taskSchedulerService.deallocateTask(task, false, null);
+    taskSchedulerService.deallocateTask(task, false, null, null);
     requestHandler.drainRequest(2);
     assertEquals(1, requestHandler.deallocateCount);
     assertEquals(1, requestHandler.allocateCount);

http://git-wip-us.apache.org/repos/asf/tez/blob/fee059b9/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 6af9815..d956ff9 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
@@ -193,7 +193,7 @@ public class TestTaskScheduler {
                            addContainerRequest((CookieContainerRequest) any());
 
     // returned from task requests before allocation happens
-    assertFalse(scheduler.deallocateTask(mockTask1, true, null));
+    assertFalse(scheduler.deallocateTask(mockTask1, true, null, null));
     verify(mockApp, times(0)).containerBeingReleased(any(ContainerId.class));
     verify(mockRMClient, times(1)).
                         removeContainerRequest((CookieContainerRequest) any());
@@ -201,7 +201,7 @@ public class TestTaskScheduler {
                                  releaseAssignedContainer((ContainerId) any());
 
     // deallocating unknown task
-    assertFalse(scheduler.deallocateTask(mockTask1, true, null));
+    assertFalse(scheduler.deallocateTask(mockTask1, true, null, null));
     verify(mockApp, times(0)).containerBeingReleased(any(ContainerId.class));
     verify(mockRMClient, times(1)).
                         removeContainerRequest((CookieContainerRequest) any());
@@ -346,7 +346,7 @@ public class TestTaskScheduler {
     verify(mockRMClient).releaseAssignedContainer(mockCId4);
 
     // deallocate allocated task
-    assertTrue(scheduler.deallocateTask(mockTask1, true, null));
+    assertTrue(scheduler.deallocateTask(mockTask1, true, null, null));
     drainableAppCallback.drain();
     verify(mockApp).containerBeingReleased(mockCId1);
     verify(mockRMClient).releaseAssignedContainer(mockCId1);
@@ -466,7 +466,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, null));
+    assertTrue(scheduler.deallocateTask(mockTask4, true, null, null));
     drainableAppCallback.drain();
     verify(mockApp).containerBeingReleased(mockCId6);
     verify(mockRMClient).releaseAssignedContainer(mockCId6);
@@ -496,7 +496,7 @@ public class TestTaskScheduler {
         removeContainerRequest((CookieContainerRequest) any());
     verify(mockRMClient, times(8)).addContainerRequest(
         (CookieContainerRequest) any());
-    assertFalse(scheduler.deallocateTask(mockTask1, true, null));
+    assertFalse(scheduler.deallocateTask(mockTask1, true, null, null));
 
     List<NodeReport> mockUpdatedNodes = mock(List.class);
     scheduler.onNodesUpdated(mockUpdatedNodes);
@@ -760,7 +760,7 @@ public class TestTaskScheduler {
     verify(mockRMClient).releaseAssignedContainer(mockCId4);
 
     // deallocate allocated task
-    assertTrue(scheduler.deallocateTask(mockTask1, true, null));
+    assertTrue(scheduler.deallocateTask(mockTask1, true, null, null));
     drainableAppCallback.drain();
     verify(mockApp).containerBeingReleased(mockCId1);
     verify(mockRMClient).releaseAssignedContainer(mockCId1);
@@ -890,7 +890,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, null));
+    assertTrue(scheduler.deallocateTask(mockTask4, true, null, null));
     drainableAppCallback.drain();
     verify(mockApp).containerBeingReleased(mockCId6);
     verify(mockRMClient).releaseAssignedContainer(mockCId6);
@@ -979,8 +979,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, null));
-    assertTrue(scheduler.deallocateTask(mockTask6, true, null));
+    assertTrue(scheduler.deallocateTask(mockTask5, true, null, null));
+    assertTrue(scheduler.deallocateTask(mockTask6, true, null, null));
     drainableAppCallback.drain();
     verify(mockApp).containerBeingReleased(mockCId7);
     verify(mockApp).containerBeingReleased(mockCId8);

http://git-wip-us.apache.org/repos/asf/tez/blob/fee059b9/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 3e68a4c..1550085 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
@@ -107,7 +107,7 @@ public class TestTaskSchedulerEventHandler {
   
   class MockTaskSchedulerEventHandler extends TaskSchedulerEventHandler {
 
-    AtomicBoolean notify = new AtomicBoolean(false);
+    final AtomicBoolean notify = new AtomicBoolean(false);
     
     public MockTaskSchedulerEventHandler(AppContext appContext,
         DAGClientServer clientService, EventHandler eventHandler,
@@ -120,7 +120,7 @@ public class TestTaskSchedulerEventHandler {
     protected void instantiateScheduelrs(String host, int port, String trackingUrl,
                                          AppContext appContext) {
       taskSchedulers[0] = mockTaskScheduler;
-      taskSchedulerServiceWrappers[0] = new ServicePluginLifecycleAbstractService(taskSchedulers[0]);
+      taskSchedulerServiceWrappers[0] = new ServicePluginLifecycleAbstractService<>(taskSchedulers[0]);
     }
     
     @Override
@@ -154,7 +154,6 @@ public class TestTaskSchedulerEventHandler {
     mockWebUIService = mock(WebUIService.class);
     when(mockAppContext.getAllContainers()).thenReturn(mockAMContainerMap);
     when(mockClientService.getBindAddress()).thenReturn(new InetSocketAddress(10000));
-    Configuration conf = new Configuration(false);
     schedulerHandler = new MockTaskSchedulerEventHandler(
         mockAppContext, mockClientService, mockEventHandler, mockSigMatcher, mockWebUIService);
   }
@@ -412,9 +411,8 @@ public class TestTaskSchedulerEventHandler {
   @Test(timeout = 5000)
   public void testNoSchedulerSpecified() throws IOException {
     try {
-      TSEHForMultipleSchedulersTest tseh =
-          new TSEHForMultipleSchedulersTest(mockAppContext, mockClientService, mockEventHandler,
-              mockSigMatcher, mockWebUIService, null, false);
+      new TSEHForMultipleSchedulersTest(mockAppContext, mockClientService, mockEventHandler,
+          mockSigMatcher, mockWebUIService, null, false);
       fail("Expecting an IllegalStateException with no schedulers specified");
     } catch (IllegalArgumentException e) {
     }
@@ -686,7 +684,8 @@ public class TestTaskSchedulerEventHandler {
 
     @Override
     public boolean deallocateTask(Object task, boolean taskSucceeded,
-                                  TaskAttemptEndReason endReason) {
+                                  TaskAttemptEndReason endReason,
+                                  String diagnostics) {
       return false;
     }
 

http://git-wip-us.apache.org/repos/asf/tez/blob/fee059b9/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 f9952d8..13fa4c5 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
@@ -127,7 +127,8 @@ 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), eq(0));
+    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());
@@ -137,14 +138,14 @@ public class TestAMContainer {
     wc.verifyState(AMContainerState.IDLE);
     wc.verifyNoOutgoingEvents();
     assertNull(wc.amContainer.getCurrentTaskAttempt());
-    verify(wc.tal).unregisterTaskAttempt(wc.taskAttemptID, 0, TaskAttemptEndReason.OTHER);
+    verifyUnregisterTaskAttempt(wc.tal, wc.taskAttemptID, 0, TaskAttemptEndReason.OTHER, null);
 
     // Container completed
     wc.containerCompleted();
     wc.verifyHistoryStopEvent();
     wc.verifyState(AMContainerState.COMPLETED);
     wc.verifyNoOutgoingEvents();
-    verify(wc.tal).unregisterRunningContainer(wc.containerID, 0, ContainerEndReason.COMPLETED);
+    verifyUnregisterRunningContainer(wc.tal, wc.containerID, 0, ContainerEndReason.COMPLETED, null);
     verify(wc.chh).unregister(wc.containerID);
 
     assertEquals(1, wc.amContainer.getAllTaskAttempts().size());
@@ -178,21 +179,23 @@ 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), eq(0));
+    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(wc.taskAttemptID,
+        argumentCaptor.getAllValues().get(0).getTask().getTaskAttemptID());
 
     wc.taskAttemptSucceeded(wc.taskAttemptID);
     wc.verifyState(AMContainerState.IDLE);
     wc.verifyNoOutgoingEvents();
     assertNull(wc.amContainer.getCurrentTaskAttempt());
-    verify(wc.tal).unregisterTaskAttempt(wc.taskAttemptID, 0, TaskAttemptEndReason.OTHER);
+    verifyUnregisterTaskAttempt(wc.tal, wc.taskAttemptID, 0, TaskAttemptEndReason.OTHER, null);
 
     wc.containerCompleted();
     wc.verifyHistoryStopEvent();
     wc.verifyState(AMContainerState.COMPLETED);
     wc.verifyNoOutgoingEvents();
-    verify(wc.tal).unregisterRunningContainer(wc.containerID, 0, ContainerEndReason.COMPLETED);
+    verifyUnregisterRunningContainer(wc.tal, wc.containerID, 0, ContainerEndReason.COMPLETED, null);
     verify(wc.chh).unregister(wc.containerID);
 
     assertEquals(1, wc.amContainer.getAllTaskAttempts().size());
@@ -228,22 +231,25 @@ 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), eq(0));
+    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(wc.taskAttemptID,
+        argumentCaptor.getAllValues().get(0).getTask().getTaskAttemptID());
 
     // Attempt succeeded
     wc.taskAttemptSucceeded(wc.taskAttemptID);
     wc.verifyState(AMContainerState.IDLE);
     wc.verifyNoOutgoingEvents();
     assertNull(wc.amContainer.getCurrentTaskAttempt());
-    verify(wc.tal).unregisterTaskAttempt(wc.taskAttemptID, 0, TaskAttemptEndReason.OTHER);
+    verifyUnregisterTaskAttempt(wc.tal, wc.taskAttemptID, 0, TaskAttemptEndReason.OTHER, null);
 
     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), eq(0));
+    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());
 
@@ -252,14 +258,14 @@ public class TestAMContainer {
     wc.verifyState(AMContainerState.IDLE);
     wc.verifyNoOutgoingEvents();
     assertNull(wc.amContainer.getCurrentTaskAttempt());
-    verify(wc.tal).unregisterTaskAttempt(taId2, 0, TaskAttemptEndReason.OTHER);
+    verifyUnregisterTaskAttempt(wc.tal, wc.taskAttemptID, 0, TaskAttemptEndReason.OTHER, null);
 
     // Container completed
     wc.containerCompleted();
     wc.verifyHistoryStopEvent();
     wc.verifyState(AMContainerState.COMPLETED);
     wc.verifyNoOutgoingEvents();
-    verify(wc.tal).unregisterRunningContainer(wc.containerID, 0, ContainerEndReason.COMPLETED);
+    verifyUnregisterRunningContainer(wc.tal, wc.containerID, 0, ContainerEndReason.COMPLETED, null);
     verify(wc.chh).unregister(wc.containerID);
 
     assertEquals(2, wc.amContainer.getAllTaskAttempts().size());
@@ -292,7 +298,8 @@ public class TestAMContainer {
     wc.verifyHistoryStopEvent();
     wc.verifyState(AMContainerState.COMPLETED);
     wc.verifyNoOutgoingEvents();
-    verify(wc.tal).unregisterRunningContainer(wc.containerID, 0, ContainerEndReason.OTHER);
+    verifyUnregisterRunningContainer(wc.tal, wc.containerID, 0, ContainerEndReason.OTHER,
+        "received a STOP_REQUEST");
     verify(wc.chh).unregister(wc.containerID);
 
     assertNull(wc.amContainer.getCurrentTaskAttempt());
@@ -329,7 +336,8 @@ public class TestAMContainer {
     wc.verifyHistoryStopEvent();
     wc.verifyState(AMContainerState.COMPLETED);
     wc.verifyNoOutgoingEvents();
-    verify(wc.tal).unregisterRunningContainer(wc.containerID, 0, ContainerEndReason.OTHER);
+    verifyUnregisterRunningContainer(wc.tal, wc.containerID, 0, ContainerEndReason.OTHER,
+        "received a STOP_REQUEST");
     verify(wc.chh).unregister(wc.containerID);
 
     assertNull(wc.amContainer.getCurrentTaskAttempt());
@@ -352,7 +360,8 @@ public class TestAMContainer {
     wc.assignTaskAttempt(taID2);
 
     wc.verifyState(AMContainerState.STOP_REQUESTED);
-    verify(wc.tal).unregisterRunningContainer(wc.containerID, 0, ContainerEndReason.FRAMEWORK_ERROR);
+    verifyUnregisterRunningContainer(wc.tal, wc.containerID, 0, ContainerEndReason.FRAMEWORK_ERROR,
+        "Multiple simultaneous taskAttempt");
     verify(wc.chh).unregister(wc.containerID);
     // 1 for NM stop request. 2 TERMINATING to TaskAttempt.
     outgoingEvents = wc.verifyCountAndGetOutgoingEvents(3);
@@ -390,7 +399,8 @@ public class TestAMContainer {
     wc.assignTaskAttempt(taID2);
 
     wc.verifyState(AMContainerState.STOP_REQUESTED);
-    verify(wc.tal).unregisterRunningContainer(wc.containerID, 0, ContainerEndReason.FRAMEWORK_ERROR);
+    verifyUnregisterRunningContainer(wc.tal, wc.containerID, 0, ContainerEndReason.FRAMEWORK_ERROR,
+        "Multiple simultaneous taskAttempt");
     verify(wc.chh).unregister(wc.containerID);
     // 1 for NM stop request. 2 TERMINATING to TaskAttempt.
     outgoingEvents = wc.verifyCountAndGetOutgoingEvents(3);
@@ -426,7 +436,8 @@ public class TestAMContainer {
 
     wc.containerTimedOut();
     wc.verifyState(AMContainerState.STOP_REQUESTED);
-    verify(wc.tal).unregisterRunningContainer(wc.containerID, 0, ContainerEndReason.OTHER);
+    verifyUnregisterRunningContainer(wc.tal, wc.containerID, 0, ContainerEndReason.OTHER,
+        "timed out");
     verify(wc.chh).unregister(wc.containerID);
     // 1 to TA, 1 for RM de-allocate.
     outgoingEvents = wc.verifyCountAndGetOutgoingEvents(2);
@@ -460,7 +471,8 @@ public class TestAMContainer {
 
     wc.stopRequest();
     wc.verifyState(AMContainerState.STOP_REQUESTED);
-    verify(wc.tal).unregisterRunningContainer(wc.containerID, 0, ContainerEndReason.OTHER);
+    verifyUnregisterRunningContainer(wc.tal, wc.containerID, 0, ContainerEndReason.OTHER,
+        "received a STOP_REQUEST");
     verify(wc.chh).unregister(wc.containerID);
     // 1 to TA, 1 for RM de-allocate.
     outgoingEvents = wc.verifyCountAndGetOutgoingEvents(2);
@@ -494,7 +506,8 @@ public class TestAMContainer {
     wc.launchFailed();
     wc.verifyState(AMContainerState.STOPPING);
     verify(wc.tal).registerRunningContainer(wc.containerID, 0);
-    verify(wc.tal).unregisterRunningContainer(wc.containerID, 0, ContainerEndReason.LAUNCH_FAILED);
+    verifyUnregisterRunningContainer(wc.tal, wc.containerID, 0, ContainerEndReason.LAUNCH_FAILED,
+        "launchFailed");
 
     outgoingEvents = wc.verifyCountAndGetOutgoingEvents(2);
     verifyUnOrderedOutgoingEventTypes(outgoingEvents,
@@ -544,7 +557,7 @@ public class TestAMContainer {
     wc.containerCompleted();
     wc.verifyState(AMContainerState.COMPLETED);
     verify(wc.tal).registerRunningContainer(wc.containerID, 0);
-    verify(wc.tal).unregisterRunningContainer(wc.containerID, 0, ContainerEndReason.COMPLETED);
+    verifyUnregisterRunningContainer(wc.tal, wc.containerID, 0, ContainerEndReason.COMPLETED, null);
 
     outgoingEvents = wc.verifyCountAndGetOutgoingEvents(1);
     verifyUnOrderedOutgoingEventTypes(outgoingEvents,
@@ -571,10 +584,10 @@ public class TestAMContainer {
 
     wc.assignTaskAttempt(wc.taskAttemptID);
 
-    wc.containerCompleted(ContainerExitStatus.DISKS_FAILED, TaskAttemptTerminationCause.NODE_DISK_ERROR);
+    wc.containerCompleted(ContainerExitStatus.DISKS_FAILED, TaskAttemptTerminationCause.NODE_DISK_ERROR, "DiskFailed");
     wc.verifyState(AMContainerState.COMPLETED);
     verify(wc.tal).registerRunningContainer(wc.containerID, 0);
-    verify(wc.tal).unregisterRunningContainer(wc.containerID, 0, ContainerEndReason.OTHER);
+    verifyUnregisterRunningContainer(wc.tal, wc.containerID, 0, ContainerEndReason.OTHER, "DiskFailed");
 
     outgoingEvents = wc.verifyCountAndGetOutgoingEvents(1);
     verifyUnOrderedOutgoingEventTypes(outgoingEvents,
@@ -602,10 +615,11 @@ public class TestAMContainer {
 
     wc.assignTaskAttempt(wc.taskAttemptID);
 
-    wc.containerCompleted(ContainerExitStatus.ABORTED, TaskAttemptTerminationCause.NODE_FAILED);
+    wc.containerCompleted(ContainerExitStatus.ABORTED, TaskAttemptTerminationCause.NODE_FAILED, "NodeFailed");
     wc.verifyState(AMContainerState.COMPLETED);
     verify(wc.tal).registerRunningContainer(wc.containerID, 0);
-    verify(wc.tal).unregisterRunningContainer(wc.containerID, 0, ContainerEndReason.NODE_FAILED);
+    verifyUnregisterRunningContainer(wc.tal, wc.containerID, 0, ContainerEndReason.NODE_FAILED,
+        "NodeFailed");
 
     outgoingEvents = wc.verifyCountAndGetOutgoingEvents(1);
     verifyUnOrderedOutgoingEventTypes(outgoingEvents,
@@ -636,7 +650,7 @@ public class TestAMContainer {
     wc.containerCompleted();
     wc.verifyState(AMContainerState.COMPLETED);
     verify(wc.tal).registerRunningContainer(wc.containerID, 0);
-    verify(wc.tal).unregisterRunningContainer(wc.containerID, 0, ContainerEndReason.COMPLETED);
+    verifyUnregisterRunningContainer(wc.tal, wc.containerID, 0, ContainerEndReason.COMPLETED, null);
     verify(wc.chh).register(wc.containerID);
     verify(wc.chh).unregister(wc.containerID);
 
@@ -665,7 +679,7 @@ public class TestAMContainer {
     wc.containerCompleted();
     wc.verifyState(AMContainerState.COMPLETED);
     verify(wc.tal).registerRunningContainer(wc.containerID, 0);
-    verify(wc.tal).unregisterRunningContainer(wc.containerID, 0, ContainerEndReason.COMPLETED);
+    verifyUnregisterRunningContainer(wc.tal, wc.containerID, 0, ContainerEndReason.COMPLETED, null);
     verify(wc.chh).register(wc.containerID);
     verify(wc.chh).unregister(wc.containerID);
 
@@ -697,10 +711,12 @@ public class TestAMContainer {
     wc.containerLaunched();
     wc.verifyState(AMContainerState.RUNNING);
 
-    wc.containerCompleted(ContainerExitStatus.PREEMPTED, TaskAttemptTerminationCause.EXTERNAL_PREEMPTION);
+    wc.containerCompleted(ContainerExitStatus.PREEMPTED,
+        TaskAttemptTerminationCause.EXTERNAL_PREEMPTION, "Container preempted externally");
     wc.verifyState(AMContainerState.COMPLETED);
     verify(wc.tal).registerRunningContainer(wc.containerID, 0);
-    verify(wc.tal).unregisterRunningContainer(wc.containerID, 0, ContainerEndReason.EXTERNAL_PREEMPTION);
+    verifyUnregisterRunningContainer(wc.tal, wc.containerID, 0,
+        ContainerEndReason.EXTERNAL_PREEMPTION, "Container preempted externally");
     verify(wc.chh).register(wc.containerID);
     verify(wc.chh).unregister(wc.containerID);
 
@@ -734,11 +750,12 @@ public class TestAMContainer {
     wc.containerLaunched();
     wc.verifyState(AMContainerState.RUNNING);
 
-    wc.containerCompleted(ContainerExitStatus.INVALID, TaskAttemptTerminationCause.INTERNAL_PREEMPTION);
+    wc.containerCompleted(ContainerExitStatus.INVALID,
+        TaskAttemptTerminationCause.INTERNAL_PREEMPTION, "Container preempted internally");
     wc.verifyState(AMContainerState.COMPLETED);
     verify(wc.tal).registerRunningContainer(wc.containerID, 0);
-    verify(wc.tal).unregisterRunningContainer(wc.containerID, 0,
-        ContainerEndReason.INTERNAL_PREEMPTION);
+    verifyUnregisterRunningContainer(wc.tal, wc.containerID, 0,
+        ContainerEndReason.INTERNAL_PREEMPTION, "Container preempted internally");
     verify(wc.chh).register(wc.containerID);
     verify(wc.chh).unregister(wc.containerID);
 
@@ -772,10 +789,11 @@ public class TestAMContainer {
     wc.containerLaunched();
     wc.verifyState(AMContainerState.RUNNING);
 
-    wc.containerCompleted(ContainerExitStatus.DISKS_FAILED, TaskAttemptTerminationCause.NODE_DISK_ERROR);
+    wc.containerCompleted(ContainerExitStatus.DISKS_FAILED,
+        TaskAttemptTerminationCause.NODE_DISK_ERROR, "NodeDiskError");
     wc.verifyState(AMContainerState.COMPLETED);
     verify(wc.tal).registerRunningContainer(wc.containerID, 0);
-    verify(wc.tal).unregisterRunningContainer(wc.containerID, 0, ContainerEndReason.OTHER);
+    verifyUnregisterRunningContainer(wc.tal, wc.containerID, 0, ContainerEndReason.OTHER, "NodeDiskError");
     verify(wc.chh).register(wc.containerID);
     verify(wc.chh).unregister(wc.containerID);
 
@@ -1194,6 +1212,7 @@ public class TestAMContainer {
 
     public AMContainerImpl amContainer;
 
+    @SuppressWarnings("deprecation") // ContainerId
     public WrappedContainer(boolean shouldProfile, String profileString) {
       applicationID = ApplicationId.newInstance(rmIdentifier, 1);
       appAttemptID = ApplicationAttemptId.newInstance(applicationID, 1);
@@ -1286,7 +1305,8 @@ 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>(), ""), 0, 0));
+          new ContainerContext(localResources, credentials, new HashMap<String, String>(), ""), 0,
+          0));
     }
 
     public void assignTaskAttempt(TezTaskAttemptID taID) {
@@ -1333,10 +1353,12 @@ public class TestAMContainer {
       amContainer.handle(new AMContainerEventCompleted(containerID, ContainerExitStatus.SUCCESS, null,
           TaskAttemptTerminationCause.CONTAINER_EXITED));
     }
-    
-    public void containerCompleted(int exitStatus, TaskAttemptTerminationCause errCause) {
+
+    public void containerCompleted(int exitStatus, TaskAttemptTerminationCause errCause,
+                                   String diagnostics) {
       reset(eventHandler);
-      amContainer.handle(new AMContainerEventCompleted(containerID, exitStatus, null, errCause));
+      amContainer.handle(
+          new AMContainerEventCompleted(containerID, exitStatus, diagnostics, errCause));
     }
 
     public void containerTimedOut() {
@@ -1417,4 +1439,33 @@ public class TestAMContainer {
         LocalResourceType.FILE, LocalResourceVisibility.APPLICATION, 1, 1000000);
     return lr;
   }
+
+  private void verifyUnregisterRunningContainer(TaskAttemptListener tal, ContainerId containerId,
+                                                int taskCommId,
+                                                ContainerEndReason containerEndReason,
+                                                String diagContains) {
+    ArgumentCaptor<String> argumentCaptor = ArgumentCaptor.forClass(String.class);
+    verify(tal).unregisterRunningContainer(eq(containerId), eq(taskCommId), eq(containerEndReason),
+        argumentCaptor.capture());
+    assertEquals(1, argumentCaptor.getAllValues().size());
+    if (diagContains != null) {
+      assertTrue(argumentCaptor.getValue().contains(diagContains));
+    } else {
+      assertNull(argumentCaptor.getValue());
+    }
+  }
+
+  private void verifyUnregisterTaskAttempt(TaskAttemptListener tal, TezTaskAttemptID taId,
+                                           int taskCommId, TaskAttemptEndReason endReason,
+                                           String diagContains) {
+    ArgumentCaptor<String> argumentCaptor = ArgumentCaptor.forClass(String.class);
+    verify(tal)
+        .unregisterTaskAttempt(eq(taId), eq(taskCommId), eq(endReason), argumentCaptor.capture());
+    assertEquals(1, argumentCaptor.getAllValues().size());
+    if (diagContains != null) {
+      assertTrue(argumentCaptor.getValue().contains(diagContains));
+    } else {
+      assertNull(argumentCaptor.getValue());
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/fee059b9/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 611e8cc..4883351 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
@@ -183,18 +183,26 @@ public class JoinValidate extends TezExampleBase {
     }
   }
 
+  // This is for internal use only, to use this example for external service testing.
+  // Not meant as documentation for the example.
   protected VertexExecutionContext getDefaultExecutionContext() {
     return null;
   }
 
+  // This is for internal use only, to use this example for external service testing.
+  // Not meant as documentation for the example.
   protected VertexExecutionContext getLhsExecutionContext() {
     return null;
   }
 
+  // This is for internal use only, to use this example for external service testing.
+  // Not meant as documentation for the example.
   protected VertexExecutionContext getRhsExecutionContext() {
     return null;
   }
 
+  // This is for internal use only, to use this example for external service testing.
+  // Not meant as documentation for the example.
   protected VertexExecutionContext getValidateExecutionContext() {
     return null;
   }

http://git-wip-us.apache.org/repos/asf/tez/blob/fee059b9/tez-ext-service-tests/pom.xml
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/pom.xml b/tez-ext-service-tests/pom.xml
index f95f4ca..5a1907f 100644
--- a/tez-ext-service-tests/pom.xml
+++ b/tez-ext-service-tests/pom.xml
@@ -81,11 +81,6 @@
     <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>

http://git-wip-us.apache.org/repos/asf/tez/blob/fee059b9/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 17f8a87..8b91dde 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
@@ -57,7 +57,7 @@ public class TezTestServiceTaskSchedulerService extends TaskScheduler {
   private final ConcurrentMap<Object, ContainerId> runningTasks =
       new ConcurrentHashMap<Object, ContainerId>();
 
-  // AppIdIdentifier to avoid conflicts with other containres in the system.
+  // AppIdIdentifier to avoid conflicts with other containers in the system.
 
   // Per instance
   private final int memoryPerInstance;
@@ -181,7 +181,7 @@ public class TezTestServiceTaskSchedulerService extends TaskScheduler {
   }
 
   @Override
-  public boolean deallocateTask(Object task, boolean taskSucceeded, TaskAttemptEndReason endReason) {
+  public boolean deallocateTask(Object task, boolean taskSucceeded, TaskAttemptEndReason endReason, String diagnostics) {
     ContainerId containerId = runningTasks.remove(task);
     if (containerId == null) {
       LOG.error("Could not determine ContainerId for task: " + task +
@@ -235,6 +235,7 @@ public class TezTestServiceTaskSchedulerService extends TaskScheduler {
           .newInstance(appId, appAttemptId.getAttemptId());
     }
 
+    @SuppressWarnings("deprecation")
     public Container createContainer(Resource capability, Priority priority, String hostname, int port) {
       ContainerId containerId = ContainerId.newInstance(customAppAttemptId, nextId.getAndIncrement());
       NodeId nodeId = NodeId.newInstance(hostname, port);

http://git-wip-us.apache.org/repos/asf/tez/blob/fee059b9/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 ef8f9e4..127967a 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
@@ -98,8 +98,8 @@ public class TezTestServiceTaskCommunicatorImpl extends TezTaskCommunicatorImpl
   }
 
   @Override
-  public void registerContainerEnd(ContainerId containerId, ContainerEndReason endReason) {
-    super.registerContainerEnd(containerId, endReason);
+  public void registerContainerEnd(ContainerId containerId, ContainerEndReason endReason, String diagnostics) {
+    super.registerContainerEnd(containerId, endReason, diagnostics);
   }
 
   @Override
@@ -154,7 +154,7 @@ public class TezTestServiceTaskCommunicatorImpl extends TezTaskCommunicatorImpl
               String message = re.toString();
               if (message.contains(RejectedExecutionException.class.getName())) {
                 getContext().taskKilled(taskSpec.getTaskAttemptID(),
-                    TaskAttemptEndReason.SERVICE_BUSY, "Service Busy");
+                    TaskAttemptEndReason.EXECUTOR_BUSY, "Service Busy");
               } else {
                 getContext()
                     .taskFailed(taskSpec.getTaskAttemptID(), TaskAttemptEndReason.OTHER,
@@ -175,8 +175,8 @@ public class TezTestServiceTaskCommunicatorImpl extends TezTaskCommunicatorImpl
   }
 
   @Override
-  public void unregisterRunningTaskAttempt(TezTaskAttemptID taskAttemptID, TaskAttemptEndReason endReason) {
-    super.unregisterRunningTaskAttempt(taskAttemptID, endReason);
+  public void unregisterRunningTaskAttempt(TezTaskAttemptID taskAttemptID, TaskAttemptEndReason endReason, String diagnostics) {
+    super.unregisterRunningTaskAttempt(taskAttemptID, endReason, diagnostics);
     // Nothing else to do for now. The push API in the test does not support termination of a running task
   }
 

http://git-wip-us.apache.org/repos/asf/tez/blob/fee059b9/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 472a43c..3b4c768 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
@@ -454,7 +454,7 @@ public class ContainerRunnerImpl extends AbstractService implements ContainerRun
       try {
         shouldDie = !taskRunner.run();
         if (shouldDie) {
-          LOG.info("Got a shouldDie notification via hearbeats. Shutting down");
+          LOG.info("Got a shouldDie notification via heartbeats. Shutting down");
           return new ContainerExecutionResult(ContainerExecutionResult.ExitStatus.SUCCESS, null,
               "Asked to die by the AM");
         }

http://git-wip-us.apache.org/repos/asf/tez/blob/fee059b9/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezTaskRunner2.java
----------------------------------------------------------------------
diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezTaskRunner2.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezTaskRunner2.java
index d8539c5..7fd4c75 100644
--- a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezTaskRunner2.java
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezTaskRunner2.java
@@ -273,10 +273,10 @@ public class TezTaskRunner2 {
           isFirstError = true;
           killTaskRequested.set(true);
         } else {
-          logErrorIngored("killTask", null);
+          logErrorIgnored("killTask", null);
         }
       } else {
-        logErrorIngored("killTask", null);
+        logErrorIgnored("killTask", null);
       }
     }
     if (isFirstError) {
@@ -331,10 +331,10 @@ public class TezTaskRunner2 {
             errorReporterToAm.set(true);
             oobSignalErrorInProgress = true;
           } else {
-            logErrorIngored("signalFatalError", message);
+            logErrorIgnored("signalFatalError", message);
           }
         } else {
-          logErrorIngored("signalFatalError", message);
+          logErrorIgnored("signalFatalError", message);
         }
       }
 
@@ -394,14 +394,14 @@ public class TezTaskRunner2 {
             registerFirstException(t, null);
             isFirstError = true;
           } else {
-            logErrorIngored("umbilicalFatalError", null);
+            logErrorIgnored("umbilicalFatalError", null);
           }
           // A race is possible between a task succeeding, and a subsequent timed heartbeat failing.
           // These errors can be ignored, since a task can only succeed if the synchronous taskSucceeded
           // method does not throw an exception, in which case task success is registered with the AM.
           // Leave subsequent heartbeat errors to the next entity to communicate using the TaskReporter
         } else {
-          logErrorIngored("umbilicalFatalError", null);
+          logErrorIgnored("umbilicalFatalError", null);
         }
         // Since this error came from the taskReporter - there's no point attempting to report a failure back to it.
         // However, the task does need to be cleaned up
@@ -425,7 +425,7 @@ public class TezTaskRunner2 {
         logAborting("shutdownRequested");
         killTaskInternal();
       } else {
-        logErrorIngored("shutdownRequested", null);
+        logErrorIgnored("shutdownRequested", null);
       }
     }
   }
@@ -474,7 +474,7 @@ public class TezTaskRunner2 {
         (successReportAttempted ? "success" : "failure/killed"), t);
   }
 
-  private void logErrorIngored(String ignoredEndReason, String errorMessage) {
+  private void logErrorIgnored(String ignoredEndReason, String errorMessage) {
     LOG.info(
         "Ignoring {} request since the task with id {} has ended for reason: {}. IgnoredError: {} ",
         ignoredEndReason, task.getTaskAttemptID(),

http://git-wip-us.apache.org/repos/asf/tez/blob/fee059b9/tez-runtime-internals/src/test/java/org/apache/tez/runtime/task/TaskExecutionTestHelpers.java
----------------------------------------------------------------------
diff --git a/tez-runtime-internals/src/test/java/org/apache/tez/runtime/task/TaskExecutionTestHelpers.java b/tez-runtime-internals/src/test/java/org/apache/tez/runtime/task/TaskExecutionTestHelpers.java
index fc42da3..7502c41 100644
--- a/tez-runtime-internals/src/test/java/org/apache/tez/runtime/task/TaskExecutionTestHelpers.java
+++ b/tez-runtime-internals/src/test/java/org/apache/tez/runtime/task/TaskExecutionTestHelpers.java
@@ -437,6 +437,7 @@ public class TaskExecutionTestHelpers {
     }
   }
 
+  @SuppressWarnings("deprecation")
   public static ContainerId createContainerId(ApplicationId appId) {
     ApplicationAttemptId appAttemptId = ApplicationAttemptId.newInstance(appId, 1);
     ContainerId containerId = ContainerId.newInstance(appAttemptId, 1);

http://git-wip-us.apache.org/repos/asf/tez/blob/fee059b9/tez-runtime-internals/src/test/java/org/apache/tez/runtime/task/TestContainerExecution.java
----------------------------------------------------------------------
diff --git a/tez-runtime-internals/src/test/java/org/apache/tez/runtime/task/TestContainerExecution.java b/tez-runtime-internals/src/test/java/org/apache/tez/runtime/task/TestContainerExecution.java
index c1616af..c3c4705 100644
--- a/tez-runtime-internals/src/test/java/org/apache/tez/runtime/task/TestContainerExecution.java
+++ b/tez-runtime-internals/src/test/java/org/apache/tez/runtime/task/TestContainerExecution.java
@@ -40,6 +40,7 @@ public class TestContainerExecution {
       executor = MoreExecutors.listeningDecorator(rawExecutor);
       ApplicationId appId = ApplicationId.newInstance(10000, 1);
       ApplicationAttemptId appAttemptId = ApplicationAttemptId.newInstance(appId, 1);
+      @SuppressWarnings("deprecation")
       ContainerId containerId = ContainerId.newInstance(appAttemptId, 1);
 
       TaskExecutionTestHelpers.TezTaskUmbilicalForTest


[02/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/699634f7/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 6ea1388..656bca1 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/699634f7/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/699634f7/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/699634f7/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 b7a3a87..daf1db6 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/699634f7/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/699634f7/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 f9a1c5e..44dcd1f 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.
@@ -124,7 +125,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());
@@ -134,14 +135,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());
@@ -160,7 +161,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();
@@ -175,7 +176,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());
 
@@ -183,13 +184,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());
@@ -208,7 +209,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.
@@ -225,7 +226,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());
 
@@ -234,13 +235,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());
 
@@ -249,14 +250,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());
@@ -289,7 +290,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());
@@ -326,7 +327,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());
@@ -349,7 +350,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);
@@ -381,13 +382,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);
@@ -423,7 +424,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);
@@ -457,7 +458,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);
@@ -487,11 +488,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,
@@ -540,8 +541,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,
@@ -570,8 +571,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,
@@ -601,8 +602,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,
@@ -632,8 +633,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);
 
@@ -661,8 +662,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);
 
@@ -696,8 +697,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);
 
@@ -733,8 +734,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);
 
@@ -770,8 +771,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);
 
@@ -1014,7 +1015,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);
@@ -1027,7 +1028,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());
@@ -1050,7 +1051,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);
@@ -1103,7 +1104,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());
@@ -1112,7 +1113,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());
@@ -1122,7 +1123,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());
@@ -1130,7 +1131,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());
@@ -1140,7 +1141,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());
@@ -1150,7 +1151,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());
@@ -1203,9 +1204,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);
@@ -1231,7 +1233,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() {
@@ -1281,7 +1283,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/699634f7/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/699634f7/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 1d124a6..ba17ba0 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
@@ -20,6 +20,7 @@ package org.apache.tez.examples;
 
 import java.io.IOException;
 import java.util.Set;
+import java.util.Map;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -135,7 +136,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
@@ -152,15 +153,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());
@@ -170,6 +174,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/699634f7/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/699634f7/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/699634f7/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/699634f7/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));
   }
 }


[08/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/26405ef5
Tree: http://git-wip-us.apache.org/repos/asf/tez/tree/26405ef5
Diff: http://git-wip-us.apache.org/repos/asf/tez/diff/26405ef5

Branch: refs/heads/TEZ-2003
Commit: 26405ef524d0bdd89afeac2a6b4fd34e72893724
Parents: 01ea9f7
Author: Siddharth Seth <ss...@apache.org>
Authored: Tue Apr 7 13:13:30 2015 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Fri Aug 14 13:46:43 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/26405ef5/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/26405ef5/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 394277a..080c20f 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/26405ef5/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,


[38/50] [abbrv] tez git commit: TEZ-2675. Add javadocs for new pluggable components, fix problems reported by jenkins. (sseth)

Posted by ss...@apache.org.
TEZ-2675. Add javadocs for new pluggable components, fix problems
reported by jenkins. (sseth)


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

Branch: refs/heads/TEZ-2003
Commit: efea3a557622b5c9d1d82671c691751db08f471f
Parents: 73ce6c5
Author: Siddharth Seth <ss...@apache.org>
Authored: Fri Aug 7 14:49:58 2015 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Fri Aug 14 13:47:10 2015 -0700

----------------------------------------------------------------------
 TEZ-2003-CHANGES.txt                            |   1 +
 pom.xml                                         |   1 +
 .../java/org/apache/tez/client/TezClient.java   |  52 ++++-
 .../tez/common/ServicePluginLifecycle.java      |   9 +
 .../main/java/org/apache/tez/dag/api/DAG.java   |   2 +-
 .../tez/dag/api/NamedEntityDescriptor.java      |  17 ++
 .../java/org/apache/tez/dag/api/Vertex.java     |  38 +++-
 .../api/ContainerLaunchRequest.java             |  11 +-
 .../serviceplugins/api/ContainerLauncher.java   |  35 ++++
 .../api/ContainerLauncherContext.java           |  63 +++++-
 .../api/ContainerLauncherOperationBase.java     |  17 ++
 .../api/ContainerStopRequest.java               |   3 +
 .../api/ServicePluginsDescriptor.java           |  19 +-
 .../tez/serviceplugins/api/TaskScheduler.java   | 151 +++++++++++++--
 .../api/TaskSchedulerContext.java               | 190 +++++++++++++++----
 .../apache/tez/dag/api/TaskCommunicator.java    |  95 +++++++++-
 .../tez/dag/api/TaskCommunicatorContext.java    | 100 +++++++++-
 .../dag/app/ContainerLauncherContextImpl.java   |   1 +
 .../dag/app/TaskAttemptListenerImpTezDag.java   |  15 +-
 .../app/launcher/ContainerLauncherRouter.java   |  14 +-
 .../dag/app/rm/TaskSchedulerContextImpl.java    |   3 +
 .../dag/app/rm/TaskSchedulerEventHandler.java   |  16 +-
 .../apache/tez/runtime/task/TezTaskRunner2.java |   2 +-
 23 files changed, 744 insertions(+), 111 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/efea3a55/TEZ-2003-CHANGES.txt
----------------------------------------------------------------------
diff --git a/TEZ-2003-CHANGES.txt b/TEZ-2003-CHANGES.txt
index b133ea3..75fac88 100644
--- a/TEZ-2003-CHANGES.txt
+++ b/TEZ-2003-CHANGES.txt
@@ -44,5 +44,6 @@ ALL CHANGES:
   TEZ-2626. Fix log lines with DEBUG in messages, consolidate TEZ-2003 TODOs.
   TEZ-2126. Add unit tests for verifying multiple schedulers, launchers, communicators.
   TEZ-2698. rebase 08/05
+  TEZ-2675. Add javadocs for new pluggable components, fix problems reported by jenkins
 
 INCOMPATIBLE CHANGES:

http://git-wip-us.apache.org/repos/asf/tez/blob/efea3a55/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 7ae5f31..bf2a6cf 100644
--- a/pom.xml
+++ b/pom.xml
@@ -780,6 +780,7 @@
           <configuration>
             <excludes>
               <exclude>CHANGES.txt</exclude>
+              <exclude>TEZ-2003-CHANGES.txt</exclude>
             </excludes>
           </configuration>
         </plugin>

http://git-wip-us.apache.org/repos/asf/tez/blob/efea3a55/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 e8b5386..373be81 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
@@ -282,7 +282,7 @@ public class TezClient {
    * Only LocalResourceType.FILE is supported. All files will be treated as
    * private.
    * 
-   * @param localFiles
+   * @param localFiles the files to be made available in the AM
    */
   public synchronized void addAppMasterLocalFiles(Map<String, LocalResource> localFiles) {
     Preconditions.checkNotNull(localFiles);
@@ -312,7 +312,7 @@ public class TezClient {
    * Master for the next DAG. <br>In session mode, credentials, if needed, must be
    * set before calling start()
    * 
-   * @param credentials
+   * @param credentials credentials
    */
   public synchronized void setAppMasterCredentials(Credentials credentials) {
     Preconditions
@@ -881,6 +881,9 @@ public class TezClient {
          append(tezDagIdFormat.get().format(1)).toString();
   }
 
+  /**
+   * A builder for setting up an instance of {@link org.apache.tez.client.TezClient}
+   */
   @Public
   public static class TezClientBuilder {
     final String name;
@@ -890,6 +893,15 @@ public class TezClient {
     private Credentials credentials;
     ServicePluginsDescriptor servicePluginsDescriptor;
 
+    /**
+     * Create an instance of a TezClientBuilder
+     *
+     * @param name
+     *          Name of the client. Used for logging etc. This will also be used
+     *          as app master name is session mode
+     * @param tezConf
+     *          Configuration for the framework
+     */
     private TezClientBuilder(String name, TezConfiguration tezConf) {
       this.name = name;
       this.tezConf = tezConf;
@@ -897,26 +909,62 @@ public class TezClient {
           TezConfiguration.TEZ_AM_SESSION_MODE, TezConfiguration.TEZ_AM_SESSION_MODE_DEFAULT);
     }
 
+    /**
+     * Specify whether this client is a session or not
+     * @param isSession whether the client is a session
+     * @return the current builder
+     */
     public TezClientBuilder setIsSession(boolean isSession) {
       this.isSession = isSession;
       return this;
     }
 
+    /**
+     * Set local resources to be used by the AppMaster
+     *
+     * @param localResources local files for the App Master
+     * @return the files to be added to the AM
+     */
     public TezClientBuilder setLocalResources(Map<String, LocalResource> localResources) {
       this.localResourceMap = localResources;
       return this;
     }
 
+    /**
+     * Setup security credentials
+     *
+     * @param credentials
+     *          Set security credentials to be used inside the app master, if
+     *          needed. Tez App Master needs credentials to access the staging
+     *          directory and for most HDFS cases these are automatically obtained
+     *          by Tez client. If the staging directory is on a file system for
+     *          which credentials cannot be obtained or for any credentials needed
+     *          by user code running inside the App Master, credentials must be
+     *          supplied by the user. These will be used by the App Master for the
+     *          next DAG. <br>
+     *          In session mode, credentials, if needed, must be set before
+     *          calling start()
+     * @return the current builder
+     */
     public TezClientBuilder setCredentials(Credentials credentials) {
       this.credentials = credentials;
       return this;
     }
 
+    /**
+     * Specify the service plugins that will be running in the AM
+     * @param servicePluginsDescriptor the service plugin descriptor with details about the plugins running in the AM
+     * @return the current builder
+     */
     public TezClientBuilder setServicePluginDescriptor(ServicePluginsDescriptor servicePluginsDescriptor) {
       this.servicePluginsDescriptor = servicePluginsDescriptor;
       return this;
     }
 
+    /**
+     * Build the actual instance of the {@link TezClient}
+     * @return an instance of {@link TezClient}
+     */
     public TezClient build() {
       return new TezClient(name, tezConf, isSession, localResourceMap, credentials,
           servicePluginsDescriptor);

http://git-wip-us.apache.org/repos/asf/tez/blob/efea3a55/tez-api/src/main/java/org/apache/tez/common/ServicePluginLifecycle.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/common/ServicePluginLifecycle.java b/tez-api/src/main/java/org/apache/tez/common/ServicePluginLifecycle.java
index 2eaa7be..b52b08c 100644
--- a/tez-api/src/main/java/org/apache/tez/common/ServicePluginLifecycle.java
+++ b/tez-api/src/main/java/org/apache/tez/common/ServicePluginLifecycle.java
@@ -17,6 +17,15 @@ package org.apache.tez.common;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 
+/**
+ * Defines a lifecycle for a Service. The typical implementation for services when used within the
+ * Tez framework would be
+ * 1. Construct the object.
+ * 2. initialize()
+ * 3. start()
+ * stop() - is invoked when the service is no longer required, and could be invoked while in any
+ * state, in case of failures
+ */
 @InterfaceAudience.Private
 @InterfaceStability.Unstable
 public interface ServicePluginLifecycle {

http://git-wip-us.apache.org/repos/asf/tez/blob/efea3a55/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 fce9522..927039a 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
@@ -343,7 +343,7 @@ public class DAG {
    *
    * @param vertexExecutionContext the default execution context for the DAG
    *
-   * @return
+   * @return this DAG
    */
   @Public
   @InterfaceStability.Unstable

http://git-wip-us.apache.org/repos/asf/tez/blob/efea3a55/tez-api/src/main/java/org/apache/tez/dag/api/NamedEntityDescriptor.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/NamedEntityDescriptor.java b/tez-api/src/main/java/org/apache/tez/dag/api/NamedEntityDescriptor.java
index 17c8c6c..426d4eb 100644
--- a/tez-api/src/main/java/org/apache/tez/dag/api/NamedEntityDescriptor.java
+++ b/tez-api/src/main/java/org/apache/tez/dag/api/NamedEntityDescriptor.java
@@ -14,9 +14,14 @@
 
 package org.apache.tez.dag.api;
 
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
 import com.google.common.base.Preconditions;
 import org.apache.hadoop.classification.InterfaceAudience;
 
+@SuppressWarnings("unchecked")
 public class NamedEntityDescriptor<T extends NamedEntityDescriptor<T>> extends EntityDescriptor<NamedEntityDescriptor<T>>  {
   private final String entityName;
 
@@ -37,6 +42,18 @@ public class NamedEntityDescriptor<T extends NamedEntityDescriptor<T>> extends E
   }
 
   @Override
+  public void write(DataOutput out) throws IOException {
+    throw new UnsupportedOperationException(
+        "write is not expected to be used for a NamedEntityDescriptor");
+  }
+
+  @Override
+  public void readFields(DataInput in) throws IOException {
+    throw new UnsupportedOperationException(
+        "readFields is not expected to be used for a NamedEntityDescriptor");
+  }
+
+  @Override
   public String toString() {
     boolean hasPayload =
         getUserPayload() == null ? false : getUserPayload().getPayload() == null ? false : true;

http://git-wip-us.apache.org/repos/asf/tez/blob/efea3a55/tez-api/src/main/java/org/apache/tez/dag/api/Vertex.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/Vertex.java b/tez-api/src/main/java/org/apache/tez/dag/api/Vertex.java
index 8953ae1..3f52a3d 100644
--- a/tez-api/src/main/java/org/apache/tez/dag/api/Vertex.java
+++ b/tez-api/src/main/java/org/apache/tez/dag/api/Vertex.java
@@ -419,13 +419,16 @@ public class Vertex {
    *
    * @param vertexExecutionContext the execution context for the vertex.
    *
-   * @return
+   * @return this Vertex
    */
   public Vertex setExecutionContext(VertexExecutionContext vertexExecutionContext) {
     this.vertexExecutionContext = vertexExecutionContext;
     return this;
   }
 
+  /**
+   * The execution context for a running vertex.
+   */
   @Public
   @InterfaceStability.Unstable
   public static class VertexExecutionContext {
@@ -435,15 +438,39 @@ public class Vertex {
     final String containerLauncherName;
     final String taskCommName;
 
+    /**
+     * Create an execution context which specifies whether the vertex needs to be executed in the
+     * AM
+     *
+     * @param executeInAm whether to execute the vertex in the AM
+     * @return the relevant execution context
+     */
     public static VertexExecutionContext createExecuteInAm(boolean executeInAm) {
       return new VertexExecutionContext(executeInAm, false);
     }
 
+    /**
+     * Create an execution context which specifies whether the vertex needs to be executed in
+     * regular containers
+     *
+     * @param executeInContainers whether to execute the vertex in regular containers
+     * @return the relevant execution context
+     */
     public static VertexExecutionContext createExecuteInContainers(boolean executeInContainers) {
       return new VertexExecutionContext(false, executeInContainers);
     }
 
-    public static VertexExecutionContext create(String taskSchedulerName, String containerLauncherName,
+    /**
+     * @param taskSchedulerName     the task scheduler name which was setup while creating the
+     *                              {@link org.apache.tez.client.TezClient}
+     * @param containerLauncherName the container launcher name which was setup while creating the
+     *                              {@link org.apache.tez.client.TezClient}
+     * @param taskCommName          the task communicator name which was setup while creating the
+     *                              {@link org.apache.tez.client.TezClient}
+     * @return the relevant execution context
+     */
+    public static VertexExecutionContext create(String taskSchedulerName,
+                                                String containerLauncherName,
                                                 String taskCommName) {
       return new VertexExecutionContext(taskSchedulerName, containerLauncherName, taskCommName);
     }
@@ -453,12 +480,13 @@ public class Vertex {
     }
 
     private VertexExecutionContext(String taskSchedulerName, String containerLauncherName,
-                                  String taskCommName) {
+                                   String taskCommName) {
       this(false, false, taskSchedulerName, containerLauncherName, taskCommName);
     }
 
-    private VertexExecutionContext(boolean executeInAm, boolean executeInContainers, String taskSchedulerName, String containerLauncherName,
-                      String taskCommName) {
+    private VertexExecutionContext(boolean executeInAm, boolean executeInContainers,
+                                   String taskSchedulerName, String containerLauncherName,
+                                   String taskCommName) {
       if (executeInAm || executeInContainers) {
         Preconditions.checkState(!(executeInAm && executeInContainers),
             "executeInContainers and executeInAM are mutually exclusive");

http://git-wip-us.apache.org/repos/asf/tez/blob/efea3a55/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ContainerLaunchRequest.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ContainerLaunchRequest.java b/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ContainerLaunchRequest.java
index cfd7ca7..f998fa2 100644
--- a/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ContainerLaunchRequest.java
+++ b/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ContainerLaunchRequest.java
@@ -22,6 +22,9 @@ import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.Token;
 
+/**
+ * Contains specifications for a container which needs to be launched
+ */
 @InterfaceAudience.Public
 @InterfaceStability.Unstable
 public class ContainerLaunchRequest extends ContainerLauncherOperationBase {
@@ -46,6 +49,10 @@ public class ContainerLaunchRequest extends ContainerLauncherOperationBase {
 
   // TODO Post TEZ-2003. TEZ-2625. ContainerLaunchContext needs to be built here instead of being passed in.
   // Basic specifications need to be provided here
+  /**
+   * The {@link ContainerLauncherContext} for the container being launched
+   * @return the container launch context for the launch request
+   */
   public ContainerLaunchContext getContainerLaunchContext() {
     return clc;
   }
@@ -53,7 +60,7 @@ public class ContainerLaunchRequest extends ContainerLauncherOperationBase {
   /**
    * Get the name of the task communicator which will be used to communicate
    * with the task that will run in this container.
-   * @return
+   * @return the task communicator to be used for this request
    */
   public String getTaskCommunicatorName() {
     return taskCommName;
@@ -61,7 +68,7 @@ public class ContainerLaunchRequest extends ContainerLauncherOperationBase {
 
   /**
    * Get the name of the scheduler which allocated this container.
-   * @return
+   * @return the scheduler name which provided the container
    */
   public String getSchedulerName() {
     return schedulerName;

http://git-wip-us.apache.org/repos/asf/tez/blob/efea3a55/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ContainerLauncher.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ContainerLauncher.java b/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ContainerLauncher.java
index 7f58f77..5a77b69 100644
--- a/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ContainerLauncher.java
+++ b/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ContainerLauncher.java
@@ -33,22 +33,57 @@ public abstract class ContainerLauncher implements ServicePluginLifecycle {
     this.containerLauncherContext = containerLauncherContext;
   }
 
+  /**
+   * An entry point for initialization.
+   * Order of service setup. Constructor, initialize(), start() - when starting a service.
+   *
+   * @throws Exception
+   */
   @Override
   public void initialize() throws Exception {
   }
 
+  /**
+   * An entry point for starting the service.
+   * Order of service setup. Constructor, initialize(), start() - when starting a service.
+   *
+   * @throws Exception
+   */
   @Override
   public void start() throws Exception {
   }
 
+  /**
+   * Stop the service. This could be invoked at any point, when the service is no longer required -
+   * including in case of errors.
+   *
+   * @throws Exception
+   */
   @Override
   public void shutdown() throws Exception {
   }
 
+  /**
+   * Get the {@link ContainerLauncherContext} associated with this instance of the container
+   * launcher, which is used to communicate with the rest of the system
+   *
+   * @return an instance of {@link ContainerLauncherContext}
+   */
   public final ContainerLauncherContext getContext() {
     return this.containerLauncherContext;
   }
 
+  /**
+   * A request to launch the specified container
+   *
+   * @param launchRequest the actual launch request
+   */
   public abstract void launchContainer(ContainerLaunchRequest launchRequest);
+
+  /**
+   * A request to stop a specific container
+   *
+   * @param stopRequest the actual stop request
+   */
   public abstract void stopContainer(ContainerStopRequest stopRequest);
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/efea3a55/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ContainerLauncherContext.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ContainerLauncherContext.java b/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ContainerLauncherContext.java
index 5da38b8..dcd9e80 100644
--- a/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ContainerLauncherContext.java
+++ b/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ContainerLauncherContext.java
@@ -24,30 +24,87 @@ import org.apache.tez.dag.api.UserPayload;
 @InterfaceStability.Unstable
 public interface ContainerLauncherContext {
 
-  // TODO Post TEZ-2003. Tez abstraction for ContainerId, NodeId, other YARN constructs
+  // TODO TEZ-2003 (post) TEZ-2664 Tez abstraction for ContainerId, NodeId, other YARN constructs
 
   // Reporting APIs
+
+  /**
+   * Inform the framework that a container has been launched
+   *
+   * @param containerId the id of the container that has been launched
+   */
   void containerLaunched(ContainerId containerId);
 
+  /**
+   * Inform the framework of an issue while trying to launch a container.
+   *
+   * @param containerId the id of the container which failed to launch
+   * @param diagnostics diagnostics for the failure
+   */
   void containerLaunchFailed(ContainerId containerId, String diagnostics);
 
+  /**
+   * Inform the framework that a request has been made to stop a container
+   *
+   * @param containerId the id of the associated container
+   */
   void containerStopRequested(ContainerId containerId);
 
+  /**
+   * Inform the framework that the attempt to stop a container failed
+   *
+   * @param containerId the id of the associated container
+   * @param diagnostics diagnostics for the failure
+   */
   void containerStopFailed(ContainerId containerId, String diagnostics);
 
-  // TODO Post TEZ-2003. TaskAttemptEndReason does not belong here, and is an unnecessary leak.
+  // TODO TEZ-2003 (post). TEZ-2676 TaskAttemptEndReason does not belong here, and is an unnecessary leak.
   // ContainerCompleted is normally generated by the scheduler in case of YARN since the RM informs about completion.
   // For other sources, there may not be a central entity making this information available. The ContainerLauncher
   // on the stop request will likely be the best place to generate it.
-  void containerCompleted(ContainerId containerId, int exitStatus, String diagnostics, TaskAttemptEndReason endReason);
+
+  /**
+   * Inform the scheduler that a container was successfully stopped
+   *
+   * @param containerId the id of the associated container
+   * @param exitStatus  the exit status of the container
+   * @param diagnostics diagnostics associated with the container end
+   * @param endReason   the end reason for the task running in the container
+   */
+  void containerCompleted(ContainerId containerId, int exitStatus, String diagnostics,
+                          TaskAttemptEndReason endReason);
 
   // Lookup APIs
 
+  /**
+   * Get the UserPayload that was configured while setting up the launcher
+   *
+   * @return the initially configured user payload
+   */
   UserPayload getInitialUserPayload();
 
+  /**
+   * Get the number of nodes being handled by the specified source
+   *
+   * @param sourceName the relevant source name
+   * @return the initial payload
+   */
   int getNumNodes(String sourceName);
 
+  /**
+   * Get the application attempt id for the running application. Relevant when running under YARN
+   *
+   * @return the applicationAttemptId for the running app
+   */
   ApplicationAttemptId getApplicationAttemptId();
 
+  /**
+   * Get meta info from the specified TaskCommunicator. This assumes that the launched has been
+   * setup
+   * along with a compatible TaskCommunicator, and the launcher knows how to read this meta-info
+   *
+   * @param taskCommName the name of the task communicator
+   * @return meta info for the requested task communicator
+   */
   Object getTaskCommunicatorMetaInfo(String taskCommName);
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/efea3a55/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ContainerLauncherOperationBase.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ContainerLauncherOperationBase.java b/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ContainerLauncherOperationBase.java
index 29e0420..260b681 100644
--- a/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ContainerLauncherOperationBase.java
+++ b/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ContainerLauncherOperationBase.java
@@ -24,6 +24,11 @@ import org.apache.hadoop.yarn.api.records.Token;
 @InterfaceStability.Unstable
 public class ContainerLauncherOperationBase {
 
+  // TODO TEZ-2702 (TEZ-2003 post)
+  // - Get rid of YARN constructs.
+  // - ContainerToken may not always be required
+
+
   private final NodeId nodeId;
   private final ContainerId containerId;
   private final Token containerToken;
@@ -36,14 +41,26 @@ public class ContainerLauncherOperationBase {
     this.containerToken = containerToken;
   }
 
+  /**
+   * Get the node on whcih this container is to be launched
+   * @return
+   */
   public NodeId getNodeId() {
     return nodeId;
   }
 
+  /**
+   * Get the containerId for the container
+   * @return
+   */
   public ContainerId getContainerId() {
     return containerId;
   }
 
+  /**
+   * Get the security token for the container. Primarily for YARN
+   * @return
+   */
   public Token getContainerToken() {
     return containerToken;
   }

http://git-wip-us.apache.org/repos/asf/tez/blob/efea3a55/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ContainerStopRequest.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ContainerStopRequest.java b/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ContainerStopRequest.java
index cb0af31..be7d00a 100644
--- a/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ContainerStopRequest.java
+++ b/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ContainerStopRequest.java
@@ -20,6 +20,9 @@ import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.Token;
 
+/**
+ * Contains specifications for a container which needs to be stopped
+ */
 @InterfaceAudience.Public
 @InterfaceStability.Unstable
 public class ContainerStopRequest extends ContainerLauncherOperationBase {

http://git-wip-us.apache.org/repos/asf/tez/blob/efea3a55/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ServicePluginsDescriptor.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ServicePluginsDescriptor.java b/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ServicePluginsDescriptor.java
index 2e4fc46..ce35350 100644
--- a/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ServicePluginsDescriptor.java
+++ b/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ServicePluginsDescriptor.java
@@ -18,6 +18,10 @@ import com.google.common.base.Preconditions;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 
+/**
+ * An {@link ServicePluginsDescriptor} describes the list of plugins running within the AM for
+ * sourcing resources, launching and executing work.
+ */
 @InterfaceAudience.Public
 @InterfaceStability.Unstable
 public class ServicePluginsDescriptor {
@@ -53,7 +57,7 @@ public class ServicePluginsDescriptor {
    * @param taskSchedulerDescriptor the task scheduler plugin descriptors
    * @param containerLauncherDescriptors the container launcher plugin descriptors
    * @param taskCommunicatorDescriptors the task communicator plugin descriptors
-   * @return
+   * @return a {@link ServicePluginsDescriptor} instance
    */
   public static ServicePluginsDescriptor create(TaskSchedulerDescriptor[] taskSchedulerDescriptor,
                                                 ContainerLauncherDescriptor[] containerLauncherDescriptors,
@@ -69,7 +73,7 @@ public class ServicePluginsDescriptor {
    * @param taskSchedulerDescriptor the task scheduler plugin descriptors
    * @param containerLauncherDescriptors the container launcher plugin descriptors
    * @param taskCommunicatorDescriptors the task communicator plugin descriptors
-   * @return
+   * @return a {@link ServicePluginsDescriptor} instance
    */
   public static ServicePluginsDescriptor create(boolean enableUber,
                                                 TaskSchedulerDescriptor[] taskSchedulerDescriptor,
@@ -88,7 +92,7 @@ public class ServicePluginsDescriptor {
    * @param taskSchedulerDescriptor the task scheduler plugin descriptors
    * @param containerLauncherDescriptors the container launcher plugin descriptors
    * @param taskCommunicatorDescriptors the task communicator plugin descriptors
-   * @return
+   * @return a {@link ServicePluginsDescriptor} instance
    */
   public static ServicePluginsDescriptor create(boolean enableContainers, boolean enableUber,
                                                 TaskSchedulerDescriptor[] taskSchedulerDescriptor,
@@ -103,30 +107,35 @@ public class ServicePluginsDescriptor {
    * execution is enabled by default
    *
    * @param enableUber whether to enable execution in the AM or not
-   * @return
+   * @return a {@link ServicePluginsDescriptor} instance
    */
   public static ServicePluginsDescriptor create(boolean enableUber) {
     return new ServicePluginsDescriptor(true, enableUber, null, null, null);
   }
 
 
+  @InterfaceAudience.Private
   public boolean areContainersEnabled() {
     return enableContainers;
   }
 
+  @InterfaceAudience.Private
   public boolean isUberEnabled() {
     return enableUber;
   }
 
+  @InterfaceAudience.Private
   public TaskSchedulerDescriptor[] getTaskSchedulerDescriptors() {
     return taskSchedulerDescriptors;
   }
 
+  @InterfaceAudience.Private
   public ContainerLauncherDescriptor[] getContainerLauncherDescriptors() {
     return containerLauncherDescriptors;
   }
 
+  @InterfaceAudience.Private
   public TaskCommunicatorDescriptor[] getTaskCommunicatorDescriptors() {
     return taskCommunicatorDescriptors;
   }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tez/blob/efea3a55/tez-api/src/main/java/org/apache/tez/serviceplugins/api/TaskScheduler.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/serviceplugins/api/TaskScheduler.java b/tez-api/src/main/java/org/apache/tez/serviceplugins/api/TaskScheduler.java
index 9ff2bd5..f05bddc 100644
--- a/tez-api/src/main/java/org/apache/tez/serviceplugins/api/TaskScheduler.java
+++ b/tez-api/src/main/java/org/apache/tez/serviceplugins/api/TaskScheduler.java
@@ -16,21 +16,30 @@ package org.apache.tez.serviceplugins.api;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.service.AbstractService;
+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.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.tez.common.ServicePluginLifecycle;
 
+/**
+ * This class represents the API for a custom TaskScheduler which can be run within the Tez AM.
+ * This can be used to source resources from different sources, as well as control the logic of
+ * how these resources get allocated to the different tasks within a DAG which needs resources.
+ * <p/>
+ * The plugin is initialized with an instance of {@link TaskSchedulerContext} - which provides
+ * a mechanism to notify the system about allocation decisions and resources to the Tez framework.
+ */
 @InterfaceAudience.Public
 @InterfaceStability.Unstable
 public abstract class TaskScheduler implements ServicePluginLifecycle {
 
   // TODO TEZ-2003 (post) TEZ-2668
   // - Should setRegister / unregister be part of APIs when not YARN specific ?
-  // - Include vertex / task information in therequest so that the scheduler can make decisions
+  // - Include vertex / task information in the request so that the scheduler can make decisions
   // around prioritizing tasks in the same vertex when others exist at the same priority.
+  // There should be an interface around Object task - if it's meant to be used for equals / hashCode.
 
   private final TaskSchedulerContext taskSchedulerContext;
 
@@ -38,53 +47,169 @@ public abstract class TaskScheduler implements ServicePluginLifecycle {
     this.taskSchedulerContext = taskSchedulerContext;
   }
 
+  /**
+   * An entry point for initialization.
+   * Order of service setup. Constructor, initialize(), start() - when starting a service.
+   *
+   * @throws Exception
+   */
   @Override
   public void initialize() throws Exception {
   }
 
+  /**
+   * An entry point for starting the service.
+   * Order of service setup. Constructor, initialize(), start() - when starting a service.
+   *
+   * @throws Exception
+   */
   @Override
   public void start() throws Exception {
   }
 
+  /**
+   * Stop the service. This could be invoked at any point, when the service is no longer required -
+   * including in case of errors.
+   *
+   * @throws Exception
+   */
   @Override
   public void shutdown() throws Exception {
   }
 
+  /**
+   * Get the {@link TaskSchedulerContext} associated with this instance of the scheduler, which is
+   * used to communicate with the rest of the system
+   *
+   * @return an instance of {@link TaskSchedulerContext}
+   */
+  public final TaskSchedulerContext getContext() {
+    return taskSchedulerContext;
+  }
 
+  /**
+   * Get the currently available resources from this source
+   *
+   * @return the resources available at the time of invocation
+   */
   public abstract Resource getAvailableResources();
 
-  public abstract int getClusterNodeCount();
-
-  public abstract void dagComplete();
-
+  /**
+   * Get the total available resources from this source
+   *
+   * @return the total available resources from the source
+   */
   public abstract Resource getTotalResources();
 
+  /**
+   * Get the number of nodes available from the source
+   *
+   * @return the number of nodes
+   */
+  public abstract int getClusterNodeCount();
+
+  /**
+   * Indication to a source that a node has been blacklisted, and should not be used for subsequent
+   * allocations.
+   *
+   * @param nodeId te nodeId to be blacklisted
+   */
   public abstract void blacklistNode(NodeId nodeId);
 
+  /**
+   * Indication to a source that a node has been un-blacklisted, and can be used from subsequent
+   * allocations
+   *
+   * @param nodeId the nodeId to be unblacklisted
+   */
   public abstract void unblacklistNode(NodeId nodeId);
 
+  /**
+   * A request to the source to allocate resources for a requesting task, with location information
+   * optionally specified
+   *
+   * @param task               the task for which resources are being accepted.
+   * @param capability         the required resources to run this task
+   * @param hosts              the preferred host locations for the task
+   * @param racks              the preferred rack locations for the task
+   * @param priority           the priority of the request for this allocation. A lower value
+   *                           implies a higher priority
+   * @param containerSignature the specifications for the container (environment, etc) which will
+   *                           be
+   *                           used for this task - if applicable
+   * @param clientCookie       a cookie associated with this request. This should be returned back
+   *                           via the {@link TaskSchedulerContext#taskAllocated(Object, Object,
+   *                           Container)} method when a task is assigned to a resource
+   */
   public abstract void allocateTask(Object task, Resource capability,
                                     String[] hosts, String[] racks, Priority priority,
                                     Object containerSignature, Object clientCookie);
 
   /**
-   * Allocate affinitized to a specific container
+   * A request to the source to allocate resources for a requesting task, based on a previously used
+   * container
+   *
+   * @param task               the task for which resources are being accepted.
+   * @param capability         the required resources to run this task
+   * @param containerId        a previous container which is used as an indication as to where this
+   *                           task should be placed
+   * @param priority           the priority of the request for this allocation. A lower value
+   *                           implies a higher priority
+   * @param containerSignature the specifications for the container (environment, etc) which will
+   *                           be
+   *                           used for this task - if applicable
+   * @param clientCookie       a cookie associated with this request. This should be returned back
+   *                           via the {@link TaskSchedulerContext#taskAllocated(Object, Object,
+   *                           Container)} method when a task is assigned to a resource
    */
   public abstract void allocateTask(Object task, Resource capability,
-                                    ContainerId containerId, Priority priority, Object containerSignature,
+                                    ContainerId containerId, Priority priority,
+                                    Object containerSignature,
                                     Object clientCookie);
 
-  /** 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);
+  /**
+   * A request to deallocate a task. This is typically a result of a task completing - with success
+   * or failure. It could also be the result of a decision to not run the task, before it is
+   * allocated or started.
+   * <p/>
+   * Plugin writers need to de-allocate containers via the context once it's no longer required, for
+   * correct book-keeping
+   *
+   * @param task          the task being de-allocated.
+   * @param taskSucceeded whether the task succeeded or not
+   * @param endReason     the reason for the task failure
+   * @return true if the task was associated with a container, false if the task was not associated
+   * with a container
+   */
+  public abstract boolean deallocateTask(Object task, boolean taskSucceeded,
+                                         TaskAttemptEndReason endReason);
 
+  /**
+   * A request to de-allocate a previously allocated container.
+   *
+   * @param containerId the containerId to de-allocate
+   * @return the task which was previously associated with this container, null otherwise
+   */
   public abstract Object deallocateContainer(ContainerId containerId);
 
+  /**
+   * Inform the scheduler that it should unregister. This is primarily valid for schedulers which
+   * require registration (YARN a.t.m)
+   */
   public abstract void setShouldUnregister();
 
+  /**
+   * Checks with the scheduler whether it has unregistered.
+   *
+   * @return true if the scheduler has unregistered. False otherwise.
+   */
   public abstract boolean hasUnregistered();
 
+  /**
+   * Indicates to the scheduler that the currently running dag has completed.
+   * This can be used to reset dag specific statistics, potentially release resources and prepare
+   * for a new DAG.
+   */
+  public abstract void dagComplete();
 
-  public final TaskSchedulerContext getContext() {
-    return taskSchedulerContext;
-  }
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/efea3a55/tez-api/src/main/java/org/apache/tez/serviceplugins/api/TaskSchedulerContext.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/serviceplugins/api/TaskSchedulerContext.java b/tez-api/src/main/java/org/apache/tez/serviceplugins/api/TaskSchedulerContext.java
index dbbf75c..a24061f 100644
--- a/tez-api/src/main/java/org/apache/tez/serviceplugins/api/TaskSchedulerContext.java
+++ b/tez-api/src/main/java/org/apache/tez/serviceplugins/api/TaskSchedulerContext.java
@@ -31,15 +31,24 @@ import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.tez.common.ContainerSignatureMatcher;
 import org.apache.tez.dag.api.UserPayload;
 
+/**
+ * Context for a {@link TaskScheduler}
+ * <p/>
+ * This provides methods for a scheduler to interact with the Tez framework.
+ * <p/>
+ * Calls into this should be outside of locks, which may also be obtained by methods in the
+ * scheduler
+ * which implement the {@link TaskScheduler} interface
+ */
 @InterfaceAudience.Public
 @InterfaceStability.Unstable
-
 public interface TaskSchedulerContext {
 
-  public class AppFinalStatus {
+  class AppFinalStatus {
     public final FinalApplicationStatus exitStatus;
     public final String exitMessage;
     public final String postCompletionTrackingUrl;
+
     public AppFinalStatus(FinalApplicationStatus exitStatus,
                           String exitMessage,
                           String posCompletionTrackingUrl) {
@@ -49,67 +58,180 @@ public interface TaskSchedulerContext {
     }
   }
 
+  /**
+   * Indicates the state the AM is in.
+   */
   enum AMState {
-    IDLE, RUNNING_APP, COMPLETED
+    IDLE,
+    RUNNING_APP,
+    COMPLETED
   }
 
   // TODO TEZ-2003 (post) TEZ-2664. Remove references to YARN constructs like Container, ContainerStatus, NodeReport
   // TODO TEZ-2003 (post) TEZ-2668 Enhancements to TaskScheduler interfaces
   // - setApplicationRegistrationData may not be relevant to non YARN clusters
   // - getAppFinalStatus may not be relevant to non YARN clusters
-  // upcall to app must be outside locks
-  public void taskAllocated(Object task,
-                            Object appCookie,
-                            Container container);
-  // this may end up being called for a task+container pair that the app
-  // has not heard about. this can happen because of a race between
-  // taskAllocated() upcall and deallocateTask() downcall
-  public void containerCompleted(Object taskLastAllocated,
-                                 ContainerStatus containerStatus);
-  public void containerBeingReleased(ContainerId containerId);
-  public void nodesUpdated(List<NodeReport> updatedNodes);
-  public void appShutdownRequested();
-
-  // TODO Post TEZ-2003, this method specifically needs some cleaning up.
-  // ClientAMSecretKey is only relevant when running under YARN. As are ApplicationACLs.
-  public void setApplicationRegistrationData(
+
+
+  /**
+   * Indicate to the framework that a container is being assigned to a task.
+   *
+   * @param task      the task for which a container is being assigned. This should be the same
+   *                  instance that was provided when requesting for an allocation
+   * @param appCookie the cookie which was provided while requesting allocation for this task
+   * @param container the actual container assigned to the task
+   */
+  void taskAllocated(Object task,
+                     Object appCookie,
+                     Container container);
+
+
+  /**
+   * Indicate to the framework that a container has completed. This is typically used by sources
+   * which have
+   * a means to indicate a container failure to the scheduler (typically centrally managed
+   * schedulers - YARN)
+   *
+   * @param taskLastAllocated the task that was allocated to this container, if any. This is the
+   *                          same instance that was passed in while requesting an allocation
+   * @param containerStatus   the status with which the container ended
+   */
+  void containerCompleted(Object taskLastAllocated,
+                          ContainerStatus containerStatus);
+
+  /**
+   * Indicates to the framework that a container is being released.
+   *
+   * @param containerId the id of the container being released
+   */
+  void containerBeingReleased(ContainerId containerId);
+
+
+  /**
+   * Provide an update to the framework about the status of nodes available to this report
+   *
+   * @param updatedNodes a list of updated node reports
+   */
+  void nodesUpdated(List<NodeReport> updatedNodes);
+
+  /**
+   * Inform the framework that an app shutdown is required. This should typically not be used, other
+   * than
+   * by the YARN scheduler.
+   */
+  void appShutdownRequested();
+
+  /**
+   * Provide an update to the framework about specific information about the source managed by this
+   * scheduler.
+   *
+   * @param maxContainerCapability the total resource capability of the source
+   * @param appAcls                ACLs for the source
+   * @param clientAMSecretKey      a secret key provided by the source
+   */
+  void setApplicationRegistrationData(
       Resource maxContainerCapability,
       Map<ApplicationAccessType, String> appAcls,
       ByteBuffer clientAMSecretKey
   );
-  public void onError(Throwable t);
-  public float getProgress();
-  public void preemptContainer(ContainerId containerId);
 
-  public AppFinalStatus getFinalAppStatus();
+  /**
+   * Indicate to the framework that the scheduler has run into an error. This will cause
+   * the DAG and application to be killed.
+   *
+   * @param t the relevant error
+   */
+  void onError(Throwable t);
+
+  /**
+   * Inform the framework that the scheduler has determined that a previously allocated container
+   * needs to be preempted
+   *
+   * @param containerId the containerId to be preempted
+   */
+  void preemptContainer(ContainerId containerId);
+
+  /**
+   * Get the final status for the application, which could be provided to the coordinator of the
+   * source.
+   * Primarily relevant to YARN
+   *
+   * @return the final Application status
+   */
+  AppFinalStatus getFinalAppStatus();
 
 
   // Getters
 
-  public UserPayload getInitialUserPayload();
+  /**
+   * Get the UserPayload that was configured while setting up the scheduler
+   *
+   * @return the initially configured user payload
+   */
+  UserPayload getInitialUserPayload();
 
-  public String getAppTrackingUrl();
+  /**
+   * Get the tracking URL for the application. Primarily relevant to YARN
+   *
+   * @return the trackingUrl for the app
+   */
+  String getAppTrackingUrl();
+
+  /**
+   * Request the framework for progress of the running DAG. This value must be between 0 and 1
+   *
+   * @return progress
+   */
+  float getProgress();
 
   /**
    * A custom cluster identifier allocated to schedulers to generate an AppId, if not making
    * use of YARN
-   * @return
+   *
+   * @return the custom cluster identifier
    */
-  public long getCustomClusterIdentifier();
+  long getCustomClusterIdentifier();
 
-  public ContainerSignatureMatcher getContainerSignatureMatcher();
+  /**
+   * Get an instance of {@link ContainerSignatureMatcher} which can be used to check whether the
+   * specifications of a container match what is required by a task.
+   *
+   * @return an instance of {@link ContainerSignatureMatcher}
+   */
+  ContainerSignatureMatcher getContainerSignatureMatcher();
 
   /**
    * Get the application attempt id for the running application. Relevant when running under YARN
-   * @return
+   *
+   * @return the applicationAttemptId for the running app
    */
-  public ApplicationAttemptId getApplicationAttemptId();
+  ApplicationAttemptId getApplicationAttemptId();
 
-  public String getAppHostName();
+  /**
+   * Get the hostname on which the app is running
+   *
+   * @return the hostname
+   */
+  String getAppHostName();
 
-  public int getAppClientPort();
+  /**
+   * Get the port on which the DAG client is listening
+   *
+   * @return the client port
+   */
+  int getAppClientPort();
 
-  public boolean isSession();
+  /**
+   * Check whether the AM is running in session mode.
+   *
+   * @return true if session mode, false otherwise
+   */
+  boolean isSession();
 
-  public AMState getAMState();
+  /**
+   * Get the state of the AppMaster
+   *
+   * @return the app master state
+   */
+  AMState getAMState();
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/efea3a55/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 794d390..4fc541c 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
@@ -29,6 +29,19 @@ import org.apache.tez.runtime.api.impl.TaskSpec;
 
 // TODO TEZ-2003 (post) TEZ-2665. Move to the tez-api module
 // TODO TEZ-2003 (post) TEZ-2664. Ideally, don't expose YARN containerId; instead expose a Tez specific construct.
+
+/**
+ * This class represents the API for a custom TaskCommunicator which can be run within the Tez AM.
+ * This is used to communicate with running services, potentially launching tasks, and getting
+ * updates from running tasks.
+ * <p/>
+ * The plugin is initialized with an instance of {@link TaskCommunicatorContext} - which provides
+ * a mechanism to notify the system about allocation decisions and resources to the Tez framework.
+ *
+ * If setting up a heartbeat between the task and the AM, the framework is responsible for error checking
+ * of this heartbeat mechanism, handling lost or duplicate responses.
+ *
+ */
 public abstract class TaskCommunicator implements ServicePluginLifecycle {
 
   // TODO TEZ-2003 (post) TEZ-2666 Enhancements to interface
@@ -45,34 +58,100 @@ public abstract class TaskCommunicator implements ServicePluginLifecycle {
     this.taskCommunicatorContext = taskCommunicatorContext;
   }
 
+  /**
+   * Get the {@link TaskCommunicatorContext} associated with this instance of the scheduler, which
+   * is
+   * used to communicate with the rest of the system
+   *
+   * @return an instance of {@link TaskCommunicatorContext}
+   */
   public TaskCommunicatorContext getContext() {
     return taskCommunicatorContext;
   }
 
+  /**
+   * An entry point for initialization.
+   * Order of service setup. Constructor, initialize(), start() - when starting a service.
+   *
+   * @throws Exception
+   */
   @Override
   public void initialize() throws Exception {
   }
 
+  /**
+   * An entry point for starting the service.
+   * Order of service setup. Constructor, initialize(), start() - when starting a service.
+   *
+   * @throws Exception
+   */
   @Override
   public void start() throws Exception {
   }
 
+  /**
+   * Stop the service. This could be invoked at any point, when the service is no longer required -
+   * including in case of errors.
+   *
+   * @throws Exception
+   */
   @Override
   public void shutdown() throws Exception {
   }
 
 
+  /**
+   * Register a new container.
+   *
+   * @param containerId the associated containerId
+   * @param hostname    the hostname on which the container runs
+   * @param port        the port for the service which is running the container
+   */
   public abstract void registerRunningContainer(ContainerId containerId, String hostname, int port);
 
+  /**
+   * Register the end of a container. This can be caused by preemption, the container completing
+   * successfully, etc.
+   *
+   * @param containerId the associated containerId
+   * @param endReason   the end reason for the container completing
+   */
   public abstract void registerContainerEnd(ContainerId containerId, ContainerEndReason endReason);
 
+  /**
+   * Register a task attempt to execute on a container
+   *
+   * @param containerId         the containerId on which this task needs to run
+   * @param taskSpec            the task specifications for the task to be executed
+   * @param additionalResources additional local resources which may be required to run this task
+   *                            on
+   *                            the container
+   * @param credentials         the credentials required to run this task
+   * @param credentialsChanged  whether the credentials are different from the original credentials
+   *                            associated with this container
+   * @param priority            the priority of the task being executed
+   */
   public abstract void registerRunningTaskAttempt(ContainerId containerId, TaskSpec taskSpec,
                                                   Map<String, LocalResource> additionalResources,
                                                   Credentials credentials,
                                                   boolean credentialsChanged, int priority);
 
-  public abstract void unregisterRunningTaskAttempt(TezTaskAttemptID taskAttemptID, TaskAttemptEndReason endReason);
+  /**
+   * Register the completion of a task. This may be a result of preemption, the container dying,
+   * the
+   * node dying, the task completing to success
+   *
+   * @param taskAttemptID the task attempt which has completed / needs to be completed
+   * @param endReason     the endReason for the task attempt.
+   */
+  public abstract void unregisterRunningTaskAttempt(TezTaskAttemptID taskAttemptID,
+                                                    TaskAttemptEndReason endReason);
 
+  /**
+   * Return the address, if any, that the service listens on
+   *
+   * @return the address
+   */
   public abstract InetSocketAddress getAddress();
 
   /**
@@ -82,11 +161,13 @@ public abstract class TaskCommunicator implements ServicePluginLifecycle {
    * 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>
-   *
+   * <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
+   * <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}
@@ -97,16 +178,18 @@ public abstract class TaskCommunicator implements ServicePluginLifecycle {
   /**
    * 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.
-   *
+   * <p/>
    * After this, the contents returned from querying the context may change at any point - due to
    * the next dag being submitted.
    */
   public abstract void dagComplete(String dagName);
 
   /**
-   * Share meta-information such as host:port information where the Task Communicator may be listening.
+   * Share meta-information such as host:port information where the Task Communicator may be
+   * listening.
    * Primarily for use by compatible launchers to learn this information.
-   * @return
+   *
+   * @return meta info for the task communicator
    */
   public abstract Object getMetaInfo();
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/efea3a55/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 8073f6a..0a684e7 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
@@ -42,31 +42,112 @@ public interface TaskCommunicatorContext {
   // - Maybe add book-keeping as a helper library, instead of each impl tracking container to task etc.
   // - Handling of containres / tasks which no longer exist in the system (formalized interface instead of a shouldDie notification)
 
+  /**
+   * Get the UserPayload that was configured while setting up the task communicator
+   *
+   * @return the initially configured user payload
+   */
   UserPayload getInitialUserPayload();
 
+  /**
+   * Get the application attempt id for the running application. Relevant when running under YARN
+   *
+   * @return the applicationAttemptId for the running app
+   */
   ApplicationAttemptId getApplicationAttemptId();
+
+  /**
+   * Get credentials associated with the AppMaster
+   *
+   * @return credentials
+   */
   Credentials getCredentials();
 
+  /**
+   * Check whether a running attempt can commit. This provides a leader election mechanism amongst
+   * multiple running attempts
+   *
+   * @param taskAttemptId the associated task attempt id
+   * @return whether the attempt can commit or not
+   * @throws IOException
+   */
   boolean canCommit(TezTaskAttemptID taskAttemptId) throws IOException;
 
+  /**
+   * Mechanism for a {@link TaskCommunicator} to provide updates on a running task, as well as
+   * receive new information which may need to be propagated to the task. This includes events
+   * generated by the task and events which need to be sent to the task
+   * This method must be invoked periodically to receive updates for a running task
+   *
+   * @param request the update from the running task.
+   * @return the response that is requried by the task.
+   * @throws IOException
+   * @throws TezException
+   */
   TaskHeartbeatResponse heartbeat(TaskHeartbeatRequest request) throws IOException, TezException;
 
+  /**
+   * Check whether the container is known by the framework. The state of this container is
+   * irrelevant
+   *
+   * @param containerId the relevant container id
+   * @return true if the container is known, false if it isn't
+   */
   boolean isKnownContainer(ContainerId containerId);
 
+  /**
+   * Inform the framework that a task is alive. This needs to be invoked periodically to avoid the
+   * task attempt timing out.
+   * Invocations to heartbeat provides the same keep-alive functionality
+   *
+   * @param taskAttemptId the relevant task attempt
+   */
   void taskAlive(TezTaskAttemptID taskAttemptId);
 
+  /**
+   * Inform the framework that a container is alive. This need to be invoked periodically to avoid
+   * the container attempt timing out.
+   * Invocations to heartbeat provides the same keep-alive functionality
+   *
+   * @param containerId the relevant container id
+   */
   void containerAlive(ContainerId containerId);
 
+  /**
+   * Inform the framework that the task has started execution
+   *
+   * @param taskAttemptId the relevant task attempt id
+   * @param containerId   the containerId in which the task attempt is running
+   */
   void taskStartedRemotely(TezTaskAttemptID taskAttemptId, ContainerId containerId);
 
-  void taskKilled(TezTaskAttemptID taskAttemptId, TaskAttemptEndReason taskAttemptEndReason, @Nullable String diagnostics);
+  /**
+   * Inform the framework that a task has been killed
+   *
+   * @param taskAttemptId        the relevant task attempt id
+   * @param taskAttemptEndReason the reason for the task attempt being killed
+   * @param diagnostics          any diagnostics messages which are relevant to the task attempt
+   *                             kill
+   */
+  void taskKilled(TezTaskAttemptID taskAttemptId, TaskAttemptEndReason taskAttemptEndReason,
+                  @Nullable String diagnostics);
 
-  void taskFailed(TezTaskAttemptID taskAttemptId, TaskAttemptEndReason taskAttemptEndReason, @Nullable String diagnostics);
+  /**
+   * Inform the framework that a task has failed
+   *
+   * @param taskAttemptId        the relevant task attempt id
+   * @param taskAttemptEndReason the reason for the task failure
+   * @param diagnostics          any diagnostics messages which are relevant to the task attempt
+   *                             failure
+   */
+  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>
-   *
+   * via {@link org.apache.tez.runtime.api.InputInitializer#onVertexStateUpdated(org.apache.tez.dag.api.event.VertexStateUpdate)}
+   * </p>
+   * <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.
@@ -76,6 +157,7 @@ public interface TaskCommunicatorContext {
 
   /**
    * Get the name of the currently executing dag
+   *
    * @return the name of the currently executing dag
    */
   String getCurretnDagName();
@@ -83,6 +165,7 @@ public interface TaskCommunicatorContext {
   /**
    * 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
    */
@@ -90,13 +173,15 @@ public interface TaskCommunicatorContext {
 
   /**
    * Get the total number of tasks in the given vertex
-   * @param vertexName
+   *
+   * @param vertexName the relevant vertex name
    * @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
    */
@@ -104,6 +189,7 @@ public interface TaskCommunicatorContext {
 
   /**
    * Get the number of running tasks for a given vertex
+   *
    * @param vertexName the vertex name
    * @return the number of running tasks for the vertex
    */
@@ -111,14 +197,16 @@ public interface TaskCommunicatorContext {
 
   /**
    * 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
+   * @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/efea3a55/tez-dag/src/main/java/org/apache/tez/dag/app/ContainerLauncherContextImpl.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/ContainerLauncherContextImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/ContainerLauncherContextImpl.java
index 92bbbdc..3a2efc5 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/ContainerLauncherContextImpl.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/ContainerLauncherContextImpl.java
@@ -29,6 +29,7 @@ import org.apache.tez.dag.app.rm.container.AMContainerEventType;
 import org.apache.tez.dag.history.DAGHistoryEvent;
 import org.apache.tez.dag.history.events.ContainerLaunchedEvent;
 
+@SuppressWarnings("unchecked")
 public class ContainerLauncherContextImpl implements ContainerLauncherContext {
 
   private final AppContext context;

http://git-wip-us.apache.org/repos/asf/tez/blob/efea3a55/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 6c1dad9..ad6f2c4 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
@@ -178,13 +178,7 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
           taskCommClazz.getConstructor(TaskCommunicatorContext.class);
       ctor.setAccessible(true);
       return ctor.newInstance(taskCommunicatorContext);
-    } catch (NoSuchMethodException e) {
-      throw new TezUncheckedException(e);
-    } catch (InvocationTargetException e) {
-      throw new TezUncheckedException(e);
-    } catch (InstantiationException e) {
-      throw new TezUncheckedException(e);
-    } catch (IllegalAccessException e) {
+    } catch (NoSuchMethodException | InvocationTargetException | InstantiationException | IllegalAccessException e) {
       throw new TezUncheckedException(e);
     }
   }
@@ -398,13 +392,6 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
           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()));
 

http://git-wip-us.apache.org/repos/asf/tez/blob/efea3a55/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 57b4aee..d0cee21 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
@@ -62,7 +62,7 @@ public class ContainerLauncherRouter extends AbstractService
     containerLaunchers = new ContainerLauncher[] {containerLauncher};
     containerLauncherContexts = new ContainerLauncherContext[] {containerLauncher.getContext()};
     containerLauncherServiceWrappers = new ServicePluginLifecycleAbstractService[]{
-        new ServicePluginLifecycleAbstractService(containerLauncher)};
+        new ServicePluginLifecycleAbstractService<>(containerLauncher)};
   }
 
   // Accepting conf to setup final parameters, if required.
@@ -89,7 +89,7 @@ public class ContainerLauncherRouter extends AbstractService
       containerLauncherContexts[i] = containerLauncherContext;
       containerLaunchers[i] = createContainerLauncher(containerLauncherDescriptors.get(i), context,
           containerLauncherContext, taskAttemptListener, workingDirectory, i, isPureLocalMode);
-      containerLauncherServiceWrappers[i] = new ServicePluginLifecycleAbstractService(containerLaunchers[i]);
+      containerLauncherServiceWrappers[i] = new ServicePluginLifecycleAbstractService<>(containerLaunchers[i]);
     }
   }
 
@@ -138,6 +138,7 @@ public class ContainerLauncherRouter extends AbstractService
   }
 
   @VisibleForTesting
+  @SuppressWarnings("unchecked")
   ContainerLauncher createCustomContainerLauncher(ContainerLauncherContext containerLauncherContext,
                                                   NamedEntityDescriptor containerLauncherDescriptor) {
     LOG.info("Creating container launcher {}:{} ", containerLauncherDescriptor.getEntityName(),
@@ -150,15 +151,10 @@ public class ContainerLauncherRouter extends AbstractService
           .getConstructor(ContainerLauncherContext.class);
       ctor.setAccessible(true);
       return ctor.newInstance(containerLauncherContext);
-    } catch (NoSuchMethodException e) {
-      throw new TezUncheckedException(e);
-    } catch (InvocationTargetException e) {
-      throw new TezUncheckedException(e);
-    } catch (InstantiationException e) {
-      throw new TezUncheckedException(e);
-    } catch (IllegalAccessException e) {
+    } catch (NoSuchMethodException | InvocationTargetException | InstantiationException | IllegalAccessException e) {
       throw new TezUncheckedException(e);
     }
+
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/tez/blob/efea3a55/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerContextImpl.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerContextImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerContextImpl.java
index 7f1d5a3..2a9797f 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerContextImpl.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerContextImpl.java
@@ -57,6 +57,9 @@ public class TaskSchedulerContextImpl implements TaskSchedulerContext {
 
   }
 
+  // this may end up being called for a task+container pair that the app
+  // has not heard about. this can happen because of a race between
+  // taskAllocated() upcall and deallocateTask() downcall
   @Override
   public void taskAllocated(Object task, Object appCookie, Container container) {
     tseh.taskAllocated(schedulerId, task, appCookie, container);

http://git-wip-us.apache.org/repos/asf/tez/blob/efea3a55/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 7c36232..f001909 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
@@ -144,6 +144,7 @@ public class TaskSchedulerEventHandler extends AbstractService implements
    * @param webUI
    * @param schedulerDescriptors the list of scheduler descriptors. Tez internal classes will not have the class names populated.
    *                         An empty list defaults to using the YarnTaskScheduler as the only source.
+   * @param isPureLocalMode whether the AM is running in local mode
    */
   @SuppressWarnings("rawtypes")
   public TaskSchedulerEventHandler(AppContext appContext,
@@ -423,6 +424,7 @@ public class TaskSchedulerEventHandler extends AbstractService implements
     return new LocalTaskSchedulerService(taskSchedulerContext);
   }
 
+  @SuppressWarnings("unchecked")
   TaskScheduler createCustomTaskScheduler(TaskSchedulerContext taskSchedulerContext,
                                           NamedEntityDescriptor taskSchedulerDescriptor,
                                           int schedulerId) {
@@ -436,13 +438,7 @@ public class TaskSchedulerEventHandler extends AbstractService implements
           .getConstructor(TaskSchedulerContext.class);
       ctor.setAccessible(true);
       return ctor.newInstance(taskSchedulerContext);
-    } catch (NoSuchMethodException e) {
-      throw new TezUncheckedException(e);
-    } catch (InvocationTargetException e) {
-      throw new TezUncheckedException(e);
-    } catch (InstantiationException e) {
-      throw new TezUncheckedException(e);
-    } catch (IllegalAccessException e) {
+    } catch (NoSuchMethodException | InvocationTargetException | InstantiationException | IllegalAccessException e) {
       throw new TezUncheckedException(e);
     }
   }
@@ -453,7 +449,7 @@ public class TaskSchedulerEventHandler extends AbstractService implements
     int j = 0;
     for (int i = 0; i < taskSchedulerDescriptors.length; i++) {
       long customAppIdIdentifier;
-      if (isPureLocalMode || taskSchedulerDescriptors[i].equals(
+      if (isPureLocalMode || taskSchedulerDescriptors[i].getEntityName().equals(
           TezConstants.getTezYarnServicePluginName())) { // Use the app identifier from the appId.
         customAppIdIdentifier = appContext.getApplicationID().getClusterTimestamp();
       } else {
@@ -463,7 +459,7 @@ public class TaskSchedulerEventHandler extends AbstractService implements
           customAppIdIdentifier);
       taskSchedulers[i] = createTaskScheduler(host, port,
           trackingUrl, appContext, taskSchedulerDescriptors[i], customAppIdIdentifier, i);
-      taskSchedulerServiceWrappers[i] = new ServicePluginLifecycleAbstractService(taskSchedulers[i]);
+      taskSchedulerServiceWrappers[i] = new ServicePluginLifecycleAbstractService<>(taskSchedulers[i]);
     }
   }
 
@@ -741,7 +737,7 @@ public class TaskSchedulerEventHandler extends AbstractService implements
   public boolean hasUnregistered() {
     boolean result = true;
     for (int i = 0 ; i < taskSchedulers.length ; i++) {
-      result |= this.taskSchedulers[i].hasUnregistered();
+      result = result & this.taskSchedulers[i].hasUnregistered();
       if (result == false) {
         return result;
       }

http://git-wip-us.apache.org/repos/asf/tez/blob/efea3a55/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezTaskRunner2.java
----------------------------------------------------------------------
diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezTaskRunner2.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezTaskRunner2.java
index 1a8828d..d8539c5 100644
--- a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezTaskRunner2.java
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezTaskRunner2.java
@@ -122,7 +122,7 @@ public class TezTaskRunner2 {
    * the AM - since a task KILL is an external event, and whoever invoked it should
    * be able to track it.
    *
-   * @return
+   * @return the taskRunner result
    */
   public TaskRunner2Result run() {
     try {


[42/50] [abbrv] tez git commit: TEZ-2626. Fix log lines with DEBUG in messages, consolidate TEZ-2003 TODOs. (sseth)

Posted by ss...@apache.org.
TEZ-2626. Fix log lines with DEBUG in messages, consolidate TEZ-2003 TODOs. (sseth)


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

Branch: refs/heads/TEZ-2003
Commit: f30a3fe3751666505ebdfe39bb7e1f32d2b09736
Parents: e445576
Author: Siddharth Seth <ss...@apache.org>
Authored: Thu Jul 30 13:39:40 2015 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Fri Aug 14 13:47:10 2015 -0700

----------------------------------------------------------------------
 TEZ-2003-CHANGES.txt                            |  1 +
 pom.xml                                         |  2 +-
 .../serviceplugins/api/ContainerLauncher.java   |  4 ---
 .../tez/serviceplugins/api/TaskScheduler.java   |  5 ++++
 .../api/TaskSchedulerContext.java               |  6 ++--
 .../apache/tez/dag/api/TaskCommunicator.java    | 29 +++++++-------------
 .../tez/dag/api/TaskCommunicatorContext.java    | 24 ++++++++--------
 .../tez/dag/api/TaskHeartbeatRequest.java       |  6 ++--
 .../tez/dag/api/TaskHeartbeatResponse.java      |  2 +-
 .../dag/app/TaskAttemptListenerImpTezDag.java   |  7 ++---
 .../dag/app/TaskCommunicatorContextImpl.java    |  2 +-
 .../tez/dag/app/TezTaskCommunicatorImpl.java    |  3 +-
 .../apache/tez/dag/app/dag/impl/DAGImpl.java    |  2 +-
 .../tez/dag/app/dag/impl/TaskAttemptImpl.java   |  2 +-
 .../dag/app/rm/TaskSchedulerEventHandler.java   |  6 ----
 .../rm/container/AMContainerEventAssignTA.java  |  2 --
 tez-ext-service-tests/pom.xml                   |  1 -
 .../TezTestServiceContainerLauncher.java        |  4 +--
 .../TezTestServiceNoOpContainerLauncher.java    |  2 +-
 .../rm/TezTestServiceTaskSchedulerService.java  |  6 ++--
 .../tez/service/impl/ContainerRunnerImpl.java   |  6 ++--
 .../tez/tests/TestExternalTezServices.java      |  2 --
 .../internals/api/TaskReporterInterface.java    |  4 +--
 23 files changed, 51 insertions(+), 77 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/f30a3fe3/TEZ-2003-CHANGES.txt
----------------------------------------------------------------------
diff --git a/TEZ-2003-CHANGES.txt b/TEZ-2003-CHANGES.txt
index 9b3967a..c7a3dcc 100644
--- a/TEZ-2003-CHANGES.txt
+++ b/TEZ-2003-CHANGES.txt
@@ -41,5 +41,6 @@ ALL CHANGES:
   TEZ-2653. Change service contexts to expose a user specified payload instead of the AM configuration.
   TEZ-2441. Add tests for TezTaskRunner2.
   TEZ-2657. Add tests for client side changes - specifying plugins, etc.
+  TEZ-2626. Fix log lines with DEBUG in messages, consolidate TEZ-2003 TODOs.
 
 INCOMPATIBLE CHANGES:

http://git-wip-us.apache.org/repos/asf/tez/blob/f30a3fe3/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 1ced4e7..7ae5f31 100644
--- a/pom.xml
+++ b/pom.xml
@@ -682,10 +682,10 @@
     <module>tez-examples</module>
     <module>tez-tests</module>
     <module>tez-dag</module>
+    <module>tez-ext-service-tests</module>
     <module>tez-ui</module>
     <module>tez-plugins</module>
     <module>tez-tools</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/f30a3fe3/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ContainerLauncher.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ContainerLauncher.java b/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ContainerLauncher.java
index 8337dcb..7f58f77 100644
--- a/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ContainerLauncher.java
+++ b/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ContainerLauncher.java
@@ -16,7 +16,6 @@ package org.apache.tez.serviceplugins.api;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.service.AbstractService;
 import org.apache.tez.common.ServicePluginLifecycle;
 
 /**
@@ -30,9 +29,6 @@ public abstract class ContainerLauncher implements ServicePluginLifecycle {
 
   private final ContainerLauncherContext containerLauncherContext;
 
-  // TODO TEZ-2003 Simplify this by moving away from AbstractService. Potentially Guava AbstractService.
-  // A serviceInit(Configuration) is not likely to be very useful, and will expose unnecessary internal
-  // configuration to the services if populated with the AM Configuration
   public ContainerLauncher(ContainerLauncherContext containerLauncherContext) {
     this.containerLauncherContext = containerLauncherContext;
   }

http://git-wip-us.apache.org/repos/asf/tez/blob/f30a3fe3/tez-api/src/main/java/org/apache/tez/serviceplugins/api/TaskScheduler.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/serviceplugins/api/TaskScheduler.java b/tez-api/src/main/java/org/apache/tez/serviceplugins/api/TaskScheduler.java
index a5b054f..9ff2bd5 100644
--- a/tez-api/src/main/java/org/apache/tez/serviceplugins/api/TaskScheduler.java
+++ b/tez-api/src/main/java/org/apache/tez/serviceplugins/api/TaskScheduler.java
@@ -27,6 +27,11 @@ import org.apache.tez.common.ServicePluginLifecycle;
 @InterfaceStability.Unstable
 public abstract class TaskScheduler implements ServicePluginLifecycle {
 
+  // TODO TEZ-2003 (post) TEZ-2668
+  // - Should setRegister / unregister be part of APIs when not YARN specific ?
+  // - Include vertex / task information in therequest so that the scheduler can make decisions
+  // around prioritizing tasks in the same vertex when others exist at the same priority.
+
   private final TaskSchedulerContext taskSchedulerContext;
 
   public TaskScheduler(TaskSchedulerContext taskSchedulerContext) {

http://git-wip-us.apache.org/repos/asf/tez/blob/f30a3fe3/tez-api/src/main/java/org/apache/tez/serviceplugins/api/TaskSchedulerContext.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/serviceplugins/api/TaskSchedulerContext.java b/tez-api/src/main/java/org/apache/tez/serviceplugins/api/TaskSchedulerContext.java
index 6f37641..dbbf75c 100644
--- a/tez-api/src/main/java/org/apache/tez/serviceplugins/api/TaskSchedulerContext.java
+++ b/tez-api/src/main/java/org/apache/tez/serviceplugins/api/TaskSchedulerContext.java
@@ -53,7 +53,10 @@ public interface TaskSchedulerContext {
     IDLE, RUNNING_APP, COMPLETED
   }
 
-  // TODO Post TEZ-2003. Remove references to YARN constructs like Container, ContainerStatus, NodeReport
+  // TODO TEZ-2003 (post) TEZ-2664. Remove references to YARN constructs like Container, ContainerStatus, NodeReport
+  // TODO TEZ-2003 (post) TEZ-2668 Enhancements to TaskScheduler interfaces
+  // - setApplicationRegistrationData may not be relevant to non YARN clusters
+  // - getAppFinalStatus may not be relevant to non YARN clusters
   // upcall to app must be outside locks
   public void taskAllocated(Object task,
                             Object appCookie,
@@ -78,7 +81,6 @@ public interface TaskSchedulerContext {
   public float getProgress();
   public void preemptContainer(ContainerId containerId);
 
-  // TODO Post TEZ-2003. Another method which is primarily relevant to YARN clusters for unregistration.
   public AppFinalStatus getFinalAppStatus();
 
 

http://git-wip-us.apache.org/repos/asf/tez/blob/f30a3fe3/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 f221414..794d390 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
@@ -27,9 +27,18 @@ import org.apache.tez.serviceplugins.api.TaskAttemptEndReason;
 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
+// TODO TEZ-2003 (post) TEZ-2665. Move to the tez-api module
+// TODO TEZ-2003 (post) TEZ-2664. Ideally, don't expose YARN containerId; instead expose a Tez specific construct.
 public abstract class TaskCommunicator implements ServicePluginLifecycle {
 
+  // TODO TEZ-2003 (post) TEZ-2666 Enhancements to interface
+  // - registerContainerEnd should provide the end reason / possible rename
+  // - get rid of getAddress
+  // - Add methods to support task preemption
+  // - Add a dagStarted notification, along with a payload
+  // - taskSpec breakup into a clean interface
+  // - Add methods to report task / container completion
+
   private final TaskCommunicatorContext taskCommunicatorContext;
 
   public TaskCommunicator(TaskCommunicatorContext taskCommunicatorContext) {
@@ -52,36 +61,20 @@ public abstract class TaskCommunicator implements ServicePluginLifecycle {
   public void shutdown() throws Exception {
   }
 
-  // TODO Post TEZ-2003 Move this into the API module. Moving this requires abstractions for
-  // TaskSpec and related classes. (assuming that's efficient for execution)
 
-  // 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, ContainerEndReason endReason);
 
-  // TODO TEZ-2003 Provide additional inforamtion like reason for container end / Task end.
-  // Was it caused by preemption - or as a result of a general task completion / container completion
-
-  // 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, 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, TaskAttemptEndReason endReason);
 
-  // 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.
-
   /**
    * Receive notifications on vertex state changes.
    * <p/>
@@ -108,8 +101,6 @@ public abstract class TaskCommunicator implements ServicePluginLifecycle {
    * 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/f30a3fe3/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 a1e94a3..8073f6a 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
@@ -28,23 +28,27 @@ 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
+// TODO TEZ-2003 (post) TEZ-2665. Move to 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.
+  // TODO TEZ-2003 (post) TEZ-2666 Enhancements to API
+  // - Consolidate usage of IDs
+  // - Split the heartbeat API to a liveness check and a status update
+  // - Rename and consolidate TaskHeartbeatResponse and TaskHeartbeatRequest
+  // - Fix taskStarted needs to be invoked before launching the actual task.
+  // - Potentially add methods to report availability stats to the scheduler
+  // - Report taskSuccess via a method instead of the heartbeat
+  // - Add methods to signal container / task state changes
+  // - Maybe add book-keeping as a helper library, instead of each impl tracking container to task etc.
+  // - Handling of containres / tasks which no longer exist in the system (formalized interface instead of a shouldDie notification)
 
   UserPayload getInitialUserPayload();
 
   ApplicationAttemptId getApplicationAttemptId();
   Credentials getCredentials();
 
-  // 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
-  // KKK Rename this API
   TaskHeartbeatResponse heartbeat(TaskHeartbeatRequest request) throws IOException, TezException;
 
   boolean isKnownContainer(ContainerId containerId);
@@ -53,13 +57,10 @@ public interface TaskCommunicatorContext {
 
   void containerAlive(ContainerId containerId);
 
-  // TODO TEZ-2003 Move to vertex, taskIndex, version. Rename to taskAttempt*
   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);
 
-  // TODO TEZ-2003 Move to vertex, taskIndex, version. Rename to taskAttempt*
   void taskFailed(TezTaskAttemptID taskAttemptId, TaskAttemptEndReason taskAttemptEndReason, @Nullable String diagnostics);
 
   /**
@@ -72,9 +73,6 @@ public interface TaskCommunicatorContext {
    * @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.
 
   /**
    * Get the name of the currently executing dag

http://git-wip-us.apache.org/repos/asf/tez/blob/f30a3fe3/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
index b5ff991..d0c22d3 100644
--- 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
@@ -19,13 +19,11 @@ 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
+// TODO TEZ-2003 (post) TEZ-2665. Move to 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
+  // TODO TEZ-2003 (post) TEZ-2666 Ideally containerIdentifier should not be part of the request.
   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;

http://git-wip-us.apache.org/repos/asf/tez/blob/f30a3fe3/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
index 7f063c4..dcf89ff 100644
--- 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
@@ -18,7 +18,7 @@ import java.util.List;
 
 import org.apache.tez.runtime.api.impl.TezEvent;
 
-// TODO TEZ-2003 Move this into the tez-api module
+// TODO TEZ-2003 (post) TEZ-2665. Move to the tez-api module
 public class TaskHeartbeatResponse {
 
   private final boolean shouldDie;

http://git-wip-us.apache.org/repos/asf/tez/blob/f30a3fe3/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 cc109a6..941e583 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
@@ -162,7 +162,6 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
       LOG.info("Using Default Local Task Communicator");
       return new TezLocalTaskCommunicatorImpl(taskCommunicatorContexts[taskCommIndex]);
     } else {
-      // TODO TEZ-2003. Use the payload
       LOG.info("Using TaskCommunicator {}:{} " + taskCommDescriptor.getEntityName(), taskCommDescriptor.getClassName());
       Class<? extends TaskCommunicator> taskCommClazz = (Class<? extends TaskCommunicator>) ReflectionUtils
           .getClazz(taskCommDescriptor.getClassName());
@@ -217,7 +216,7 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
         // 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,
+        // TODO TEZ-2003 (post) TEZ-2666. 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");
@@ -278,7 +277,7 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
                          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,
+    // TODO TEZ-2003 (post) TEZ-2671 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,
@@ -290,7 +289,7 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
                          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,
+    // TODO TEZ-2003 (post) TEZ-2671 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,

http://git-wip-us.apache.org/repos/asf/tez/blob/f30a3fe3/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 cc315b7..0f10305 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
@@ -43,6 +43,7 @@ import org.apache.tez.dag.records.TezTaskAttemptID;
 @InterfaceAudience.Private
 public class TaskCommunicatorContextImpl implements TaskCommunicatorContext, VertexStateUpdateListener {
 
+  // TODO TEZ-2003 (post) TEZ-2669 Propagate errors baack to the AM with proper error reporting
 
   private final AppContext context;
   private final TaskAttemptListenerImpTezDag taskAttemptListener;
@@ -188,7 +189,6 @@ public class TaskCommunicatorContextImpl implements TaskCommunicatorContext, Ver
     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/f30a3fe3/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 2a5c80e..fb6d5e7 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
@@ -463,9 +463,8 @@ public class TezTaskCommunicatorImpl extends TaskCommunicator {
 
 
   // Holder for Task information, which eventually will likely be VertexImplm taskIndex, attemptIndex
+  // TODO TEZ-2003. TEZ-2670. Remove this class.
   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;
 
     TaskAttempt(TezTaskAttemptID taskAttemptId) {

http://git-wip-us.apache.org/repos/asf/tez/blob/f30a3fe3/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 25518b0..17f5675 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
@@ -1453,7 +1453,7 @@ public class DAGImpl implements org.apache.tez.dag.app.dag.DAG,
     // check task resources, only check it in non-local mode
     if (!appContext.isLocal()) {
       for (Vertex v : vertexMap.values()) {
-        // TODO TEZ-2003 (post) Ideally, this should be per source.
+        // TODO TEZ-2003 (post) TEZ-2624 Ideally, this should be per source.
         if (v.getTaskResource().compareTo(appContext.getClusterInfo().getMaxContainerCapability()) > 0) {
           String msg = "Vertex's TaskResource is beyond the cluster container capability," +
               "Vertex=" + v.getLogIdentifier() +", Requested TaskResource=" + v.getTaskResource()

http://git-wip-us.apache.org/repos/asf/tez/blob/f30a3fe3/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 cb26c55..65ea3fb 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
@@ -191,7 +191,7 @@ 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
+  // TODO TEZ-2003 (post) TEZ-2667 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
 

http://git-wip-us.apache.org/repos/asf/tez/blob/f30a3fe3/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 4c2e631..c86f638 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
@@ -502,7 +502,6 @@ public class TaskSchedulerEventHandler extends AbstractService implements
       taskSchedulerServiceWrappers[i].start();
       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();
       }
@@ -564,8 +563,6 @@ public class TaskSchedulerEventHandler extends AbstractService implements
     appCallbackExecutor.awaitTermination(1000l, TimeUnit.MILLISECONDS);
   }
 
-  // TODO TEZ-2003 Consolidate TaskSchedulerAppCallback methods once these methods are moved into context
-
   // TaskSchedulerAppCallback methods with schedulerId, where relevant
   public synchronized void taskAllocated(int schedulerId, Object task,
                                            Object appCookie,
@@ -651,7 +648,6 @@ public class TaskSchedulerEventHandler extends AbstractService implements
       Resource maxContainerCapability,
       Map<ApplicationAccessType, String> appAcls, 
       ByteBuffer clientAMSecretKey) {
-    // TODO TEZ-2003 (post) Ideally clusterInfo should be available per source rather than a global view.
     this.appContext.getClusterInfo().setMaxContainerCapability(
         maxContainerCapability);
     this.appAcls = appAcls;
@@ -751,7 +747,6 @@ public class TaskSchedulerEventHandler extends AbstractService implements
     this.shouldUnregisterFlag.set(true);
     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();
       }
     }
@@ -764,7 +759,6 @@ public class TaskSchedulerEventHandler extends AbstractService implements
   public boolean 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;

http://git-wip-us.apache.org/repos/asf/tez/blob/f30a3fe3/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 0398882..682cd02 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,8 +27,6 @@ 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/f30a3fe3/tez-ext-service-tests/pom.xml
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/pom.xml b/tez-ext-service-tests/pom.xml
index 907e129..f95f4ca 100644
--- a/tez-ext-service-tests/pom.xml
+++ b/tez-ext-service-tests/pom.xml
@@ -23,7 +23,6 @@
     <version>0.8.0-TEZ-2003-SNAPSHOT</version>
   </parent>
 
-  <!-- TODO TEZ-2003 Merge this into the tez-tests module -->
   <artifactId>tez-ext-service-tests</artifactId>
 
   <dependencies>

http://git-wip-us.apache.org/repos/asf/tez/blob/f30a3fe3/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 f31a07b..845a27b 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
@@ -37,8 +37,6 @@ import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos.RunContainer
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-// TODO TEZ-2003 look for all LOG.*(DBG and LOG.*(DEBUG messages
-
 public class TezTestServiceContainerLauncher extends ContainerLauncher {
 
   // TODO Support interruptability of tasks which haven't yet been launched.
@@ -119,7 +117,7 @@ public class TezTestServiceContainerLauncher extends ContainerLauncher {
 
   @Override
   public void stopContainer(ContainerStopRequest stopRequest) {
-    LOG.info("DEBUG: Ignoring STOP_REQUEST for event: " + stopRequest);
+    LOG.info("Ignoring stopContainer for event: " + stopRequest);
     // that the container is actually done (normally received from RM)
     // TODO Sending this out for an un-launched container is invalid
     getContext().containerStopRequested(stopRequest.getContainerId());

http://git-wip-us.apache.org/repos/asf/tez/blob/f30a3fe3/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 7b42296..d265736 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
@@ -39,7 +39,7 @@ public class TezTestServiceNoOpContainerLauncher extends ContainerLauncher {
 
   @Override
   public void stopContainer(ContainerStopRequest stopRequest) {
-    LOG.info("DEBUG: Ignoring STOP_REQUEST {}", stopRequest);
+    LOG.info("Ignoring stopRequest {}", stopRequest);
     getContext().containerStopRequested(stopRequest.getContainerId());
   }
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/f30a3fe3/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 0d87995..17f8a87 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
@@ -151,12 +151,12 @@ public class TezTestServiceTaskSchedulerService extends TaskScheduler {
 
   @Override
   public void blacklistNode(NodeId nodeId) {
-    LOG.info("DEBUG: BlacklistNode not supported");
+    LOG.info("BlacklistNode not supported");
   }
 
   @Override
   public void unblacklistNode(NodeId nodeId) {
-    LOG.info("DEBUG: unBlacklistNode not supported");
+    LOG.info("unBlacklistNode not supported");
   }
 
   @Override
@@ -195,7 +195,7 @@ public class TezTestServiceTaskSchedulerService extends TaskScheduler {
 
   @Override
   public Object deallocateContainer(ContainerId containerId) {
-    LOG.info("DEBUG: Ignoring deallocateContainer for containerId: " + containerId);
+    LOG.info("Ignoring deallocateContainer for containerId: " + containerId);
     return null;
   }
 

http://git-wip-us.apache.org/repos/asf/tez/blob/f30a3fe3/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 f3fc442..472a43c 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
@@ -173,7 +173,7 @@ public class ContainerRunnerImpl extends AbstractService implements ContainerRun
         throw new TezException(e);
       }
     }
-    LOG.info("DEBUG: Dirs are: " + Arrays.toString(localDirs));
+    LOG.info("Dirs for {} are {}", request.getContainerIdString(), Arrays.toString(localDirs));
 
 
     // Setup workingDir. This is otherwise setup as Environment.PWD
@@ -193,7 +193,7 @@ public class ContainerRunnerImpl extends AbstractService implements ContainerRun
     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");
+    LOG.info("Registering request with the ShuffleHandler for containerId {}", request.getContainerIdString());
     ShuffleHandler.get().registerApplication(request.getApplicationIdString(), jobToken, request.getUser());
 
 
@@ -255,7 +255,7 @@ public class ContainerRunnerImpl extends AbstractService implements ContainerRun
     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");
+    LOG.info("Registering request with the ShuffleHandler for containerId {}", request.getContainerIdString());
     ShuffleHandler.get().registerApplication(request.getApplicationIdString(), jobToken, request.getUser());
     TaskRunnerCallable callable = new TaskRunnerCallable(request, new Configuration(getConfig()),
         new ExecutionContextImpl(localAddress.get().getHostName()), env, localDirs,

http://git-wip-us.apache.org/repos/asf/tez/blob/f30a3fe3/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 2c52ae3..3701455 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
@@ -130,8 +130,6 @@ public class TestExternalTezServices {
       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

http://git-wip-us.apache.org/repos/asf/tez/blob/f30a3fe3/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
index 47a61ab..9a5a3ab 100644
--- 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
@@ -26,8 +26,6 @@ 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);
@@ -43,4 +41,4 @@ public interface TaskReporterInterface {
 
   void shutdown();
 
-}
+}
\ No newline at end of file


[26/50] [abbrv] tez git commit: TEZ-2508. rebase 06/01. (sseth)

Posted by ss...@apache.org.
TEZ-2508. rebase 06/01. (sseth)


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

Branch: refs/heads/TEZ-2003
Commit: dcd767e61047c6cbab845a518f1af050c63bb5b1
Parents: 2a3e2b3
Author: Siddharth Seth <ss...@apache.org>
Authored: Mon Jun 1 16:37:26 2015 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Fri Aug 14 13:46:44 2015 -0700

----------------------------------------------------------------------
 TEZ-2003-CHANGES.txt                                         | 1 +
 .../java/org/apache/tez/dag/api/TaskHeartbeatRequest.java    | 7 +++++++
 .../java/org/apache/tez/dag/api/TaskHeartbeatResponse.java   | 8 +++++++-
 .../org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java | 8 ++++----
 .../java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java | 3 ++-
 .../apache/tez/runtime/LogicalIOProcessorRuntimeTask.java    | 3 ---
 6 files changed, 21 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/dcd767e6/TEZ-2003-CHANGES.txt
----------------------------------------------------------------------
diff --git a/TEZ-2003-CHANGES.txt b/TEZ-2003-CHANGES.txt
index 42c2e1e..55002fe 100644
--- a/TEZ-2003-CHANGES.txt
+++ b/TEZ-2003-CHANGES.txt
@@ -30,5 +30,6 @@ ALL CHANGES:
   TEZ-2471. NPE in LogicalIOProcessorRuntimeTask while printing thread info.
   TEZ-2495. Inform TaskCommunicaor about Task and Container termination reasons.
   TEZ-2502. Fix for TezTaskRunner2 not killing tasks properly in all situations.
+  TEZ-2508. rebase 06/01
 
 INCOMPATIBLE CHANGES:

http://git-wip-us.apache.org/repos/asf/tez/blob/dcd767e6/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
index f6bc8f0..b5ff991 100644
--- 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
@@ -29,15 +29,18 @@ public class TaskHeartbeatRequest {
   private final TezTaskAttemptID taskAttemptId;
   private final List<TezEvent> events;
   private final int startIndex;
+  private final int preRoutedStartIndex;
   private final int maxEvents;
 
 
   public TaskHeartbeatRequest(String containerIdentifier, TezTaskAttemptID taskAttemptId, List<TezEvent> events, int startIndex,
+                              int preRoutedStartIndex,
                               int maxEvents) {
     this.containerIdentifier = containerIdentifier;
     this.taskAttemptId = taskAttemptId;
     this.events = events;
     this.startIndex = startIndex;
+    this.preRoutedStartIndex = preRoutedStartIndex;
     this.maxEvents = maxEvents;
   }
 
@@ -57,6 +60,10 @@ public class TaskHeartbeatRequest {
     return startIndex;
   }
 
+  public int getPreRoutedStartIndex() {
+    return preRoutedStartIndex;
+  }
+
   public int getMaxEvents() {
     return maxEvents;
   }

http://git-wip-us.apache.org/repos/asf/tez/blob/dcd767e6/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
index b826e76..7f063c4 100644
--- 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
@@ -23,12 +23,14 @@ public class TaskHeartbeatResponse {
 
   private final boolean shouldDie;
   private final int nextFromEventId;
+  private final int nextPreRoutedEventId;
   private final List<TezEvent> events;
 
-  public TaskHeartbeatResponse(boolean shouldDie, List<TezEvent> events, int nextFromEventId) {
+  public TaskHeartbeatResponse(boolean shouldDie, List<TezEvent> events, int nextFromEventId, int nextPreRoutedEventId) {
     this.shouldDie = shouldDie;
     this.events = events;
     this.nextFromEventId = nextFromEventId;
+    this.nextPreRoutedEventId = nextPreRoutedEventId;
   }
 
   public boolean isShouldDie() {
@@ -42,4 +44,8 @@ public class TaskHeartbeatResponse {
   public int getNextFromEventId() {
     return nextFromEventId;
   }
+
+  public int getNextPreRoutedEventId() {
+    return nextPreRoutedEventId;
+  }
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/dcd767e6/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 1c61a0d..e2d44e2 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
@@ -79,7 +79,7 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
   protected final TaskHeartbeatHandler taskHeartbeatHandler;
   protected final ContainerHeartbeatHandler containerHeartbeatHandler;
 
-  private final TaskHeartbeatResponse RESPONSE_SHOULD_DIE = new TaskHeartbeatResponse(true, null, 0);
+  private final TaskHeartbeatResponse RESPONSE_SHOULD_DIE = new TaskHeartbeatResponse(true, null, 0, 0);
 
   private final ConcurrentMap<TezTaskAttemptID, ContainerId> registeredAttempts =
       new ConcurrentHashMap<TezTaskAttemptID, ContainerId>();
@@ -195,7 +195,7 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
     // So - avoiding synchronization.
 
     pingContainerHeartbeatHandler(containerId);
-    TaskAttemptEventInfo eventInfo = new TaskAttemptEventInfo(0, null);
+    TaskAttemptEventInfo eventInfo = new TaskAttemptEventInfo(0, null, 0);
     TezTaskAttemptID taskAttemptID = request.getTaskAttemptId();
     if (taskAttemptID != null) {
       ContainerId containerIdFromMap = registeredAttempts.get(taskAttemptID);
@@ -241,10 +241,10 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
       eventInfo = context
           .getCurrentDAG()
           .getVertex(taskAttemptID.getTaskID().getVertexID())
-          .getTaskAttemptTezEvents(taskAttemptID, request.getStartIndex(),
+          .getTaskAttemptTezEvents(taskAttemptID, request.getStartIndex(), request.getPreRoutedStartIndex(),
               request.getMaxEvents());
     }
-    return new TaskHeartbeatResponse(false, eventInfo.getEvents(), eventInfo.getNextFromEventId());
+    return new TaskHeartbeatResponse(false, eventInfo.getEvents(), eventInfo.getNextFromEventId(), eventInfo.getNextPreRoutedFromEventId());
   }
   public void taskAlive(TezTaskAttemptID taskAttemptId) {
     taskHeartbeatHandler.pinged(taskAttemptId);

http://git-wip-us.apache.org/repos/asf/tez/blob/dcd767e6/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 3774eb4..83322f2 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
@@ -363,13 +363,14 @@ public class TezTaskCommunicatorImpl extends TaskCommunicator {
         }
         TaskHeartbeatRequest tRequest = new TaskHeartbeatRequest(request.getContainerIdentifier(),
             request.getCurrentTaskAttemptID(), request.getEvents(), request.getStartIndex(),
-            request.getMaxEvents());
+            request.getPreRoutedStartIndex(), request.getMaxEvents());
         tResponse = taskCommunicatorContext.heartbeat(tRequest);
       }
       TezHeartbeatResponse response = new TezHeartbeatResponse();
       response.setLastRequestId(requestId);
       response.setEvents(tResponse.getEvents());
       response.setNextFromEventId(tResponse.getNextFromEventId());
+      response.setNextPreRoutedEventId(tResponse.getNextPreRoutedEventId());
       containerInfo.lastRequestId = requestId;
       containerInfo.lastResponse = response;
       return response;

http://git-wip-us.apache.org/repos/asf/tez/blob/dcd767e6/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 449fa0f..c79da5d 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
@@ -172,9 +172,6 @@ public class LogicalIOProcessorRuntimeTask extends RuntimeTask {
     this.outputsMap = new ConcurrentHashMap<String, LogicalOutput>(numOutputs);
     this.outputContextMap = new ConcurrentHashMap<String, OutputContext>(numOutputs);
 
-    this.initializedInputs = new ConcurrentHashMap<String, LogicalInput>();
-    this.initializedOutputs = new ConcurrentHashMap<String, LogicalOutput>();
-
     this.runInputMap = new LinkedHashMap<String, LogicalInput>();
     this.runOutputMap = new LinkedHashMap<String, LogicalOutput>();
 


[36/50] [abbrv] tez git commit: TEZ-2652. Cleanup the way services are specified for an AM and vertices. (sseth)

Posted by ss...@apache.org.
http://git-wip-us.apache.org/repos/asf/tez/blob/cfd625e9/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 7c6a6a4..594e6d3 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
@@ -17,18 +17,21 @@ package org.apache.tez.dag.app.launcher;
 import java.lang.reflect.Constructor;
 import java.lang.reflect.InvocationTargetException;
 import java.net.UnknownHostException;
+import java.util.List;
 
 import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.Lists;
 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.NamedEntityDescriptor;
+import org.apache.tez.dag.api.TezConstants;
 import org.apache.tez.dag.app.ServicePluginLifecycleAbstractService;
 import org.apache.tez.serviceplugins.api.ContainerLaunchRequest;
 import org.apache.tez.serviceplugins.api.ContainerLauncher;
 import org.apache.tez.serviceplugins.api.ContainerLauncherContext;
 import org.apache.tez.serviceplugins.api.ContainerStopRequest;
-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.ContainerLauncherContextImpl;
@@ -63,35 +66,35 @@ public class ContainerLauncherRouter extends AbstractService
   public ContainerLauncherRouter(Configuration conf, AppContext context,
                                  TaskAttemptListener taskAttemptListener,
                                  String workingDirectory,
-                                 String[] containerLauncherClassIdentifiers,
+                                 List<NamedEntityDescriptor> containerLauncherDescriptors,
                                  boolean isPureLocalMode) throws UnknownHostException {
     super(ContainerLauncherRouter.class.getName());
 
     this.appContext = context;
-    if (containerLauncherClassIdentifiers == null || containerLauncherClassIdentifiers.length == 0) {
+    if (containerLauncherDescriptors == null || containerLauncherDescriptors.isEmpty()) {
       if (isPureLocalMode) {
-        containerLauncherClassIdentifiers =
-            new String[]{TezConstants.TEZ_AM_SERVICE_PLUGINS_LOCAL_MODE_NAME_DEFAULT};
+        containerLauncherDescriptors = Lists.newArrayList(new NamedEntityDescriptor(
+            TezConstants.getTezUberServicePluginName(), null));
       } else {
-        containerLauncherClassIdentifiers =
-            new String[]{TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT};
+        containerLauncherDescriptors = Lists.newArrayList(new NamedEntityDescriptor(
+            TezConstants.getTezYarnServicePluginName(), null));
       }
     }
-    containerLauncherContexts = new ContainerLauncherContext[containerLauncherClassIdentifiers.length];
-    containerLaunchers = new ContainerLauncher[containerLauncherClassIdentifiers.length];
-    containerLauncherServiceWrappers = new ServicePluginLifecycleAbstractService[containerLauncherClassIdentifiers.length];
+    containerLauncherContexts = new ContainerLauncherContext[containerLauncherDescriptors.size()];
+    containerLaunchers = new ContainerLauncher[containerLauncherDescriptors.size()];
+    containerLauncherServiceWrappers = new ServicePluginLifecycleAbstractService[containerLauncherDescriptors.size()];
 
 
-    for (int i = 0; i < containerLauncherClassIdentifiers.length; i++) {
+    for (int i = 0; i < containerLauncherDescriptors.size(); i++) {
       ContainerLauncherContext containerLauncherContext = new ContainerLauncherContextImpl(context, taskAttemptListener);
       containerLauncherContexts[i] = containerLauncherContext;
-      containerLaunchers[i] = createContainerLauncher(containerLauncherClassIdentifiers[i], context,
+      containerLaunchers[i] = createContainerLauncher(containerLauncherDescriptors.get(i), context,
           containerLauncherContext, taskAttemptListener, workingDirectory, isPureLocalMode, conf);
       containerLauncherServiceWrappers[i] = new ServicePluginLifecycleAbstractService(containerLaunchers[i]);
     }
   }
 
-  private ContainerLauncher createContainerLauncher(String containerLauncherClassIdentifier,
+  private ContainerLauncher createContainerLauncher(NamedEntityDescriptor containerLauncherDescriptor,
                                                     AppContext context,
                                                     ContainerLauncherContext containerLauncherContext,
                                                     TaskAttemptListener taskAttemptListener,
@@ -99,11 +102,12 @@ public class ContainerLauncherRouter extends AbstractService
                                                     boolean isPureLocalMode,
                                                     Configuration conf) throws
       UnknownHostException {
-    if (containerLauncherClassIdentifier.equals(TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT)) {
+    if (containerLauncherDescriptor.getEntityName().equals(
+        TezConstants.getTezYarnServicePluginName())) {
       LOG.info("Creating DefaultContainerLauncher");
       return new ContainerLauncherImpl(containerLauncherContext);
-    } else if (containerLauncherClassIdentifier
-        .equals(TezConstants.TEZ_AM_SERVICE_PLUGINS_LOCAL_MODE_NAME_DEFAULT)) {
+    } else if (containerLauncherDescriptor.getEntityName()
+        .equals(TezConstants.getTezUberServicePluginName())) {
       LOG.info("Creating LocalContainerLauncher");
       // TODO Post TEZ-2003. LocalContainerLauncher is special cased, since it makes use of
       // extensive internals which are only available at runtime. Will likely require
@@ -111,10 +115,10 @@ public class ContainerLauncherRouter extends AbstractService
       return
           new LocalContainerLauncher(containerLauncherContext, context, taskAttemptListener, workingDirectory, isPureLocalMode);
     } else {
-      LOG.info("Creating container launcher : " + containerLauncherClassIdentifier);
+      LOG.info("Creating container launcher {}:{} ", containerLauncherDescriptor.getEntityName(), containerLauncherDescriptor.getClassName());
       Class<? extends ContainerLauncher> containerLauncherClazz =
           (Class<? extends ContainerLauncher>) ReflectionUtils.getClazz(
-              containerLauncherClassIdentifier);
+              containerLauncherDescriptor.getClassName());
       try {
         Constructor<? extends ContainerLauncher> ctor = containerLauncherClazz
             .getConstructor(ContainerLauncherContext.class);

http://git-wip-us.apache.org/repos/asf/tez/blob/cfd625e9/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 d8cf080..67a088e 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
@@ -22,6 +22,7 @@ import java.lang.reflect.Constructor;
 import java.lang.reflect.InvocationTargetException;
 import java.net.InetSocketAddress;
 import java.nio.ByteBuffer;
+import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
 import java.util.concurrent.BlockingQueue;
@@ -34,6 +35,8 @@ import java.util.concurrent.atomic.AtomicBoolean;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.util.concurrent.ThreadFactoryBuilder;
 import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.tez.dag.api.NamedEntityDescriptor;
+import org.apache.tez.dag.api.TezConstants;
 import org.apache.tez.dag.app.ServicePluginLifecycleAbstractService;
 import org.apache.tez.serviceplugins.api.TaskScheduler;
 import org.apache.tez.serviceplugins.api.TaskSchedulerContext;
@@ -55,7 +58,6 @@ 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;
@@ -143,14 +145,14 @@ public class TaskSchedulerEventHandler extends AbstractService implements
    * @param eventHandler
    * @param containerSignatureMatcher
    * @param webUI
-   * @param schedulerClasses the list of scheduler classes / codes. Tez internal classes are represented as codes.
+   * @param schedulerDescriptors the list of scheduler descriptors. Tez internal classes will not have the class names populated.
    *                         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,
-      String [] schedulerClasses, boolean isPureLocalMode) {
+      List<NamedEntityDescriptor> schedulerDescriptors, boolean isPureLocalMode) {
     super(TaskSchedulerEventHandler.class.getName());
     this.appContext = appContext;
     this.eventHandler = eventHandler;
@@ -166,31 +168,34 @@ public class TaskSchedulerEventHandler extends AbstractService implements
 
     // Override everything for pure local mode
     if (isPureLocalMode) {
-      this.taskSchedulerClasses = new String[] {TezConstants.TEZ_AM_SERVICE_PLUGINS_LOCAL_MODE_NAME_DEFAULT};
+      this.taskSchedulerClasses = new String[] {TezConstants.getTezUberServicePluginName()};
       this.yarnTaskSchedulerIndex = -1;
     } else {
-      if (schedulerClasses == null || schedulerClasses.length ==0) {
-        this.taskSchedulerClasses = new String[] {TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT};
+      if (schedulerDescriptors == null || schedulerDescriptors.isEmpty()) {
+        this.taskSchedulerClasses = new String[] {TezConstants.getTezYarnServicePluginName()};
         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)) {
+
+        List<String> taskSchedulerClassList = new LinkedList<>();
+        for (int i = 0 ; i < schedulerDescriptors.size() ; i++) {
+          if (schedulerDescriptors.get(i).getEntityName().equals(
+              TezConstants.getTezYarnServicePluginName())) {
+            taskSchedulerClassList.add(schedulerDescriptors.get(i).getEntityName());
             foundYarnTaskSchedulerIndex = i;
-            break;
+          } else if (schedulerDescriptors.get(i).getEntityName().equals(
+              TezConstants.getTezUberServicePluginName())) {
+            taskSchedulerClassList.add(schedulerDescriptors.get(i).getEntityName());
+          } else {
+            taskSchedulerClassList.add(schedulerDescriptors.get(i).getClassName());
           }
         }
-        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];
-          }
-          this.taskSchedulerClasses[foundYarnTaskSchedulerIndex] = TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT;
-        } else {
-          this.taskSchedulerClasses = schedulerClasses;
+        if (foundYarnTaskSchedulerIndex == -1) {
+          taskSchedulerClassList.add(YarnTaskSchedulerService.class.getName());
+          foundYarnTaskSchedulerIndex = taskSchedulerClassList.size() -1;
         }
+        this.taskSchedulerClasses = taskSchedulerClassList.toArray(new String[taskSchedulerClassList.size()]);
         this.yarnTaskSchedulerIndex = foundYarnTaskSchedulerIndex;
       }
     }
@@ -419,10 +424,10 @@ public class TaskSchedulerEventHandler extends AbstractService implements
         new TaskSchedulerContextImpl(this, appContext, schedulerId, trackingUrl,
             customAppIdIdentifier, host, port, getConfig());
     TaskSchedulerContext wrappedContext = new TaskSchedulerContextImplWrapper(rawContext, appCallbackExecutor);
-    if (schedulerClassName.equals(TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT)) {
+    if (schedulerClassName.equals(TezConstants.getTezYarnServicePluginName())) {
       LOG.info("Creating TaskScheduler: YarnTaskSchedulerService");
       return new YarnTaskSchedulerService(wrappedContext);
-    } else if (schedulerClassName.equals(TezConstants.TEZ_AM_SERVICE_PLUGINS_LOCAL_MODE_NAME_DEFAULT)) {
+    } else if (schedulerClassName.equals(TezConstants.getTezUberServicePluginName())) {
       LOG.info("Creating TaskScheduler: Local TaskScheduler");
       return new LocalTaskSchedulerService(wrappedContext);
     } else {
@@ -454,7 +459,7 @@ public class TaskSchedulerEventHandler extends AbstractService implements
     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.
+          TezConstants.getTezYarnServicePluginName())) { // Use the app identifier from the appId.
         customAppIdIdentifier = appContext.getApplicationID().getClusterTimestamp();
       } else {
         customAppIdIdentifier = SCHEDULER_APP_ID_BASE + (j++ * SCHEDULER_APP_ID_INCREMENT);

http://git-wip-us.apache.org/repos/asf/tez/blob/cfd625e9/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 21ae5f7..17feeaa 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,6 +34,7 @@ import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicLong;
 
+import org.apache.tez.dag.api.NamedEntityDescriptor;
 import org.apache.tez.serviceplugins.api.ContainerLaunchRequest;
 import org.apache.tez.serviceplugins.api.ContainerLauncher;
 import org.apache.tez.serviceplugins.api.ContainerLauncherContext;
@@ -486,7 +487,7 @@ public class MockDAGAppMaster extends DAGAppMaster {
       Credentials credentials, String jobUserName, int handlerConcurrency, int numConcurrentContainers) {
     super(applicationAttemptId, containerId, nmHost, nmPort, nmHttpPort, clock, appSubmitTime,
         isSession, workingDirectory, localDirs, logDirs,  new TezApiVersionInfo().getVersion(), 1,
-        credentials, jobUserName);
+        credentials, jobUserName, null);
     containerLauncherContext = new ContainerLauncherContextImpl(getContext(), getTaskAttemptListener());
     containerLauncher = new MockContainerLauncher(launcherGoFlag, containerLauncherContext);
     shutdownHandler = new MockDAGAppMasterShutdownHandler();
@@ -500,7 +501,7 @@ public class MockDAGAppMaster extends DAGAppMaster {
   // use mock container launcher for tests
   @Override
   protected ContainerLauncherRouter createContainerLauncherRouter(final Configuration conf,
-                                                                  String[] containerLaunchers,
+                                                                  List<NamedEntityDescriptor> containerLauncherDescirptors,
                                                                   boolean isLocal)
       throws UnknownHostException {
     return new ContainerLauncherRouter(containerLauncher, getContext());

http://git-wip-us.apache.org/repos/asf/tez/blob/cfd625e9/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 41a7373..e45b0a2 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
@@ -50,6 +50,7 @@ import org.apache.tez.common.ContainerTask;
 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.NamedEntityDescriptor;
 import org.apache.tez.dag.api.TezConfiguration;
 import org.apache.tez.serviceplugins.api.ContainerEndReason;
 import org.apache.tez.serviceplugins.api.TaskAttemptEndReason;
@@ -375,10 +376,10 @@ public class TestTaskAttemptListenerImplTezDag {
     public TaskAttemptListenerImplForTest(AppContext context,
                                           TaskHeartbeatHandler thh,
                                           ContainerHeartbeatHandler chh,
-                                          String[] taskCommunicatorClassIdentifiers,
+                                          List<NamedEntityDescriptor> taskCommDescriptors,
                                           Configuration conf,
                                           boolean isPureLocalMode) {
-      super(context, thh, chh, taskCommunicatorClassIdentifiers, conf,
+      super(context, thh, chh, taskCommDescriptors, conf,
           isPureLocalMode);
     }
 

http://git-wip-us.apache.org/repos/asf/tez/blob/cfd625e9/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 3ea0446..f191175 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
@@ -32,6 +32,7 @@ import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.util.HashMap;
 import java.util.HashSet;
+import java.util.LinkedList;
 import java.util.List;
 import java.util.concurrent.atomic.AtomicBoolean;
 
@@ -48,6 +49,7 @@ 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.ContainerSignatureMatcher;
+import org.apache.tez.dag.api.NamedEntityDescriptor;
 import org.apache.tez.dag.api.TaskLocationHint;
 import org.apache.tez.dag.api.TezConfiguration;
 import org.apache.tez.dag.api.client.DAGClientServer;
@@ -91,7 +93,7 @@ public class TestTaskSchedulerEventHandler {
     public MockTaskSchedulerEventHandler(AppContext appContext,
         DAGClientServer clientService, EventHandler eventHandler,
         ContainerSignatureMatcher containerSignatureMatcher, WebUIService webUI) {
-      super(appContext, clientService, eventHandler, containerSignatureMatcher, webUI, new String[] {}, false);
+      super(appContext, clientService, eventHandler, containerSignatureMatcher, webUI, new LinkedList<NamedEntityDescriptor>(), false);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/tez/blob/cfd625e9/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 966c95a..60d37e9 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
@@ -59,6 +59,7 @@ import org.apache.hadoop.yarn.client.api.impl.AMRMClientImpl;
 import org.apache.hadoop.yarn.event.Event;
 import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.tez.common.ContainerSignatureMatcher;
+import org.apache.tez.dag.api.NamedEntityDescriptor;
 import org.apache.tez.dag.app.AppContext;
 import org.apache.tez.dag.app.ServicePluginLifecycleAbstractService;
 import org.apache.tez.dag.app.rm.YarnTaskSchedulerService.CookieContainerRequest;
@@ -130,7 +131,7 @@ class TestTaskSchedulerHelpers {
         EventHandler eventHandler,
         TezAMRMClientAsync<CookieContainerRequest> amrmClientAsync,
         ContainerSignatureMatcher containerSignatureMatcher) {
-      super(appContext, null, eventHandler, containerSignatureMatcher, null, new String[]{}, false);
+      super(appContext, null, eventHandler, containerSignatureMatcher, null, new LinkedList<NamedEntityDescriptor>(), false);
       this.amrmClientAsync = amrmClientAsync;
       this.containerSignatureMatcher = containerSignatureMatcher;
     }

http://git-wip-us.apache.org/repos/asf/tez/blob/cfd625e9/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 ba17ba0..611e8cc 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
@@ -20,8 +20,8 @@ package org.apache.tez.examples;
 
 import java.io.IOException;
 import java.util.Set;
-import java.util.Map;
 
+import org.apache.tez.dag.api.Vertex.VertexExecutionContext;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -137,6 +137,9 @@ public class JoinValidate extends TezExampleBase {
   private DAG createDag(TezConfiguration tezConf, Path lhs, Path rhs, int numPartitions)
       throws IOException {
     DAG dag = DAG.create(getDagName());
+    if (getDefaultExecutionContext() != null) {
+      dag.setExecutionContext(getDefaultExecutionContext());
+    }
 
     // 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
@@ -153,18 +156,18 @@ public class JoinValidate extends TezExampleBase {
         MRInput
             .createConfigBuilder(new Configuration(tezConf), TextInputFormat.class,
                 lhs.toUri().toString()).groupSplits(!isDisableSplitGrouping()).build());
-    setVertexProperties(lhsVertex, getLhsVertexProperties());
+    setVertexExecutionContext(lhsVertex, getLhsExecutionContext());
 
     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());
+    setVertexExecutionContext(rhsVertex, getRhsExecutionContext());
 
     Vertex joinValidateVertex = Vertex.create("joinvalidate", ProcessorDescriptor.create(
         JoinValidateProcessor.class.getName()), numPartitions);
-    setVertexProperties(joinValidateVertex, getValidateVertexProperties());
+    setVertexExecutionContext(joinValidateVertex, getValidateExecutionContext());
 
     Edge e1 = Edge.create(lhsVertex, joinValidateVertex, edgeConf.createDefaultEdgeProperty());
     Edge e2 = Edge.create(rhsVertex, joinValidateVertex, edgeConf.createDefaultEdgeProperty());
@@ -174,23 +177,25 @@ 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());
-      }
+  private void setVertexExecutionContext(Vertex vertex, VertexExecutionContext executionContext) {
+    if (executionContext != null) {
+      vertex.setExecutionContext(executionContext);
     }
   }
 
-  protected Map<String, String> getLhsVertexProperties() {
+  protected VertexExecutionContext getDefaultExecutionContext() {
     return null;
   }
 
-  protected Map<String, String> getRhsVertexProperties() {
+  protected VertexExecutionContext getLhsExecutionContext() {
     return null;
   }
 
-  protected Map<String, String> getValidateVertexProperties() {
+  protected VertexExecutionContext getRhsExecutionContext() {
+    return null;
+  }
+
+  protected VertexExecutionContext getValidateExecutionContext() {
     return null;
   }
 
@@ -240,4 +245,6 @@ public class JoinValidate extends TezExampleBase {
       }
     }
   }
+
+
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/cfd625e9/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 85f9415..0002b42 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
@@ -121,7 +121,8 @@ public class TezTestServiceContainerLauncher extends ContainerLauncher {
   private RunContainerRequestProto constructRunContainerRequest(ContainerLaunchRequest launchRequest) throws
       IOException {
     RunContainerRequestProto.Builder builder = RunContainerRequestProto.newBuilder();
-    Preconditions.checkArgument(launchRequest.getTaskCommunicatorName().equals(TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT));
+    Preconditions.checkArgument(launchRequest.getTaskCommunicatorName().equals(
+        TezConstants.getTezYarnServicePluginName()));
     InetSocketAddress address = (InetSocketAddress) getContext().getTaskCommunicatorMetaInfo(launchRequest.getTaskCommunicatorName());
     builder.setAmHost(address.getHostName()).setAmPort(address.getPort());
     builder.setAppAttemptNumber(appAttemptId.getAttemptId());

http://git-wip-us.apache.org/repos/asf/tez/blob/cfd625e9/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
index e5d2e3b..f31476f 100644
--- 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
@@ -14,36 +14,46 @@
 
 package org.apache.tez.examples;
 
-import java.util.Map;
+
+import org.apache.tez.dag.api.Vertex.VertexExecutionContext;
 
 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 VertexExecutionContext defaultExecutionContext;
+  private final VertexExecutionContext lhsContext;
+  private final VertexExecutionContext rhsContext;
+  private final VertexExecutionContext validateContext;
   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;
+  public JoinValidateConfigured(VertexExecutionContext defaultExecutionContext,
+                                VertexExecutionContext lhsContext,
+                                VertexExecutionContext rhsContext,
+                                VertexExecutionContext validateContext, String dagNameSuffix) {
+    this.defaultExecutionContext = defaultExecutionContext;
+    this.lhsContext = lhsContext;
+    this.rhsContext = rhsContext;
+    this.validateContext = validateContext;
     this.dagNameSuffix = dagNameSuffix;
   }
 
   @Override
-  protected Map<String, String> getLhsVertexProperties() {
-    return this.lhsProps;
+  protected VertexExecutionContext getDefaultExecutionContext() {
+    return this.defaultExecutionContext;
+  }
+
+  @Override
+  protected VertexExecutionContext getLhsExecutionContext() {
+    return this.lhsContext;
   }
 
   @Override
-  protected Map<String, String> getRhsVertexProperties() {
-    return this.rhsProps;
+  protected VertexExecutionContext getRhsExecutionContext() {
+    return this.rhsContext;
   }
 
   @Override
-  protected Map<String, String> getValidateVertexProperties() {
-    return this.validateProps;
+  protected VertexExecutionContext getValidateExecutionContext() {
+    return this.validateContext;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/tez/blob/cfd625e9/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 45c70f1..07dd363 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
@@ -19,7 +19,6 @@ import static org.junit.Assert.assertEquals;
 import java.io.IOException;
 import java.util.Map;
 
-import com.google.common.collect.Maps;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -28,9 +27,9 @@ 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.Vertex.VertexExecutionContext;
 import org.apache.tez.dag.api.client.DAGClient;
 import org.apache.tez.dag.api.client.DAGStatus;
 import org.apache.tez.dag.app.launcher.TezTestServiceNoOpContainerLauncher;
@@ -43,6 +42,10 @@ 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;
+import org.apache.tez.serviceplugins.api.ContainerLauncherDescriptor;
+import org.apache.tez.serviceplugins.api.ServicePluginsDescriptor;
+import org.apache.tez.serviceplugins.api.TaskCommunicatorDescriptor;
+import org.apache.tez.serviceplugins.api.TaskSchedulerDescriptor;
 import org.apache.tez.test.MiniTezCluster;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
@@ -72,9 +75,15 @@ public class TestExternalTezServices {
   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 final VertexExecutionContext EXECUTION_CONTEXT_EXT_SERVICE_PUSH =
+      VertexExecutionContext.create(
+          EXT_PUSH_ENTITY_NAME, EXT_PUSH_ENTITY_NAME, EXT_PUSH_ENTITY_NAME);
+  private static final VertexExecutionContext EXECUTION_CONTEXT_REGULAR_CONTAINERS =
+      VertexExecutionContext.createExecuteInContainers(true);
+  private static final VertexExecutionContext EXECUTION_CONTEXT_IN_AM =
+      VertexExecutionContext.createExecuteInAm(true);
+
+  private static final VertexExecutionContext EXECUTION_CONTEXT_DEFAULT = EXECUTION_CONTEXT_EXT_SERVICE_PUSH;
 
   private static String TEST_ROOT_DIR = "target" + Path.SEPARATOR + TestExternalTezServices.class.getName()
       + "-tmpDir";
@@ -127,51 +136,28 @@ public class TestExternalTezServices {
     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,
-        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,
-        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());
-
-    // 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, 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,
-        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);
+    TaskSchedulerDescriptor[] taskSchedulerDescriptors = new TaskSchedulerDescriptor[]{
+        TaskSchedulerDescriptor
+            .create(EXT_PUSH_ENTITY_NAME, TezTestServiceTaskSchedulerService.class.getName())};
+
+    ContainerLauncherDescriptor[] containerLauncherDescriptors = new ContainerLauncherDescriptor[]{
+        ContainerLauncherDescriptor
+            .create(EXT_PUSH_ENTITY_NAME, TezTestServiceNoOpContainerLauncher.class.getName())};
+
+    TaskCommunicatorDescriptor[] taskCommunicatorDescriptors = new TaskCommunicatorDescriptor[]{
+        TaskCommunicatorDescriptor
+            .create(EXT_PUSH_ENTITY_NAME, TezTestServiceTaskCommunicatorImpl.class.getName())};
 
+    ServicePluginsDescriptor servicePluginsDescriptor = ServicePluginsDescriptor.create(true, true,
+        taskSchedulerDescriptors, containerLauncherDescriptors, taskCommunicatorDescriptors);
 
     // Create a session to use for all tests.
     TezConfiguration tezClientConf = new TezConfiguration(confForJobs);
 
-    sharedTezClient = TezClient.create(TestExternalTezServices.class.getSimpleName() + "_session",
-        tezClientConf, true);
+    sharedTezClient = TezClient
+        .newBuilder(TestExternalTezServices.class.getSimpleName() + "_session", tezClientConf)
+        .setIsSession(true).setServicePluginDescriptor(servicePluginsDescriptor).build();
+
     sharedTezClient.start();
     LOG.info("Shared TezSession started");
     sharedTezClient.waitTillReady();
@@ -225,71 +211,71 @@ public class TestExternalTezServices {
   @Test(timeout = 60000)
   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);
+    runJoinValidate("AllInService", expectedExternalSubmissions, EXECUTION_CONTEXT_EXT_SERVICE_PUSH,
+        EXECUTION_CONTEXT_EXT_SERVICE_PUSH, EXECUTION_CONTEXT_EXT_SERVICE_PUSH);
   }
 
   @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);
+    runJoinValidate("AllInContainers", expectedExternalSubmissions, EXECUTION_CONTEXT_REGULAR_CONTAINERS,
+        EXECUTION_CONTEXT_REGULAR_CONTAINERS, EXECUTION_CONTEXT_REGULAR_CONTAINERS);
   }
 
   @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);
+    runJoinValidate("AllInAM", expectedExternalSubmissions, EXECUTION_CONTEXT_IN_AM,
+        EXECUTION_CONTEXT_IN_AM, EXECUTION_CONTEXT_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,
-        PROPS_EXT_SERVICE_PUSH, PROPS_REGULAR_CONTAINERS);
+    runJoinValidate("Mixed1", expectedExternalSubmissions, EXECUTION_CONTEXT_EXT_SERVICE_PUSH,
+        EXECUTION_CONTEXT_EXT_SERVICE_PUSH, EXECUTION_CONTEXT_REGULAR_CONTAINERS);
   }
 
   @Test(timeout = 60000)
   public void testMixed2() throws Exception { // M-Containers, R-ExtService
     int expectedExternalSubmissions = 0 + 3; // 3 for num reducers.
-    runJoinValidate("Mixed2", expectedExternalSubmissions, PROPS_REGULAR_CONTAINERS,
-        PROPS_REGULAR_CONTAINERS, PROPS_EXT_SERVICE_PUSH);
+    runJoinValidate("Mixed2", expectedExternalSubmissions, EXECUTION_CONTEXT_REGULAR_CONTAINERS,
+        EXECUTION_CONTEXT_REGULAR_CONTAINERS, EXECUTION_CONTEXT_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);
+    runJoinValidate("Mixed3", expectedExternalSubmissions, EXECUTION_CONTEXT_EXT_SERVICE_PUSH,
+        EXECUTION_CONTEXT_EXT_SERVICE_PUSH, EXECUTION_CONTEXT_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);
+    runJoinValidate("Mixed4", expectedExternalSubmissions, EXECUTION_CONTEXT_REGULAR_CONTAINERS,
+        EXECUTION_CONTEXT_REGULAR_CONTAINERS, EXECUTION_CONTEXT_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);
+    runJoinValidate("Mixed5", expectedExternalSubmissions, EXECUTION_CONTEXT_REGULAR_CONTAINERS,
+        EXECUTION_CONTEXT_EXT_SERVICE_PUSH, EXECUTION_CONTEXT_IN_AM);
   }
 
   @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);
+    runJoinValidate("Mixed6", expectedExternalSubmissions, EXECUTION_CONTEXT_IN_AM,
+        EXECUTION_CONTEXT_IN_AM, EXECUTION_CONTEXT_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);
+    runJoinValidate("Mixed7", expectedExternalSubmissions, EXECUTION_CONTEXT_IN_AM,
+        EXECUTION_CONTEXT_IN_AM, EXECUTION_CONTEXT_REGULAR_CONTAINERS);
   }
 
   @Test(timeout = 60000)
@@ -303,10 +289,9 @@ public class TestExternalTezServices {
     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());
-    }
+    v.setExecutionContext(EXECUTION_CONTEXT_EXT_SERVICE_PUSH);
     dag.addVertex(v);
+
     DAGClient dagClient = sharedTezClient.submitDAG(dag);
     DAGStatus dagStatus = dagClient.waitForCompletion();
     assertEquals(DAGStatus.State.SUCCEEDED, dagStatus.getState());
@@ -315,16 +300,16 @@ public class TestExternalTezServices {
 
   }
 
-  private void runJoinValidate(String name, int extExpectedCount, Map<String, String> lhsProps,
-                               Map<String, String> rhsProps,
-                               Map<String, String> validateProps) throws
+  private void runJoinValidate(String name, int extExpectedCount, VertexExecutionContext lhsContext,
+                               VertexExecutionContext rhsContext,
+                               VertexExecutionContext validateContext) throws
       Exception {
     int externalSubmissionCount = tezTestServiceCluster.getNumSubmissions();
 
     TezConfiguration tezConf = new TezConfiguration(confForJobs);
     JoinValidateConfigured joinValidate =
-        new JoinValidateConfigured(lhsProps, rhsProps,
-            validateProps, name);
+        new JoinValidateConfigured(EXECUTION_CONTEXT_DEFAULT, lhsContext, rhsContext,
+            validateContext, 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));

http://git-wip-us.apache.org/repos/asf/tez/blob/cfd625e9/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 fff39a0..353fe23 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
@@ -62,6 +62,7 @@ import org.apache.tez.common.security.TokenCache;
 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.records.DAGProtos;
 import org.apache.tez.dag.records.TezVertexID;
 import org.apache.tez.dag.utils.RelocalizationUtils;
 import org.apache.tez.runtime.api.ExecutionContext;
@@ -477,7 +478,9 @@ public class TezChild {
     }
 
     // Security framework already loaded the tokens into current ugi
-    TezUtilsInternal.addUserSpecifiedTezConfiguration(System.getenv(Environment.PWD.name()), defaultConf);
+    DAGProtos.ConfigurationProto confProto =
+        TezUtilsInternal.readUserSpecifiedTezConfiguration(System.getenv(Environment.PWD.name()));
+    TezUtilsInternal.addUserSpecifiedTezConfiguration(defaultConf, confProto.getConfKeyValuesList());
     UserGroupInformation.setConfiguration(defaultConf);
     Credentials credentials = UserGroupInformation.getCurrentUser().getCredentials();
     TezChild tezChild = newTezChild(defaultConf, host, port, containerIdentifier,


[04/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/117cb41d
Tree: http://git-wip-us.apache.org/repos/asf/tez/tree/117cb41d
Diff: http://git-wip-us.apache.org/repos/asf/tez/diff/117cb41d

Branch: refs/heads/TEZ-2003
Commit: 117cb41d5905baa2599dc06ef5d0ff89ecc85ef4
Parents: 899a310
Author: Siddharth Seth <ss...@apache.org>
Authored: Fri Feb 20 17:23:18 2015 -0800
Committer: Siddharth Seth <ss...@apache.org>
Committed: Fri Aug 14 13:46:42 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/117cb41d/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/117cb41d/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


[13/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/0d72009c
Tree: http://git-wip-us.apache.org/repos/asf/tez/tree/0d72009c
Diff: http://git-wip-us.apache.org/repos/asf/tez/diff/0d72009c

Branch: refs/heads/TEZ-2003
Commit: 0d72009c184cfa13dfa3a2baebc892fb66614305
Parents: 26405ef
Author: Siddharth Seth <ss...@apache.org>
Authored: Tue Apr 7 13:21:35 2015 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Fri Aug 14 13:46:43 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/0d72009c/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/0d72009c/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/0d72009c/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 d9a7786..3579e3f 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
@@ -48,6 +48,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;
 
@@ -66,7 +67,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);
 
@@ -98,6 +99,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,
@@ -110,12 +112,14 @@ 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;
     // KKK Make sure the callable completes before proceeding
   }
-  
+
+  @Override
   public void shutdown() {
     heartbeatExecutor.shutdownNow();
   }
@@ -413,19 +417,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/0d72009c/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 32da8fb..4c8bebc 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
@@ -69,6 +69,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;
 
@@ -114,7 +115,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/0d72009c/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 {


[19/50] [abbrv] tez git commit: TEZ-2438. tez-tools version in the branch is incorrect. (sseth)

Posted by ss...@apache.org.
TEZ-2438. tez-tools version in the branch is incorrect. (sseth)


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

Branch: refs/heads/TEZ-2003
Commit: 34f053f85cb5585faef23c6dceb9a9afa0eca667
Parents: 2b66886
Author: Siddharth Seth <ss...@apache.org>
Authored: Mon May 11 16:42:45 2015 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Fri Aug 14 13:46:44 2015 -0700

----------------------------------------------------------------------
 TEZ-2003-CHANGES.txt               | 1 +
 tez-tools/pom.xml                  | 2 +-
 tez-tools/tez-tfile-parser/pom.xml | 2 +-
 3 files changed, 3 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/34f053f8/TEZ-2003-CHANGES.txt
----------------------------------------------------------------------
diff --git a/TEZ-2003-CHANGES.txt b/TEZ-2003-CHANGES.txt
index ad167ab..66c110f 100644
--- a/TEZ-2003-CHANGES.txt
+++ b/TEZ-2003-CHANGES.txt
@@ -23,5 +23,6 @@ ALL CHANGES:
   TEZ-2414. LogicalIOProcessorRuntimeTask, RuntimeTask, TezTaskRunner should handle interrupts & carry out necessary cleanups.
   TEZ-2420. TaskRunner returning before executing the task.
   TEZ-2433. Fixes after rebase 05/08
+  TEZ-2438. tez-tools version in the branch is incorrect.
 
 INCOMPATIBLE CHANGES:

http://git-wip-us.apache.org/repos/asf/tez/blob/34f053f8/tez-tools/pom.xml
----------------------------------------------------------------------
diff --git a/tez-tools/pom.xml b/tez-tools/pom.xml
index ed13143..5b9e41a 100644
--- a/tez-tools/pom.xml
+++ b/tez-tools/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-tools</artifactId>
   <packaging>pom</packaging>

http://git-wip-us.apache.org/repos/asf/tez/blob/34f053f8/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 aea1299..9467bac 100644
--- a/tez-tools/tez-tfile-parser/pom.xml
+++ b/tez-tools/tez-tfile-parser/pom.xml
@@ -20,7 +20,7 @@
   <parent>
     <groupId>org.apache.tez</groupId>
     <artifactId>tez-tools</artifactId>
-    <version>0.8.0-SNAPSHOT</version>
+    <version>0.8.0-TEZ-2003-SNAPSHOT</version>
   </parent>
   <artifactId>tez-tfile-parser</artifactId>
 


[22/50] [abbrv] tez git commit: TEZ-2465. Retrun the status of a kill request in TaskRunner2. (sseth)

Posted by ss...@apache.org.
TEZ-2465. Retrun the status of a kill request in TaskRunner2. (sseth)


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

Branch: refs/heads/TEZ-2003
Commit: 8a38d48ea489c3033585e73936960c4eaed769ae
Parents: ec4a5c3
Author: Siddharth Seth <ss...@apache.org>
Authored: Tue May 19 13:57:58 2015 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Fri Aug 14 13:46:44 2015 -0700

----------------------------------------------------------------------
 TEZ-2003-CHANGES.txt                                         | 1 +
 .../java/org/apache/tez/runtime/task/TezTaskRunner2.java     | 8 +++++++-
 2 files changed, 8 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/8a38d48e/TEZ-2003-CHANGES.txt
----------------------------------------------------------------------
diff --git a/TEZ-2003-CHANGES.txt b/TEZ-2003-CHANGES.txt
index ed72d6b..ca3383c 100644
--- a/TEZ-2003-CHANGES.txt
+++ b/TEZ-2003-CHANGES.txt
@@ -26,5 +26,6 @@ ALL CHANGES:
   TEZ-2438. tez-tools version in the branch is incorrect.
   TEZ-2434. Allow tasks to be killed in the Runtime.
   TEZ-2443. TaskRunner2 should call abort, NPEs while cleaning up tasks.
+  TEZ-2465. Retrun the status of a kill request in TaskRunner2.
 
 INCOMPATIBLE CHANGES:

http://git-wip-us.apache.org/repos/asf/tez/blob/8a38d48e/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezTaskRunner2.java
----------------------------------------------------------------------
diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezTaskRunner2.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezTaskRunner2.java
index ffbc6e8..3bf9f84 100644
--- a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezTaskRunner2.java
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezTaskRunner2.java
@@ -247,7 +247,11 @@ public class TezTaskRunner2 {
     }
   }
 
-  public void killTask() {
+  /**
+   * Attempt to kill the running task, if it hasn't already completed for some other reason.
+   * @return true if the task kill was honored, false otherwise
+   */
+  public boolean killTask() {
     synchronized (this) {
       if (isRunningState()) {
         if (trySettingEndReason(EndReason.KILL_REQUESTED)) {
@@ -256,9 +260,11 @@ public class TezTaskRunner2 {
             taskKillStartTime = System.currentTimeMillis();
             taskRunnerCallable.interruptTask();
           }
+          return true;
         }
       }
     }
+    return false;
   }
 
 


[07/50] [abbrv] tez git commit: TEZ-2420. TaskRunner returning before executing the task. (sseth)

Posted by ss...@apache.org.
TEZ-2420. TaskRunner returning before executing the task. (sseth)


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

Branch: refs/heads/TEZ-2003
Commit: 5e328095c3f09ec2637b9967d3b70f0cb0cb1322
Parents: 2fc431d
Author: Siddharth Seth <ss...@apache.org>
Authored: Wed May 6 17:14:56 2015 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Fri Aug 14 13:46:43 2015 -0700

----------------------------------------------------------------------
 TEZ-2003-CHANGES.txt                                                | 1 +
 .../src/main/java/org/apache/tez/runtime/task/TezTaskRunner.java    | 1 +
 2 files changed, 2 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/5e328095/TEZ-2003-CHANGES.txt
----------------------------------------------------------------------
diff --git a/TEZ-2003-CHANGES.txt b/TEZ-2003-CHANGES.txt
index f8a71e8..9b2339f 100644
--- a/TEZ-2003-CHANGES.txt
+++ b/TEZ-2003-CHANGES.txt
@@ -21,5 +21,6 @@ ALL CHANGES:
   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.
+  TEZ-2420. TaskRunner returning before executing the task.
 
 INCOMPATIBLE CHANGES:

http://git-wip-us.apache.org/repos/asf/tez/blob/5e328095/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 7238d5e..dd4620a 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
@@ -105,6 +105,7 @@ public class TezTaskRunner implements TezUmbilical, ErrorReporter {
     Throwable failureCause = null;
     if (!Thread.currentThread().isInterrupted()) {
       taskFuture = executor.submit(callable);
+    } else {
       return isShutdownRequested();
     }
     try {


[37/50] [abbrv] tez git commit: TEZ-2652. Cleanup the way services are specified for an AM and vertices. (sseth)

Posted by ss...@apache.org.
TEZ-2652. Cleanup the way services are specified for an AM and vertices. (sseth)


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

Branch: refs/heads/TEZ-2003
Commit: cfd625e95b4edd836638612fbbf0eac7383be3d2
Parents: fc8a4ce
Author: Siddharth Seth <ss...@apache.org>
Authored: Tue Jul 28 14:56:20 2015 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Fri Aug 14 13:47:09 2015 -0700

----------------------------------------------------------------------
 TEZ-2003-CHANGES.txt                            |   1 +
 .../java/org/apache/tez/client/TezClient.java   |  81 ++++++++-
 .../org/apache/tez/client/TezClientUtils.java   |  56 +++++-
 .../main/java/org/apache/tez/dag/api/DAG.java   |  48 +++++-
 .../apache/tez/dag/api/DagTypeConverters.java   |  93 +++++++++-
 .../tez/dag/api/NamedEntityDescriptor.java      |  33 ++++
 .../apache/tez/dag/api/TezConfiguration.java    |  31 ----
 .../org/apache/tez/dag/api/TezConstants.java    |  11 +-
 .../java/org/apache/tez/dag/api/Vertex.java     | 110 +++++++++++-
 .../api/ContainerLauncherDescriptor.java        |  32 ++++
 .../api/ServicePluginsDescriptor.java           |  96 +++++++++++
 .../api/TaskCommunicatorDescriptor.java         |  33 ++++
 .../api/TaskSchedulerDescriptor.java            |  32 ++++
 tez-api/src/main/proto/DAGApiRecords.proto      |  25 +++
 .../apache/tez/client/TestTezClientUtils.java   |  12 +-
 .../org/apache/tez/common/TezUtilsInternal.java |  31 +++-
 .../java/org/apache/tez/client/LocalClient.java |   2 +-
 .../org/apache/tez/dag/app/DAGAppMaster.java    | 172 +++++++++++--------
 .../dag/app/TaskAttemptListenerImpTezDag.java   |  37 ++--
 .../java/org/apache/tez/dag/app/dag/DAG.java    |   2 +
 .../apache/tez/dag/app/dag/impl/DAGImpl.java    |   9 +
 .../apache/tez/dag/app/dag/impl/VertexImpl.java |  39 ++++-
 .../dag/app/launcher/ContainerLauncherImpl.java |   6 +-
 .../app/launcher/ContainerLauncherRouter.java   |  40 +++--
 .../dag/app/rm/TaskSchedulerEventHandler.java   |  47 ++---
 .../apache/tez/dag/app/MockDAGAppMaster.java    |   5 +-
 .../app/TestTaskAttemptListenerImplTezDag.java  |   5 +-
 .../app/rm/TestTaskSchedulerEventHandler.java   |   4 +-
 .../dag/app/rm/TestTaskSchedulerHelpers.java    |   3 +-
 .../org/apache/tez/examples/JoinValidate.java   |  31 ++--
 .../TezTestServiceContainerLauncher.java        |   3 +-
 .../tez/examples/JoinValidateConfigured.java    |  40 +++--
 .../tez/tests/TestExternalTezServices.java      | 131 +++++++-------
 .../org/apache/tez/runtime/task/TezChild.java   |   5 +-
 34 files changed, 995 insertions(+), 311 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/cfd625e9/TEZ-2003-CHANGES.txt
----------------------------------------------------------------------
diff --git a/TEZ-2003-CHANGES.txt b/TEZ-2003-CHANGES.txt
index e57f76f..a201942 100644
--- a/TEZ-2003-CHANGES.txt
+++ b/TEZ-2003-CHANGES.txt
@@ -37,5 +37,6 @@ ALL CHANGES:
   TEZ-2004. Define basic interface for pluggable ContainerLaunchers.
   TEZ-2005. Define basic interface for pluggable TaskScheduler.
   TEZ-2651. Pluggable services should not extend AbstractService.
+  TEZ-2652. Cleanup the way services are specified for an AM and vertices.
 
 INCOMPATIBLE CHANGES:

http://git-wip-us.apache.org/repos/asf/tez/blob/cfd625e9/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 2590879..8759fdc 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
@@ -26,6 +26,7 @@ import java.util.Map;
 import javax.annotation.Nullable;
 
 import org.apache.tez.common.RPCUtil;
+import org.apache.tez.serviceplugins.api.ServicePluginsDescriptor;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
@@ -111,8 +112,9 @@ public class TezClient {
   private static final long SLEEP_FOR_READY = 500;
   private JobTokenSecretManager jobTokenSecretManager =
       new JobTokenSecretManager();
-  private Map<String, LocalResource> additionalLocalResources = Maps.newHashMap();
-  private TezApiVersionInfo apiVersionInfo;
+  private final Map<String, LocalResource> additionalLocalResources = Maps.newHashMap();
+  private final TezApiVersionInfo apiVersionInfo;
+  private final ServicePluginsDescriptor servicePluginsDescriptor;
   private HistoryACLPolicyManager historyACLPolicyManager;
 
   private int preWarmDAGCounter = 0;
@@ -142,18 +144,44 @@ public class TezClient {
 
   @Private
   protected TezClient(String name, TezConfiguration tezConf, boolean isSession,
+                      @Nullable Map<String, LocalResource> localResources,
+                      @Nullable Credentials credentials) {
+    this(name, tezConf, isSession, localResources, credentials, null);
+  }
+
+  @Private
+  protected TezClient(String name, TezConfiguration tezConf, boolean isSession,
             @Nullable Map<String, LocalResource> localResources,
-            @Nullable Credentials credentials) {
+            @Nullable Credentials credentials, ServicePluginsDescriptor servicePluginsDescriptor) {
     this.clientName = name;
     this.isSession = isSession;
     // Set in conf for local mode AM to figure out whether in session mode or not
     tezConf.setBoolean(TezConfiguration.TEZ_AM_SESSION_MODE, isSession);
     this.amConfig = new AMConfiguration(tezConf, localResources, credentials);
     this.apiVersionInfo = new TezApiVersionInfo();
+    this.servicePluginsDescriptor = servicePluginsDescriptor;
 
     LOG.info("Tez Client Version: " + apiVersionInfo.toString());
   }
 
+
+  /**
+   * Create a new TezClientBuilder. This can be used to setup additional parameters
+   * like session mode, local resources, credentials, servicePlugins, etc.
+   * <p/>
+   * If session mode is not specified in the builder, this will be inferred from
+   * the provided TezConfiguration.
+   *
+   * @param name    Name of the client. Used for logging etc. This will also be used
+   *                as app master name is session mode
+   * @param tezConf Configuration for the framework
+   * @return An instance of {@link org.apache.tez.client.TezClient.TezClientBuilder}
+   * which can be used to construct the final TezClient.
+   */
+  public static TezClientBuilder newBuilder(String name, TezConfiguration tezConf) {
+    return new TezClientBuilder(name, tezConf);
+  }
+
   /**
    * Create a new TezClient. Session or non-session execution mode will be
    * inferred from configuration.
@@ -355,7 +383,7 @@ public class TezClient {
                 sessionAppId,
                 null, clientName, amConfig,
                 tezJarResources, sessionCredentials, usingTezArchiveDeploy, apiVersionInfo,
-                historyACLPolicyManager);
+                historyACLPolicyManager, servicePluginsDescriptor);
   
         // Set Tez Sessions to not retry on AM crashes if recovery is disabled
         if (!amConfig.getTezConfiguration().getBoolean(
@@ -771,7 +799,8 @@ public class TezClient {
       ApplicationSubmissionContext appContext = TezClientUtils
           .createApplicationSubmissionContext( 
               appId, dag, dag.getName(), amConfig, tezJarResources, credentials,
-              usingTezArchiveDeploy, apiVersionInfo, historyACLPolicyManager);
+              usingTezArchiveDeploy, apiVersionInfo, historyACLPolicyManager,
+              servicePluginsDescriptor);
       LOG.info("Submitting DAG to YARN"
           + ", applicationId=" + appId
           + ", dagName=" + dag.getName());
@@ -846,4 +875,46 @@ public class TezClient {
          append(SEPARATOR).
          append(tezDagIdFormat.get().format(1)).toString();
   }
+
+  @Public
+  public static class TezClientBuilder {
+    final String name;
+    final TezConfiguration tezConf;
+    boolean isSession;
+    private Map<String, LocalResource> localResourceMap;
+    private Credentials credentials;
+    ServicePluginsDescriptor servicePluginsDescriptor;
+
+    private TezClientBuilder(String name, TezConfiguration tezConf) {
+      this.name = name;
+      this.tezConf = tezConf;
+      isSession = tezConf.getBoolean(
+          TezConfiguration.TEZ_AM_SESSION_MODE, TezConfiguration.TEZ_AM_SESSION_MODE_DEFAULT);
+    }
+
+    public TezClientBuilder setIsSession(boolean isSession) {
+      this.isSession = isSession;
+      return this;
+    }
+
+    public TezClientBuilder setLocalResources(Map<String, LocalResource> localResources) {
+      this.localResourceMap = localResources;
+      return this;
+    }
+
+    public TezClientBuilder setCredentials(Credentials credentials) {
+      this.credentials = credentials;
+      return this;
+    }
+
+    public TezClientBuilder setServicePluginDescriptor(ServicePluginsDescriptor servicePluginsDescriptor) {
+      this.servicePluginsDescriptor = servicePluginsDescriptor;
+      return this;
+    }
+
+    public TezClient build() {
+      return new TezClient(name, tezConf, isSession, localResourceMap, credentials,
+          servicePluginsDescriptor);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/cfd625e9/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java b/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java
index 8bfaa1f..9cf1f3f 100644
--- a/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java
+++ b/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java
@@ -39,6 +39,10 @@ import java.util.Map.Entry;
 
 import com.google.common.base.Strings;
 import org.apache.commons.lang.StringUtils;
+import org.apache.tez.dag.api.NamedEntityDescriptor;
+import org.apache.tez.dag.api.records.DAGProtos.AMPluginDescriptorProto;
+import org.apache.tez.dag.api.records.DAGProtos.TezNamedEntityDescriptorProto;
+import org.apache.tez.serviceplugins.api.ServicePluginsDescriptor;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
@@ -405,6 +409,7 @@ public class TezClientUtils {
    * @param tezJarResources Resources to be used by the AM
    * @param sessionCreds the credential object which will be populated with session specific
    * @param historyACLPolicyManager
+   * @param servicePluginsDescriptor descriptor for services which may be running in the AM
    * @return an ApplicationSubmissionContext to launch a Tez AM
    * @throws IOException
    * @throws YarnException
@@ -415,7 +420,8 @@ public class TezClientUtils {
       ApplicationId appId, DAG dag, String amName,
       AMConfiguration amConfig, Map<String, LocalResource> tezJarResources,
       Credentials sessionCreds, boolean tezLrsAsArchive,
-      TezApiVersionInfo apiVersionInfo, HistoryACLPolicyManager historyACLPolicyManager)
+      TezApiVersionInfo apiVersionInfo, HistoryACLPolicyManager historyACLPolicyManager,
+      ServicePluginsDescriptor servicePluginsDescriptor)
       throws IOException, YarnException {
 
     Preconditions.checkNotNull(sessionCreds);
@@ -551,7 +557,7 @@ public class TezClientUtils {
 
     // emit conf as PB file
     ConfigurationProto finalConfProto = createFinalConfProtoForApp(amConfig.getTezConfiguration(),
-        aclConfigs);
+        aclConfigs, servicePluginsDescriptor);
     
     FSDataOutputStream amConfPBOutBinaryStream = null;
     try {
@@ -752,12 +758,8 @@ public class TezClientUtils {
         + "," + TezConstants.TEZ_CONTAINER_LOGGER_NAME);
   }
 
-  static ConfigurationProto createFinalConfProtoForApp(Configuration amConf) {
-    return createFinalConfProtoForApp(amConf, null);
-  }
-
   static ConfigurationProto createFinalConfProtoForApp(Configuration amConf,
-    Map<String, String> additionalConfigs) {
+    Map<String, String> additionalConfigs, ServicePluginsDescriptor servicePluginsDescriptor) {
     assert amConf != null;
     ConfigurationProto.Builder builder = ConfigurationProto.newBuilder();
     for (Entry<String, String> entry : amConf) {
@@ -774,9 +776,49 @@ public class TezClientUtils {
         builder.addConfKeyValues(kvp);
       }
     }
+
+    AMPluginDescriptorProto pluginDescriptorProto =
+        createAMServicePluginDescriptorProto(servicePluginsDescriptor);
+    builder.setAmPluginDescriptor(pluginDescriptorProto);
+
     return builder.build();
   }
 
+  static AMPluginDescriptorProto createAMServicePluginDescriptorProto(
+      ServicePluginsDescriptor servicePluginsDescriptor) {
+    AMPluginDescriptorProto.Builder pluginDescriptorBuilder =
+        AMPluginDescriptorProto.newBuilder();
+    if (servicePluginsDescriptor != null) {
+
+      pluginDescriptorBuilder.setContainersEnabled(servicePluginsDescriptor.areContainersEnabled());
+      pluginDescriptorBuilder.setUberEnabled(servicePluginsDescriptor.isUberEnabled());
+
+      if (servicePluginsDescriptor.getTaskSchedulerDescriptors() != null &&
+          servicePluginsDescriptor.getTaskSchedulerDescriptors().length > 0) {
+        List<TezNamedEntityDescriptorProto> namedEntityProtos = DagTypeConverters.convertNamedEntityCollectionToProto(
+            servicePluginsDescriptor.getTaskSchedulerDescriptors());
+        pluginDescriptorBuilder.addAllTaskScedulers(namedEntityProtos);
+      }
+
+      if (servicePluginsDescriptor.getContainerLauncherDescriptors() != null &&
+          servicePluginsDescriptor.getContainerLauncherDescriptors().length > 0) {
+        List<TezNamedEntityDescriptorProto> namedEntityProtos = DagTypeConverters.convertNamedEntityCollectionToProto(
+            servicePluginsDescriptor.getContainerLauncherDescriptors());
+        pluginDescriptorBuilder.addAllContainerLaunchers(namedEntityProtos);
+      }
+
+      if (servicePluginsDescriptor.getTaskCommunicatorDescriptors() != null &&
+          servicePluginsDescriptor.getTaskCommunicatorDescriptors().length > 0) {
+        List<TezNamedEntityDescriptorProto> namedEntityProtos = DagTypeConverters.convertNamedEntityCollectionToProto(
+            servicePluginsDescriptor.getTaskCommunicatorDescriptors());
+        pluginDescriptorBuilder.addAllTaskCommunicators(namedEntityProtos);
+      }
+
+    } else {
+      pluginDescriptorBuilder.setContainersEnabled(true).setUberEnabled(false);
+    }
+    return pluginDescriptorBuilder.build();
+  }
 
   /**
    * Helper function to create a YARN LocalResource

http://git-wip-us.apache.org/repos/asf/tez/blob/cfd625e9/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 8ee1682..fce9522 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
@@ -32,6 +32,9 @@ import java.util.Stack;
 
 import org.apache.commons.collections4.BidiMap;
 import org.apache.commons.collections4.bidimap.DualLinkedHashBidiMap;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.tez.dag.api.Vertex.VertexExecutionContext;
+import org.apache.tez.dag.api.records.DAGProtos;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
@@ -60,9 +63,7 @@ import org.apache.tez.dag.api.records.DAGProtos.PlanVertexType;
 import org.apache.tez.dag.api.records.DAGProtos.VertexPlan;
 
 import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Function;
 import com.google.common.base.Preconditions;
-import com.google.common.collect.Collections2;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 import com.google.common.collect.Sets;
@@ -92,6 +93,7 @@ public class DAG {
   Map<String, LocalResource> commonTaskLocalFiles = Maps.newHashMap();
   String dagInfo;
   private Map<String,String> dagConf = new HashMap<String, String>();
+  private VertexExecutionContext defaultExecutionContext;
 
   private Stack<String> topologicalVertexStack = new Stack<String>();
 
@@ -335,6 +337,26 @@ public class DAG {
     return this;
   }
 
+  /**
+   * Sets the default execution context for the DAG. This can be overridden at a per Vertex level.
+   * See {@link org.apache.tez.dag.api.Vertex#setExecutionContext(VertexExecutionContext)}
+   *
+   * @param vertexExecutionContext the default execution context for the DAG
+   *
+   * @return
+   */
+  @Public
+  @InterfaceStability.Unstable
+  public synchronized DAG setExecutionContext(VertexExecutionContext vertexExecutionContext) {
+    this.defaultExecutionContext = vertexExecutionContext;
+    return this;
+  }
+
+  @Private
+  VertexExecutionContext getDefaultExecutionContext() {
+    return this.defaultExecutionContext;
+  }
+
   @Private
   @VisibleForTesting
   public Map<String,String> getDagConf() {
@@ -707,7 +729,15 @@ public class DAG {
     if (this.dagInfo != null && !this.dagInfo.isEmpty()) {
       dagBuilder.setDagInfo(this.dagInfo);
     }
-    
+
+    // Setup default execution context.
+    VertexExecutionContext defaultContext = getDefaultExecutionContext();
+    if (defaultContext != null) {
+      DAGProtos.VertexExecutionContextProto contextProto = DagTypeConverters.convertToProto(
+          defaultContext);
+      dagBuilder.setDefaultExecutionContext(contextProto);
+    }
+
     if (!vertexGroups.isEmpty()) {
       for (VertexGroup av : vertexGroups) {
         GroupInfo groupInfo = av.getGroupInfo();
@@ -800,7 +830,17 @@ public class DAG {
       vertexBuilder.setName(vertex.getName());
       vertexBuilder.setType(PlanVertexType.NORMAL); // vertex type is implicitly NORMAL until  TEZ-46.
       vertexBuilder.setProcessorDescriptor(DagTypeConverters
-        .convertToDAGPlan(vertex.getProcessorDescriptor()));
+          .convertToDAGPlan(vertex.getProcessorDescriptor()));
+
+      // Vertex ExecutionContext setup
+      VertexExecutionContext execContext = vertex.getVertexExecutionContext();
+      if (execContext != null) {
+        DAGProtos.VertexExecutionContextProto contextProto =
+            DagTypeConverters.convertToProto(execContext);
+        vertexBuilder.setExecutionContext(contextProto);
+      }
+      // End of VertexExecutionContext setup.
+
       if (vertex.getInputs().size() > 0) {
         for (RootInputLeafOutput<InputDescriptor, InputInitializerDescriptor> input : vertex.getInputs()) {
           vertexBuilder.addInputs(DagTypeConverters.convertToDAGPlan(input));

http://git-wip-us.apache.org/repos/asf/tez/blob/cfd625e9/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 8b1d553..2e0d417 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
@@ -73,11 +73,15 @@ import org.apache.tez.dag.api.records.DAGProtos.TezCounterGroupProto;
 import org.apache.tez.dag.api.records.DAGProtos.TezCounterProto;
 import org.apache.tez.dag.api.records.DAGProtos.TezCountersProto;
 import org.apache.tez.dag.api.records.DAGProtos.TezEntityDescriptorProto;
+import org.apache.tez.dag.api.records.DAGProtos.TezNamedEntityDescriptorProto;
 import org.apache.tez.dag.api.records.DAGProtos.VertexLocationHintProto;
 
 import com.google.common.collect.Lists;
 import com.google.protobuf.ByteString;
 import com.google.protobuf.ByteString.Output;
+import org.apache.tez.serviceplugins.api.ContainerLauncherDescriptor;
+import org.apache.tez.serviceplugins.api.TaskCommunicatorDescriptor;
+import org.apache.tez.serviceplugins.api.TaskSchedulerDescriptor;
 
 @Private
 public class DagTypeConverters {
@@ -399,6 +403,8 @@ public class DagTypeConverters {
     return userPayload;
   }
 
+
+
   private static void setUserPayload(EntityDescriptor<?> entity, UserPayload payload) {
     if (payload != null) {
       entity.setUserPayload(payload);
@@ -423,6 +429,15 @@ public class DagTypeConverters {
     return od;
   }
 
+  public static NamedEntityDescriptor convertNamedDescriptorFromProto(TezNamedEntityDescriptorProto proto) {
+    String name = proto.getName();
+    String className = proto.getEntityDescriptor().getClassName();
+    UserPayload payload = convertTezUserPayloadFromDAGPlan(proto.getEntityDescriptor());
+    NamedEntityDescriptor descriptor = new NamedEntityDescriptor(name, className);
+    setUserPayload(descriptor, payload);
+    return descriptor;
+  }
+
   public static InputInitializerDescriptor convertInputInitializerDescriptorFromDAGPlan(
       TezEntityDescriptorProto proto) {
     String className = proto.getClassName();
@@ -550,11 +565,11 @@ public class DagTypeConverters {
   public static LocalResource convertPlanLocalResourceToLocalResource(
       PlanLocalResource plr) {
     return LocalResource.newInstance(
-      ConverterUtils.getYarnUrlFromPath(new Path(plr.getUri())),
-      DagTypeConverters.convertFromDAGPlan(plr.getType()),
-      DagTypeConverters.convertFromDAGPlan(plr.getVisibility()),
-      plr.getSize(), plr.getTimeStamp(),
-      plr.hasPattern() ? plr.getPattern() : null);
+        ConverterUtils.getYarnUrlFromPath(new Path(plr.getUri())),
+        DagTypeConverters.convertFromDAGPlan(plr.getType()),
+        DagTypeConverters.convertFromDAGPlan(plr.getVisibility()),
+        plr.getSize(), plr.getTimeStamp(),
+        plr.hasPattern() ? plr.getPattern() : null);
   }
 
   public static TezCounters convertTezCountersFromProto(TezCountersProto proto) {
@@ -717,4 +732,72 @@ public class DagTypeConverters {
     return payload.getPayload();
   }
 
+  public static DAGProtos.VertexExecutionContextProto convertToProto(
+      Vertex.VertexExecutionContext context) {
+    if (context == null) {
+      return null;
+    } else {
+      DAGProtos.VertexExecutionContextProto.Builder builder =
+          DAGProtos.VertexExecutionContextProto.newBuilder();
+      builder.setExecuteInAm(context.shouldExecuteInAm());
+      builder.setExecuteInContainers(context.shouldExecuteInContainers());
+      if (context.getTaskSchedulerName() != null) {
+        builder.setTaskSchedulerName(context.getTaskSchedulerName());
+      }
+      if (context.getContainerLauncherName() != null) {
+        builder.setContainerLauncherName(context.getContainerLauncherName());
+      }
+      if (context.getTaskCommName() != null) {
+        builder.setTaskCommName(context.getTaskCommName());
+      }
+      return builder.build();
+    }
+  }
+
+  public static Vertex.VertexExecutionContext convertFromProto(
+      DAGProtos.VertexExecutionContextProto proto) {
+    if (proto == null) {
+      return null;
+    } else {
+      if (proto.getExecuteInAm()) {
+        Vertex.VertexExecutionContext context =
+            Vertex.VertexExecutionContext.createExecuteInAm(proto.getExecuteInAm());
+        return context;
+      } else if (proto.getExecuteInContainers()) {
+        Vertex.VertexExecutionContext context =
+            Vertex.VertexExecutionContext.createExecuteInContainers(proto.getExecuteInContainers());
+        return context;
+      } else {
+        String taskScheduler = proto.hasTaskSchedulerName() ? proto.getTaskSchedulerName() : null;
+        String containerLauncher =
+            proto.hasContainerLauncherName() ? proto.getContainerLauncherName() : null;
+        String taskComm = proto.hasTaskCommName() ? proto.getTaskCommName() : null;
+        Vertex.VertexExecutionContext context =
+            Vertex.VertexExecutionContext.create(taskScheduler, containerLauncher, taskComm);
+        return context;
+      }
+    }
+  }
+
+  public static List<TezNamedEntityDescriptorProto> convertNamedEntityCollectionToProto(
+      NamedEntityDescriptor[] namedEntityDescriptors) {
+    List<TezNamedEntityDescriptorProto> list =
+        Lists.newArrayListWithCapacity(namedEntityDescriptors.length);
+    for (NamedEntityDescriptor namedEntity : namedEntityDescriptors) {
+      TezNamedEntityDescriptorProto namedEntityProto = convertNamedEntityToProto(namedEntity);
+      list.add(namedEntityProto);
+    }
+    return list;
+  }
+
+  public static TezNamedEntityDescriptorProto convertNamedEntityToProto(
+      NamedEntityDescriptor namedEntityDescriptor) {
+    TezNamedEntityDescriptorProto.Builder builder = TezNamedEntityDescriptorProto.newBuilder();
+    builder.setName(namedEntityDescriptor.getEntityName());
+    DAGProtos.TezEntityDescriptorProto entityProto =
+        DagTypeConverters.convertToDAGPlan(namedEntityDescriptor);
+    builder.setEntityDescriptor(entityProto);
+    return builder.build();
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/cfd625e9/tez-api/src/main/java/org/apache/tez/dag/api/NamedEntityDescriptor.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/NamedEntityDescriptor.java b/tez-api/src/main/java/org/apache/tez/dag/api/NamedEntityDescriptor.java
new file mode 100644
index 0000000..bad0d10
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/dag/api/NamedEntityDescriptor.java
@@ -0,0 +1,33 @@
+/*
+ * 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 com.google.common.base.Preconditions;
+import org.apache.hadoop.classification.InterfaceAudience;
+
+public class NamedEntityDescriptor<T extends EntityDescriptor<T>> extends EntityDescriptor<T> {
+  private final String entityName;
+
+  @InterfaceAudience.Private
+  public NamedEntityDescriptor(String entityName, String className) {
+    super(className);
+    Preconditions.checkArgument(entityName != null, "EntityName must be specified");
+    this.entityName = entityName;
+  }
+
+  public String getEntityName() {
+    return entityName;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tez/blob/cfd625e9/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 39a4c77..3b7378a 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
@@ -1215,37 +1215,6 @@ 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_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_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_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) {
     Scope validScope = PropertyScope.get(property);

http://git-wip-us.apache.org/repos/asf/tez/blob/cfd625e9/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 3b07c59..6e1cb2d 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,7 +102,14 @@ public class TezConstants {
   /// Version-related Environment variables
   public static final String TEZ_CLIENT_VERSION_ENV = "TEZ_CLIENT_VERSION";
 
+  private static final String TEZ_AM_SERVICE_PLUGIN_NAME_YARN_CONTAINERS = "TezYarn";
+  private static final String TEZ_AM_SERVICE_PLUGIN_NAME_IN_AM = "TezUber";
 
-  public static final String TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT = "Tez";
-  public static final String TEZ_AM_SERVICE_PLUGINS_LOCAL_MODE_NAME_DEFAULT = "TezLocal";
+  public static String getTezYarnServicePluginName() {
+    return TEZ_AM_SERVICE_PLUGIN_NAME_YARN_CONTAINERS;
+  }
+
+  public static String getTezUberServicePluginName() {
+    return TEZ_AM_SERVICE_PLUGIN_NAME_IN_AM;
+  }
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/cfd625e9/tez-api/src/main/java/org/apache/tez/dag/api/Vertex.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/Vertex.java b/tez-api/src/main/java/org/apache/tez/dag/api/Vertex.java
index 0ed4bd8..34124b2 100644
--- a/tez-api/src/main/java/org/apache/tez/dag/api/Vertex.java
+++ b/tez-api/src/main/java/org/apache/tez/dag/api/Vertex.java
@@ -28,11 +28,11 @@ import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.yarn.api.records.LocalResource;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.tez.common.TezCommonUtils;
 import org.apache.tez.dag.api.VertexGroup.GroupInfo;
-import org.apache.tez.dag.api.TaskLocationHint;
 import org.apache.tez.runtime.api.LogicalIOProcessor;
 
 import com.google.common.base.Preconditions;
@@ -57,6 +57,7 @@ public class Vertex {
   private final Map<String, LocalResource> taskLocalResources = new HashMap<String, LocalResource>();
   private Map<String, String> taskEnvironment = new HashMap<String, String>();
   private Map<String, String> vertexConf = new HashMap<String, String>();
+  private VertexExecutionContext vertexExecutionContext;
   private final Map<String, RootInputLeafOutput<InputDescriptor, InputInitializerDescriptor>> additionalInputs
                       = new HashMap<String, RootInputLeafOutput<InputDescriptor, InputInitializerDescriptor>>();
   private final Map<String, RootInputLeafOutput<OutputDescriptor, OutputCommitterDescriptor>> additionalOutputs
@@ -410,6 +411,108 @@ public class Vertex {
     return this;
   }
 
+  /**
+   * Sets the execution context for this Vertex - i.e. the Task Scheduler, ContainerLauncher and
+   * TaskCommunicator to be used. Also whether the vertex will be executed within the AM.
+   * If partially specified, the default components in Tez will be used - which may or may not work
+   * with the custom context.
+   *
+   * @param vertexExecutionContext the execution context for the vertex.
+   *
+   * @return
+   */
+  public Vertex setExecutionContext(VertexExecutionContext vertexExecutionContext) {
+    this.vertexExecutionContext = vertexExecutionContext;
+    return this;
+  }
+
+  @Public
+  @InterfaceStability.Unstable
+  public static class VertexExecutionContext {
+    final boolean executeInAm;
+    final boolean executeInContainers;
+    final String taskSchedulerName;
+    final String containerLauncherName;
+    final String taskCommName;
+
+    public static VertexExecutionContext createExecuteInAm(boolean executeInAm) {
+      return new VertexExecutionContext(executeInAm, false);
+    }
+
+    public static VertexExecutionContext createExecuteInContainers(boolean executeInContainers) {
+      return new VertexExecutionContext(false, executeInContainers);
+    }
+
+    public static VertexExecutionContext create(String taskSchedulerName, String containerLauncherName,
+                                                String taskCommName) {
+      return new VertexExecutionContext(taskSchedulerName, containerLauncherName, taskCommName);
+    }
+
+    private VertexExecutionContext(boolean executeInAm, boolean executeInContainers) {
+      this(executeInAm, executeInContainers, null, null, null);
+    }
+
+    private VertexExecutionContext(String taskSchedulerName, String containerLauncherName,
+                                  String taskCommName) {
+      this(false, false, taskSchedulerName, containerLauncherName, taskCommName);
+    }
+
+    private VertexExecutionContext(boolean executeInAm, boolean executeInContainers, String taskSchedulerName, String containerLauncherName,
+                      String taskCommName) {
+      if (executeInAm || executeInContainers) {
+        Preconditions.checkState(!(executeInAm && executeInContainers),
+            "executeInContainers and executeInAM are mutually exclusive");
+        Preconditions.checkState(
+            taskSchedulerName == null && containerLauncherName == null && taskCommName == null,
+            "Uber (in-AM) or container execution cannot be enabled with a custom plugins. TaskScheduler=" +
+                taskSchedulerName + ", ContainerLauncher=" + containerLauncherName +
+                ", TaskCommunicator=" + taskCommName);
+      }
+      if (taskSchedulerName != null || containerLauncherName != null || taskCommName != null) {
+        Preconditions.checkState(executeInAm == false && executeInContainers == false,
+            "Uber (in-AM) and container execution cannot be enabled with a custom plugins. TaskScheduler=" +
+                taskSchedulerName + ", ContainerLauncher=" + containerLauncherName +
+                ", TaskCommunicator=" + taskCommName);
+      }
+      this.executeInAm = executeInAm;
+      this.executeInContainers = executeInContainers;
+      this.taskSchedulerName = taskSchedulerName;
+      this.containerLauncherName = containerLauncherName;
+      this.taskCommName = taskCommName;
+    }
+
+    public boolean shouldExecuteInAm() {
+      return executeInAm;
+    }
+
+    public boolean shouldExecuteInContainers() {
+      return executeInContainers;
+    }
+
+    public String getTaskSchedulerName() {
+      return taskSchedulerName;
+    }
+
+    public String getContainerLauncherName() {
+      return containerLauncherName;
+    }
+
+    public String getTaskCommName() {
+      return taskCommName;
+    }
+
+    @Override
+    public String toString() {
+      return "VertexExecutionContext{" +
+          "executeInAm=" + executeInAm +
+          ", executeInContainers=" + executeInContainers +
+          ", taskSchedulerName='" + taskSchedulerName + '\'' +
+          ", containerLauncherName='" + containerLauncherName + '\'' +
+          ", taskCommName='" + taskCommName + '\'' +
+          '}';
+    }
+  }
+
   @Override
   public String toString() {
     return "[" + vertexName + " : " + processorDescriptor.getClassName() + "]";
@@ -475,6 +578,11 @@ public class Vertex {
     return dataSinks;
   }
 
+  @Private
+  public VertexExecutionContext getVertexExecutionContext() {
+    return this.vertexExecutionContext;
+  }
+
   List<Edge> getInputEdges() {
     return inputEdges;
   }

http://git-wip-us.apache.org/repos/asf/tez/blob/cfd625e9/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ContainerLauncherDescriptor.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ContainerLauncherDescriptor.java b/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ContainerLauncherDescriptor.java
new file mode 100644
index 0000000..ff3c90e
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ContainerLauncherDescriptor.java
@@ -0,0 +1,32 @@
+/*
+ * 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.serviceplugins.api;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.tez.dag.api.NamedEntityDescriptor;
+
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
+public class ContainerLauncherDescriptor extends NamedEntityDescriptor<ContainerLauncherDescriptor> {
+
+  private ContainerLauncherDescriptor(String containerLauncherName, String containerLauncherClassname) {
+    super(containerLauncherName, containerLauncherClassname);
+  }
+
+  public static ContainerLauncherDescriptor create(String containerLauncherName, String containerLauncherClassname) {
+    return new ContainerLauncherDescriptor(containerLauncherName, containerLauncherClassname);
+  }
+}

http://git-wip-us.apache.org/repos/asf/tez/blob/cfd625e9/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ServicePluginsDescriptor.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ServicePluginsDescriptor.java b/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ServicePluginsDescriptor.java
new file mode 100644
index 0000000..8df102a
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ServicePluginsDescriptor.java
@@ -0,0 +1,96 @@
+/*
+ * 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.serviceplugins.api;
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
+public class ServicePluginsDescriptor {
+
+  private final boolean enableContainers;
+  private final boolean enableUber;
+
+  private TaskSchedulerDescriptor[] taskSchedulerDescriptors;
+  private ContainerLauncherDescriptor[] containerLauncherDescriptors;
+  private TaskCommunicatorDescriptor[] taskCommunicatorDescriptors;
+
+  private ServicePluginsDescriptor(boolean enableContainers, boolean enableUber,
+                                   TaskSchedulerDescriptor[] taskSchedulerDescriptor,
+                                   ContainerLauncherDescriptor[] containerLauncherDescriptors,
+                                   TaskCommunicatorDescriptor[] taskCommunicatorDescriptors) {
+    this.enableContainers = enableContainers;
+    this.enableUber = enableUber;
+    Preconditions.checkArgument(taskSchedulerDescriptors == null || taskSchedulerDescriptors.length > 0,
+        "TaskSchedulerDescriptors should either not be specified or at least 1 should be provided");
+    this.taskSchedulerDescriptors = taskSchedulerDescriptor;
+    Preconditions.checkArgument(containerLauncherDescriptors == null || containerLauncherDescriptors.length > 0,
+        "ContainerLauncherDescriptor should either not be specified or at least 1 should be provided");
+    this.containerLauncherDescriptors = containerLauncherDescriptors;
+    Preconditions.checkArgument(taskCommunicatorDescriptors == null || taskCommunicatorDescriptors.length > 0,
+        "TaskCommunicatorDescriptors should either not be specified or at least 1 should be provided");
+    this.taskCommunicatorDescriptors = taskCommunicatorDescriptors;
+  }
+
+  public static ServicePluginsDescriptor create(TaskSchedulerDescriptor[] taskSchedulerDescriptor,
+                                                ContainerLauncherDescriptor[] containerLauncherDescriptors,
+                                                TaskCommunicatorDescriptor[] taskCommunicatorDescriptors) {
+    return new ServicePluginsDescriptor(true, false, taskSchedulerDescriptor,
+        containerLauncherDescriptors, taskCommunicatorDescriptors);
+  }
+
+  public static ServicePluginsDescriptor create(boolean enableUber,
+                                                TaskSchedulerDescriptor[] taskSchedulerDescriptor,
+                                                ContainerLauncherDescriptor[] containerLauncherDescriptors,
+                                                TaskCommunicatorDescriptor[] taskCommunicatorDescriptors) {
+    return new ServicePluginsDescriptor(true, enableUber, taskSchedulerDescriptor,
+        containerLauncherDescriptors, taskCommunicatorDescriptors);
+  }
+
+  public static ServicePluginsDescriptor create(boolean enableContainers, boolean enableUber,
+                                                TaskSchedulerDescriptor[] taskSchedulerDescriptor,
+                                                ContainerLauncherDescriptor[] containerLauncherDescriptors,
+                                                TaskCommunicatorDescriptor[] taskCommunicatorDescriptors) {
+    return new ServicePluginsDescriptor(enableContainers, enableUber, taskSchedulerDescriptor,
+        containerLauncherDescriptors, taskCommunicatorDescriptors);
+  }
+
+  public static ServicePluginsDescriptor create(boolean enableUber) {
+    return new ServicePluginsDescriptor(true, enableUber, null, null, null);
+  }
+
+
+  public boolean areContainersEnabled() {
+    return enableContainers;
+  }
+
+  public boolean isUberEnabled() {
+    return enableUber;
+  }
+
+  public TaskSchedulerDescriptor[] getTaskSchedulerDescriptors() {
+    return taskSchedulerDescriptors;
+  }
+
+  public ContainerLauncherDescriptor[] getContainerLauncherDescriptors() {
+    return containerLauncherDescriptors;
+  }
+
+  public TaskCommunicatorDescriptor[] getTaskCommunicatorDescriptors() {
+    return taskCommunicatorDescriptors;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tez/blob/cfd625e9/tez-api/src/main/java/org/apache/tez/serviceplugins/api/TaskCommunicatorDescriptor.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/serviceplugins/api/TaskCommunicatorDescriptor.java b/tez-api/src/main/java/org/apache/tez/serviceplugins/api/TaskCommunicatorDescriptor.java
new file mode 100644
index 0000000..57ac385
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/serviceplugins/api/TaskCommunicatorDescriptor.java
@@ -0,0 +1,33 @@
+/*
+ * 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.serviceplugins.api;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.tez.dag.api.NamedEntityDescriptor;
+
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
+public class TaskCommunicatorDescriptor extends NamedEntityDescriptor<TaskCommunicatorDescriptor> {
+
+
+  private TaskCommunicatorDescriptor(String taskCommName, String taskCommClassname) {
+    super(taskCommName, taskCommClassname);
+  }
+
+  public static TaskCommunicatorDescriptor create(String taskCommName, String taskCommClassname) {
+    return new TaskCommunicatorDescriptor(taskCommName, taskCommClassname);
+  }
+}

http://git-wip-us.apache.org/repos/asf/tez/blob/cfd625e9/tez-api/src/main/java/org/apache/tez/serviceplugins/api/TaskSchedulerDescriptor.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/serviceplugins/api/TaskSchedulerDescriptor.java b/tez-api/src/main/java/org/apache/tez/serviceplugins/api/TaskSchedulerDescriptor.java
new file mode 100644
index 0000000..12e0919
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/serviceplugins/api/TaskSchedulerDescriptor.java
@@ -0,0 +1,32 @@
+/*
+ * 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.serviceplugins.api;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.tez.dag.api.NamedEntityDescriptor;
+
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
+public class TaskSchedulerDescriptor extends NamedEntityDescriptor<TaskSchedulerDescriptor> {
+
+  private TaskSchedulerDescriptor(String taskSchedulerName, String schedulerClassname) {
+    super(taskSchedulerName, schedulerClassname);
+  }
+
+  public static TaskSchedulerDescriptor create(String taskSchedulerName, String schedulerClassName) {
+    return new TaskSchedulerDescriptor(taskSchedulerName, schedulerClassName);
+  }
+}

http://git-wip-us.apache.org/repos/asf/tez/blob/cfd625e9/tez-api/src/main/proto/DAGApiRecords.proto
----------------------------------------------------------------------
diff --git a/tez-api/src/main/proto/DAGApiRecords.proto b/tez-api/src/main/proto/DAGApiRecords.proto
index 959d4e6..ebe3259 100644
--- a/tez-api/src/main/proto/DAGApiRecords.proto
+++ b/tez-api/src/main/proto/DAGApiRecords.proto
@@ -127,6 +127,14 @@ message RootInputLeafOutputProto {
   optional TezEntityDescriptorProto controller_descriptor = 3;
 }
 
+message VertexExecutionContextProto {
+  optional bool execute_in_am = 1;
+  optional bool execute_in_containers = 2;
+  optional string task_scheduler_name = 3;
+  optional string container_launcher_name = 4;
+  optional string task_comm_name = 5;
+}
+
 message VertexPlan {
   required string name = 1;
   required PlanVertexType type = 2;
@@ -139,6 +147,7 @@ message VertexPlan {
   repeated RootInputLeafOutputProto outputs = 9;
   optional TezEntityDescriptorProto vertex_manager_plugin = 10;
   optional ConfigurationProto vertexConf = 11;
+  optional VertexExecutionContextProto execution_context = 12;
 }
 
 message PlanEdgeProperty {
@@ -162,8 +171,23 @@ message EdgePlan {
   optional TezEntityDescriptorProto edge_manager = 9;
 }
 
+message TezNamedEntityDescriptorProto {
+  optional string name = 1;
+  optional TezEntityDescriptorProto entity_descriptor = 2;
+}
+
+
+message AMPluginDescriptorProto {
+  optional bool containers_enabled = 1 [default = true];
+  optional bool uber_enabled = 2 [default = false];
+  repeated TezNamedEntityDescriptorProto task_scedulers = 3;
+  repeated TezNamedEntityDescriptorProto container_launchers = 4;
+  repeated TezNamedEntityDescriptorProto task_communicators = 5;
+}
+
 message ConfigurationProto {
   repeated PlanKeyValuePair confKeyValues = 1;
+  optional AMPluginDescriptorProto am_plugin_descriptor = 2;
 }
 
 message DAGPlan {
@@ -175,6 +199,7 @@ message DAGPlan {
   repeated PlanVertexGroupInfo vertex_groups = 6;
   repeated PlanLocalResource local_resource = 7;
   optional string dag_info = 8;
+  optional VertexExecutionContextProto default_execution_context = 9;
 }
 
 // DAG monitoring messages

http://git-wip-us.apache.org/repos/asf/tez/blob/cfd625e9/tez-api/src/test/java/org/apache/tez/client/TestTezClientUtils.java
----------------------------------------------------------------------
diff --git a/tez-api/src/test/java/org/apache/tez/client/TestTezClientUtils.java b/tez-api/src/test/java/org/apache/tez/client/TestTezClientUtils.java
index 2d4e005..8946ef0 100644
--- a/tez-api/src/test/java/org/apache/tez/client/TestTezClientUtils.java
+++ b/tez-api/src/test/java/org/apache/tez/client/TestTezClientUtils.java
@@ -261,7 +261,7 @@ public class TestTezClientUtils {
     ApplicationSubmissionContext appSubmissionContext =
         TezClientUtils.createApplicationSubmissionContext(appId, dag, "amName", amConf,
             new HashMap<String, LocalResource>(), credentials, false, new TezApiVersionInfo(),
-            mock(HistoryACLPolicyManager.class));
+            mock(HistoryACLPolicyManager.class), null);
 
     ContainerLaunchContext amClc = appSubmissionContext.getAMContainerSpec();
     Map<String, ByteBuffer> amServiceData = amClc.getServiceData();
@@ -294,7 +294,7 @@ public class TestTezClientUtils {
     ApplicationSubmissionContext appSubmissionContext =
         TezClientUtils.createApplicationSubmissionContext(appId, dag, "amName", amConf,
             new HashMap<String, LocalResource>(), credentials, false, new TezApiVersionInfo(),
-            mock(HistoryACLPolicyManager.class));
+            mock(HistoryACLPolicyManager.class), null);
 
     List<String> expectedCommands = new LinkedList<String>();
     expectedCommands.add("-Dlog4j.configuratorClass=org.apache.tez.common.TezLog4jConfigurator");
@@ -334,7 +334,7 @@ public class TestTezClientUtils {
     ApplicationSubmissionContext appSubmissionContext =
         TezClientUtils.createApplicationSubmissionContext(appId, dag, "amName", amConf,
             new HashMap<String, LocalResource>(), credentials, false, new TezApiVersionInfo(),
-            mock(HistoryACLPolicyManager.class));
+            mock(HistoryACLPolicyManager.class), null);
 
     List<String> expectedCommands = new LinkedList<String>();
     expectedCommands.add("-Dlog4j.configuratorClass=org.apache.tez.common.TezLog4jConfigurator");
@@ -516,7 +516,7 @@ public class TestTezClientUtils {
     expected.put("property1", val1);
     expected.put("property2", expVal2);
 
-    ConfigurationProto confProto = TezClientUtils.createFinalConfProtoForApp(conf, null);
+    ConfigurationProto confProto = TezClientUtils.createFinalConfProtoForApp(conf, null, null);
 
     for (PlanKeyValuePair kvPair : confProto.getConfKeyValuesList()) {
       String v = expected.remove(kvPair.getKey());
@@ -620,7 +620,7 @@ public class TestTezClientUtils {
       srcConf.set(entry.getKey(), entry.getValue());
     }
 
-    ConfigurationProto confProto = TezClientUtils.createFinalConfProtoForApp(srcConf);
+    ConfigurationProto confProto = TezClientUtils.createFinalConfProtoForApp(srcConf, null, null);
 
     for (PlanKeyValuePair kvPair : confProto.getConfKeyValuesList()) {
       String val = confMap.remove(kvPair.getKey());
@@ -677,4 +677,6 @@ public class TestTezClientUtils {
     Assert.assertTrue(resourceNames.contains("dir2-f.txt"));
   }
 
+  // TODO TEZ-2003 Add test to validate ServicePluginDescriptor propagation
+
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/cfd625e9/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 532e83c..1fb7ff9 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
@@ -57,7 +57,7 @@ public class TezUtilsInternal {
 
   private static final Logger LOG = LoggerFactory.getLogger(TezUtilsInternal.class);
 
-  public static void addUserSpecifiedTezConfiguration(String baseDir, Configuration conf) throws
+  public static ConfigurationProto readUserSpecifiedTezConfiguration(String baseDir) throws
       IOException {
     FileInputStream confPBBinaryStream = null;
     ConfigurationProto.Builder confProtoBuilder = ConfigurationProto.newBuilder();
@@ -72,14 +72,41 @@ public class TezUtilsInternal {
     }
 
     ConfigurationProto confProto = confProtoBuilder.build();
+    return confProto;
+  }
 
-    List<PlanKeyValuePair> kvPairList = confProto.getConfKeyValuesList();
+  public static void addUserSpecifiedTezConfiguration(Configuration conf,
+                                                      List<PlanKeyValuePair> kvPairList) {
     if (kvPairList != null && !kvPairList.isEmpty()) {
       for (PlanKeyValuePair kvPair : kvPairList) {
         conf.set(kvPair.getKey(), kvPair.getValue());
       }
     }
   }
+//
+//  public static void addUserSpecifiedTezConfiguration(String baseDir, Configuration conf) throws
+//      IOException {
+//    FileInputStream confPBBinaryStream = null;
+//    ConfigurationProto.Builder confProtoBuilder = ConfigurationProto.newBuilder();
+//    try {
+//      confPBBinaryStream =
+//          new FileInputStream(new File(baseDir, TezConstants.TEZ_PB_BINARY_CONF_NAME));
+//      confProtoBuilder.mergeFrom(confPBBinaryStream);
+//    } finally {
+//      if (confPBBinaryStream != null) {
+//        confPBBinaryStream.close();
+//      }
+//    }
+//
+//    ConfigurationProto confProto = confProtoBuilder.build();
+//
+//    List<PlanKeyValuePair> kvPairList = confProto.getConfKeyValuesList();
+//    if (kvPairList != null && !kvPairList.isEmpty()) {
+//      for (PlanKeyValuePair kvPair : kvPairList) {
+//        conf.set(kvPair.getKey(), kvPair.getValue());
+//      }
+//    }
+//  }
 
 
   public static byte[] compressBytes(byte[] inBytes) throws IOException {

http://git-wip-us.apache.org/repos/asf/tez/blob/cfd625e9/tez-dag/src/main/java/org/apache/tez/client/LocalClient.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/client/LocalClient.java b/tez-dag/src/main/java/org/apache/tez/client/LocalClient.java
index 1bb2002..508f817 100644
--- a/tez-dag/src/main/java/org/apache/tez/client/LocalClient.java
+++ b/tez-dag/src/main/java/org/apache/tez/client/LocalClient.java
@@ -341,7 +341,7 @@ public class LocalClient extends FrameworkClient {
       String[] localDirs, String[] logDirs, Credentials credentials, String jobUserName) {
     return new DAGAppMaster(applicationAttemptId, cId, currentHost, nmPort, nmHttpPort,
         new SystemClock(), appSubmitTime, isSession, userDir, localDirs, logDirs,
-        versionInfo.getVersion(), 1, credentials, jobUserName);
+        versionInfo.getVersion(), 1, credentials, jobUserName, null);
   }
 
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/cfd625e9/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 f3914d8..8388cfb 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
@@ -40,6 +40,7 @@ import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Iterator;
 import java.util.LinkedHashMap;
+import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
@@ -61,7 +62,11 @@ import com.google.common.collect.HashBiMap;
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.cli.GnuParser;
 import org.apache.commons.cli.Options;
+import org.apache.tez.dag.api.NamedEntityDescriptor;
 import org.apache.tez.dag.api.SessionNotRunning;
+import org.apache.tez.dag.api.records.DAGProtos.AMPluginDescriptorProto;
+import org.apache.tez.dag.api.records.DAGProtos.ConfigurationProto;
+import org.apache.tez.dag.api.records.DAGProtos.TezNamedEntityDescriptorProto;
 import org.apache.tez.dag.app.dag.event.DAGAppMasterEventDagCleanup;
 import org.apache.tez.dag.history.events.DAGRecoveredEvent;
 import org.apache.tez.dag.records.TezTaskAttemptID;
@@ -221,6 +226,7 @@ public class DAGAppMaster extends AbstractService {
   private final String workingDirectory;
   private final String[] localDirs;
   private final String[] logDirs;
+  private final AMPluginDescriptorProto amPluginDescriptorProto;
   private ContainerSignatureMatcher containerSignatureMatcher;
   private AMContainerMap containers;
   private AMNodeTracker nodes;
@@ -312,7 +318,7 @@ public class DAGAppMaster extends AbstractService {
       ContainerId containerId, String nmHost, int nmPort, int nmHttpPort,
       Clock clock, long appSubmitTime, boolean isSession, String workingDirectory,
       String [] localDirs, String[] logDirs, String clientVersion, int maxAppAttempts,
-      Credentials credentials, String jobUserName) {
+      Credentials credentials, String jobUserName, AMPluginDescriptorProto pluginDescriptorProto) {
     super(DAGAppMaster.class.getName());
     this.clock = clock;
     this.startTime = clock.getTime();
@@ -332,6 +338,7 @@ public class DAGAppMaster extends AbstractService {
     this.clientVersion = clientVersion;
     this.maxAppAttempts = maxAppAttempts;
     this.amCredentials = credentials;
+    this.amPluginDescriptorProto = pluginDescriptorProto;
     this.appMasterUgi = UserGroupInformation
         .createRemoteUser(jobUserName);
     this.appMasterUgi.addCredentials(amCredentials);
@@ -380,28 +387,47 @@ 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;
+    List<NamedEntityDescriptor> taskSchedulerDescriptors;
+    List<NamedEntityDescriptor> containerLauncherDescriptors;
+    List<NamedEntityDescriptor> taskCommunicatorDescriptors;
+    boolean tezYarnEnabled = true;
+    boolean uberEnabled = false;
 
-    String[] taskSchedulerClassIdentifiers = parsePlugins(taskSchedulers,
-        conf.getTrimmedStrings(TezConfiguration.TEZ_AM_TASK_SCHEDULERS,
-            tezDefaultClassIdentifier),
-        TezConfiguration.TEZ_AM_TASK_SCHEDULERS);
+    if (!isLocal) {
+      if (amPluginDescriptorProto == null) {
+        tezYarnEnabled = true;
+        uberEnabled = false;
+      } else {
+        tezYarnEnabled = amPluginDescriptorProto.getContainersEnabled();
+        uberEnabled = amPluginDescriptorProto.getUberEnabled();
+      }
+    } else {
+      tezYarnEnabled = false;
+      uberEnabled = true;
+    }
+
+    taskSchedulerDescriptors = parsePlugin(taskSchedulers,
+        (amPluginDescriptorProto == null || amPluginDescriptorProto.getTaskScedulersCount() == 0 ?
+            null :
+            amPluginDescriptorProto.getTaskScedulersList()),
+        tezYarnEnabled, uberEnabled);
 
-    String[] containerLauncherClassIdentifiers = parsePlugins(containerLaunchers,
-        conf.getTrimmedStrings(TezConfiguration.TEZ_AM_CONTAINER_LAUNCHERS,
-            tezDefaultClassIdentifier),
-        TezConfiguration.TEZ_AM_CONTAINER_LAUNCHERS);
+    containerLauncherDescriptors = parsePlugin(containerLaunchers,
+        (amPluginDescriptorProto == null ||
+            amPluginDescriptorProto.getContainerLaunchersCount() == 0 ? null :
+            amPluginDescriptorProto.getContainerLaunchersList()),
+        tezYarnEnabled, uberEnabled);
 
-    String[] taskCommunicatorClassIdentifiers = parsePlugins(taskCommunicators,
-        conf.getTrimmedStrings(TezConfiguration.TEZ_AM_TASK_COMMUNICATORS,
-            tezDefaultClassIdentifier),
-        TezConfiguration.TEZ_AM_TASK_COMMUNICATORS);
+    taskCommunicatorDescriptors = parsePlugin(taskCommunicators,
+        (amPluginDescriptorProto == null ||
+            amPluginDescriptorProto.getTaskCommunicatorsCount() == 0 ? null :
+            amPluginDescriptorProto.getTaskCommunicatorsList()),
+        tezYarnEnabled, uberEnabled);
 
-    LOG.info(buildPluginComponentLog(taskSchedulerClassIdentifiers, taskSchedulers, "TaskSchedulers"));
-    LOG.info(buildPluginComponentLog(containerLauncherClassIdentifiers, containerLaunchers, "ContainerLaunchers"));
-    LOG.info(buildPluginComponentLog(taskCommunicatorClassIdentifiers, taskCommunicators, "TaskCommunicators"));
+
+    LOG.info(buildPluginComponentLog(taskSchedulerDescriptors, taskSchedulers, "TaskSchedulers"));
+    LOG.info(buildPluginComponentLog(containerLauncherDescriptors, containerLaunchers, "ContainerLaunchers"));
+    LOG.info(buildPluginComponentLog(taskCommunicatorDescriptors, taskCommunicators, "TaskCommunicators"));
 
     boolean disableVersionCheck = conf.getBoolean(
         TezConfiguration.TEZ_AM_DISABLE_CLIENT_VERSION_CHECK,
@@ -468,7 +494,7 @@ public class DAGAppMaster extends AbstractService {
 
     //service to handle requests to TaskUmbilicalProtocol
     taskAttemptListener = createTaskAttemptListener(context,
-        taskHeartbeatHandler, containerHeartbeatHandler, taskCommunicatorClassIdentifiers, isLocal);
+        taskHeartbeatHandler, containerHeartbeatHandler, taskCommunicatorDescriptors, isLocal);
     addIfService(taskAttemptListener, true);
 
     containerSignatureMatcher = createContainerSignatureMatcher();
@@ -516,7 +542,7 @@ public class DAGAppMaster extends AbstractService {
 
     this.taskSchedulerEventHandler = new TaskSchedulerEventHandler(context,
         clientRpcServer, dispatcher.getEventHandler(), containerSignatureMatcher, webUIService,
-        taskSchedulerClassIdentifiers, isLocal);
+        taskSchedulerDescriptors, isLocal);
     addIfService(taskSchedulerEventHandler, true);
 
     if (enableWebUIService()) {
@@ -534,7 +560,7 @@ public class DAGAppMaster extends AbstractService {
         taskSchedulerEventHandler);
     addIfServiceDependency(taskSchedulerEventHandler, clientRpcServer);
 
-    this.containerLauncherRouter = createContainerLauncherRouter(conf, containerLauncherClassIdentifiers, isLocal);
+    this.containerLauncherRouter = createContainerLauncherRouter(conf, containerLauncherDescriptors, isLocal);
     addIfService(containerLauncherRouter, true);
     dispatcher.register(NMCommunicatorEventType.class, containerLauncherRouter);
 
@@ -1044,11 +1070,11 @@ public class DAGAppMaster extends AbstractService {
   protected TaskAttemptListener createTaskAttemptListener(AppContext context,
                                                           TaskHeartbeatHandler thh,
                                                           ContainerHeartbeatHandler chh,
-                                                          String[] taskCommunicatorClasses,
+                                                          List<NamedEntityDescriptor> entityDescriptors,
                                                           boolean isLocal) {
     TaskAttemptListener lis =
         new TaskAttemptListenerImpTezDag(context, thh, chh,
-            taskCommunicatorClasses, amConf, isLocal);
+            entityDescriptors, amConf, isLocal);
     return lis;
   }
 
@@ -1070,11 +1096,11 @@ public class DAGAppMaster extends AbstractService {
   }
 
   protected ContainerLauncherRouter createContainerLauncherRouter(Configuration conf,
-                                                                  String[] containerLauncherClasses,
+                                                                  List<NamedEntityDescriptor> containerLauncherDescriptors,
                                                                   boolean isLocal) throws
       UnknownHostException {
     return new ContainerLauncherRouter(conf, context, taskAttemptListener, workingDirectory,
-        containerLauncherClasses, isLocal);
+        containerLauncherDescriptors, isLocal);
   }
 
   public ApplicationId getAppID() {
@@ -2134,7 +2160,16 @@ public class DAGAppMaster extends AbstractService {
 
       // TODO Does this really need to be a YarnConfiguration ?
       Configuration conf = new Configuration(new YarnConfiguration());
-      TezUtilsInternal.addUserSpecifiedTezConfiguration(System.getenv(Environment.PWD.name()), conf);
+
+      ConfigurationProto confProto =
+          TezUtilsInternal.readUserSpecifiedTezConfiguration(System.getenv(Environment.PWD.name()));
+      TezUtilsInternal.addUserSpecifiedTezConfiguration(conf, confProto.getConfKeyValuesList());
+
+      AMPluginDescriptorProto amPluginDescriptorProto = null;
+      if (confProto.hasAmPluginDescriptor()) {
+        amPluginDescriptorProto = confProto.getAmPluginDescriptor();
+      }
+
       UserGroupInformation.setConfiguration(conf);
       Credentials credentials = UserGroupInformation.getCurrentUser().getCredentials();
 
@@ -2146,7 +2181,7 @@ public class DAGAppMaster extends AbstractService {
               System.getenv(Environment.PWD.name()),
               TezCommonUtils.getTrimmedStrings(System.getenv(Environment.LOCAL_DIRS.name())),
               TezCommonUtils.getTrimmedStrings(System.getenv(Environment.LOG_DIRS.name())),
-              clientVersion, maxAppAttempts, credentials, jobUserName);
+              clientVersion, maxAppAttempts, credentials, jobUserName, amPluginDescriptorProto);
       ShutdownHookManager.get().addShutdownHook(
         new DAGAppMasterShutdownHook(appMaster), SHUTDOWN_HOOK_PRIORITY);
 
@@ -2252,7 +2287,7 @@ public class DAGAppMaster extends AbstractService {
 
     LOG.info("Running DAG: " + dagPlan.getName());
     String timeStamp = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss").format(Calendar.getInstance().getTime());
-    System.err.println(timeStamp + " Running Dag: "+ newDAG.getID());
+    System.err.println(timeStamp + " Running Dag: " + newDAG.getID());
     System.out.println(timeStamp + " Running Dag: "+ newDAG.getID());
     // Job name is the same as the app name until we support multiple dags
     // for an app later
@@ -2358,60 +2393,51 @@ public class DAGAppMaster extends AbstractService {
         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) {
-    // 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);
-
-    String[] classNames = new String[pluginStrings.length];
+  private static List<NamedEntityDescriptor> parsePlugin(
+      BiMap<String, Integer> pluginMap, List<TezNamedEntityDescriptorProto> namedEntityDescriptorProtos,
+      boolean tezYarnEnabled, boolean uberEnabled) {
 
     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;
+
+    List<NamedEntityDescriptor> resultList = new LinkedList<>();
+
+    if (tezYarnEnabled) {
+      // Default classnames will be populated by individual components
+      NamedEntityDescriptor r = new NamedEntityDescriptor(
+          TezConstants.getTezYarnServicePluginName(), null);
+      resultList.add(r);
+      pluginMap.put(TezConstants.getTezYarnServicePluginName(), index);
+      index++;
+    }
+
+    if (uberEnabled) {
+      // Default classnames will be populated by individual components
+      NamedEntityDescriptor r = new NamedEntityDescriptor(
+          TezConstants.getTezUberServicePluginName(), null);
+      resultList.add(r);
+      pluginMap.put(TezConstants.getTezUberServicePluginName(), index);
       index++;
     }
-    return classNames;
+
+    if (namedEntityDescriptorProtos != null) {
+      for (TezNamedEntityDescriptorProto namedEntityDescriptorProto : namedEntityDescriptorProtos) {
+        resultList.add(DagTypeConverters
+            .convertNamedDescriptorFromProto(namedEntityDescriptorProto));
+        pluginMap.put(resultList.get(index).getEntityName(), index);
+        index++;
+      }
+    }
+    return resultList;
   }
 
-  String buildPluginComponentLog(String[] classIdentifiers, BiMap<String, Integer> map,
+  String buildPluginComponentLog(List<NamedEntityDescriptor> namedEntityDescriptors, 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++) {
+    for (int i = 0; i < namedEntityDescriptors.size(); i++) {
       sb.append("[").append(i).append(":").append(map.inverse().get(i))
-          .append(":").append(classIdentifiers[i]).append("]");
-      if (i != classIdentifiers.length - 1) {
+          .append(":").append(namedEntityDescriptors.get(i).getClassName()).append("]");
+      if (i != namedEntityDescriptors.size() - 1) {
         sb.append(",");
       }
     }

http://git-wip-us.apache.org/repos/asf/tez/blob/cfd625e9/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 599c208..1e34184 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
@@ -27,8 +27,11 @@ import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 
 import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.Lists;
 import org.apache.commons.collections4.ListUtils;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.tez.dag.api.NamedEntityDescriptor;
+import org.apache.tez.dag.api.TezConstants;
 import org.apache.tez.serviceplugins.api.ContainerEndReason;
 import org.apache.tez.dag.app.dag.event.TaskAttemptEvent;
 import org.apache.tez.dag.app.dag.event.TaskAttemptEventStatusUpdate;
@@ -46,7 +49,6 @@ import org.apache.tez.dag.api.TaskCommunicator;
 import org.apache.tez.dag.api.TaskCommunicatorContext;
 import org.apache.tez.serviceplugins.api.TaskAttemptEndReason;
 import org.apache.tez.dag.api.TaskHeartbeatResponse;
-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;
@@ -100,28 +102,28 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
 
   public TaskAttemptListenerImpTezDag(AppContext context,
                                       TaskHeartbeatHandler thh, ContainerHeartbeatHandler chh,
-                                      String [] taskCommunicatorClassIdentifiers,
+                                      List<NamedEntityDescriptor> taskCommunicatorDescriptors,
                                       Configuration conf,
                                       boolean isPureLocalMode) {
     super(TaskAttemptListenerImpTezDag.class.getName());
     this.context = context;
     this.taskHeartbeatHandler = thh;
     this.containerHeartbeatHandler = chh;
-    if (taskCommunicatorClassIdentifiers == null || taskCommunicatorClassIdentifiers.length == 0) {
+    if (taskCommunicatorDescriptors == null || taskCommunicatorDescriptors.isEmpty()) {
       if (isPureLocalMode) {
-        taskCommunicatorClassIdentifiers =
-            new String[]{TezConstants.TEZ_AM_SERVICE_PLUGINS_LOCAL_MODE_NAME_DEFAULT};
+        taskCommunicatorDescriptors = Lists.newArrayList(new NamedEntityDescriptor(
+            TezConstants.getTezUberServicePluginName(), null));
       } else {
-        taskCommunicatorClassIdentifiers =
-            new String[]{TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT};
+        taskCommunicatorDescriptors = Lists.newArrayList(new NamedEntityDescriptor(
+            TezConstants.getTezYarnServicePluginName(), null));
       }
     }
-    this.taskCommunicators = new TaskCommunicator[taskCommunicatorClassIdentifiers.length];
-    this.taskCommunicatorContexts = new TaskCommunicatorContext[taskCommunicatorClassIdentifiers.length];
-    this.taskCommunicatorServiceWrappers = new ServicePluginLifecycleAbstractService[taskCommunicatorClassIdentifiers.length];
-    for (int i = 0 ; i < taskCommunicatorClassIdentifiers.length ; i++) {
+    this.taskCommunicators = new TaskCommunicator[taskCommunicatorDescriptors.size()];
+    this.taskCommunicatorContexts = new TaskCommunicatorContext[taskCommunicatorDescriptors.size()];
+    this.taskCommunicatorServiceWrappers = new ServicePluginLifecycleAbstractService[taskCommunicatorDescriptors.size()];
+    for (int i = 0 ; i < taskCommunicatorDescriptors.size() ; i++) {
       taskCommunicatorContexts[i] = new TaskCommunicatorContextImpl(context, this, conf, i);
-      taskCommunicators[i] = createTaskCommunicator(taskCommunicatorClassIdentifiers[i], i);
+      taskCommunicators[i] = createTaskCommunicator(taskCommunicatorDescriptors.get(i), i);
       taskCommunicatorServiceWrappers[i] = new ServicePluginLifecycleAbstractService(taskCommunicators[i]);
     }
     // TODO TEZ-2118 Start using taskCommunicator indices properly
@@ -143,17 +145,18 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
     }
   }
 
-  private TaskCommunicator createTaskCommunicator(String taskCommClassIdentifier, int taskCommIndex) {
-    if (taskCommClassIdentifier.equals(TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT)) {
+  private TaskCommunicator createTaskCommunicator(NamedEntityDescriptor taskCommDescriptor, int taskCommIndex) {
+    if (taskCommDescriptor.getEntityName().equals(TezConstants.getTezYarnServicePluginName())) {
       LOG.info("Using Default Task Communicator");
       return createTezTaskCommunicator(taskCommunicatorContexts[taskCommIndex]);
-    } else if (taskCommClassIdentifier.equals(TezConstants.TEZ_AM_SERVICE_PLUGINS_LOCAL_MODE_NAME_DEFAULT)) {
+    } else if (taskCommDescriptor.getEntityName().equals(TezConstants.getTezUberServicePluginName())) {
       LOG.info("Using Default Local Task Communicator");
       return new TezLocalTaskCommunicatorImpl(taskCommunicatorContexts[taskCommIndex]);
     } else {
-      LOG.info("Using TaskCommunicator: " + taskCommClassIdentifier);
+      // TODO TEZ-2003. Use the payload
+      LOG.info("Using TaskCommunicator {}:{} " + taskCommDescriptor.getEntityName(), taskCommDescriptor.getClassName());
       Class<? extends TaskCommunicator> taskCommClazz = (Class<? extends TaskCommunicator>) ReflectionUtils
-          .getClazz(taskCommClassIdentifier);
+          .getClazz(taskCommDescriptor.getClassName());
       try {
         Constructor<? extends TaskCommunicator> ctor = taskCommClazz.getConstructor(TaskCommunicatorContext.class);
         ctor.setAccessible(true);

http://git-wip-us.apache.org/repos/asf/tez/blob/cfd625e9/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 458362f..335239e 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
@@ -98,4 +98,6 @@ public interface DAG {
 
   StateChangeNotifier getStateChangeNotifier();
 
+  org.apache.tez.dag.api.Vertex.VertexExecutionContext getDefaultExecutionContext();
+
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/cfd625e9/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 ec2ef66..25518b0 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
@@ -717,6 +717,15 @@ public class DAGImpl implements org.apache.tez.dag.app.dag.DAG,
   }
 
   @Override
+  public org.apache.tez.dag.api.Vertex.VertexExecutionContext getDefaultExecutionContext() {
+    if (jobPlan.hasDefaultExecutionContext()) {
+      return DagTypeConverters.convertFromProto(jobPlan.getDefaultExecutionContext());
+    } else {
+      return null;
+    }
+  }
+
+  @Override
   public TezCounters getAllCounters() {
 
     readLock.lock();

http://git-wip-us.apache.org/repos/asf/tez/blob/cfd625e9/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 bdab984..2e8f218 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
@@ -995,14 +995,37 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex, EventHandl
         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);
+        isLocal ? TezConstants.getTezUberServicePluginName() :
+            TezConstants.getTezYarnServicePluginName();
+
+    org.apache.tez.dag.api.Vertex.VertexExecutionContext execContext = dag.getDefaultExecutionContext();
+    if (vertexPlan.hasExecutionContext()) {
+      execContext = DagTypeConverters.convertFromProto(vertexPlan.getExecutionContext());
+      LOG.info("Using ExecutionContext from Vertex for Vertex {}", vertexName);
+    } else if (execContext != null) {
+      LOG.info("Using ExecutionContext from DAG for Vertex {}", vertexName);
+    }
+    if (execContext != null) {
+      if (execContext.shouldExecuteInAm()) {
+        tezDefaultComponentName = TezConstants.getTezUberServicePluginName();
+      }
+    }
+
+    String taskSchedulerName = tezDefaultComponentName;
+    String containerLauncherName = tezDefaultComponentName;
+    String taskCommName = tezDefaultComponentName;
+
+    if (execContext != null) {
+      if (execContext.getTaskSchedulerName() != null) {
+        taskSchedulerName = execContext.getTaskSchedulerName();
+      }
+      if (execContext.getContainerLauncherName() != null) {
+        containerLauncherName = execContext.getContainerLauncherName();
+      }
+      if (execContext.getTaskCommName() != null) {
+        taskCommName = execContext.getTaskCommName();
+      }
+    }
 
     LOG.info("Vertex: " + logIdentifier + " configured with TaskScheduler=" + taskSchedulerName +
         ", ContainerLauncher=" + containerLauncherName + ", TaskComm=" + taskCommName);

http://git-wip-us.apache.org/repos/asf/tez/blob/cfd625e9/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 34c7bc0..cba5c80 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,11 +30,11 @@ import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 
 import org.apache.commons.lang.exception.ExceptionUtils;
+import org.apache.tez.dag.api.TezConstants;
 import org.apache.tez.serviceplugins.api.ContainerLaunchRequest;
 import org.apache.tez.serviceplugins.api.ContainerLauncher;
 import org.apache.tez.serviceplugins.api.ContainerLauncherContext;
 import org.apache.tez.serviceplugins.api.ContainerStopRequest;
-import org.apache.tez.dag.api.TezConstants;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -269,8 +269,8 @@ public class ContainerLauncherImpl extends ContainerLauncher {
 
             // nodes where containers will run at *this* point of time. This is
             // *not* the cluster size and doesn't need to be.
-            int numNodes = getContext().getNumNodes(
-                TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT);
+            int numNodes =
+                getContext().getNumNodes(TezConstants.getTezYarnServicePluginName());
             int idealPoolSize = Math.min(limitOnPoolSize, numNodes);
 
             if (poolSize < idealPoolSize) {


[41/50] [abbrv] tez git commit: TEZ-2678. Fix comments from reviews - part 1. (sseth)

Posted by ss...@apache.org.
TEZ-2678. Fix comments from reviews - part 1. (sseth)


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

Branch: refs/heads/TEZ-2003
Commit: fee059b9030f76432b308105ded559c5195f922a
Parents: a960c64
Author: Siddharth Seth <ss...@apache.org>
Authored: Tue Aug 11 11:19:09 2015 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Fri Aug 14 13:47:10 2015 -0700

----------------------------------------------------------------------
 TEZ-2003-CHANGES.txt                            |   1 +
 .../java/org/apache/tez/client/TezClient.java   |   2 +-
 .../org/apache/tez/client/TezClientUtils.java   |  14 +-
 .../main/java/org/apache/tez/dag/api/DAG.java   |  76 ++++++++++-
 .../apache/tez/dag/api/DagTypeConverters.java   |   2 +-
 .../api/ContainerLauncherOperationBase.java     |   8 +-
 .../api/ServicePluginsDescriptor.java           |  13 ++
 .../api/TaskAttemptEndReason.java               |   2 +-
 .../tez/serviceplugins/api/TaskScheduler.java   |   6 +-
 .../java/org/apache/tez/dag/api/TestDAG.java    |   2 +-
 .../org/apache/tez/dag/api/TestDAGPlan.java     | 113 ++++++++++++++-
 .../tez/dag/api/TestDagTypeConverters.java      |  11 +-
 .../org/apache/tez/common/TezUtilsInternal.java |   5 +-
 .../apache/tez/dag/api/TaskCommunicator.java    |  12 +-
 .../tez/dag/api/TaskCommunicatorContext.java    |   2 +-
 .../apache/tez/dag/app/TaskAttemptListener.java |   4 +-
 .../dag/app/TaskAttemptListenerImpTezDag.java   |   9 +-
 .../dag/app/TaskCommunicatorContextImpl.java    |  15 +-
 .../tez/dag/app/TezTaskCommunicatorImpl.java    |  10 +-
 .../apache/tez/dag/app/dag/impl/DAGImpl.java    |  13 +-
 .../tez/dag/app/dag/impl/TaskAttemptImpl.java   |   9 +-
 .../app/launcher/ContainerLauncherRouter.java   |  19 +--
 .../tez/dag/app/rm/AMSchedulerEventTAEnded.java |   8 +-
 .../dag/app/rm/LocalTaskSchedulerService.java   |   2 +-
 .../dag/app/rm/TaskSchedulerEventHandler.java   |   5 +-
 .../dag/app/rm/YarnTaskSchedulerService.java    |   5 +-
 .../dag/app/rm/container/AMContainerImpl.java   |  62 +++++----
 .../tez/dag/app/rm/node/AMNodeTracker.java      |   2 +-
 .../app/TestTaskAttemptListenerImplTezDag.java  |   6 +-
 .../app/TestTaskAttemptListenerImplTezDag2.java |   3 +-
 .../app/TestTaskCommunicatorContextImpl.java    |  85 ++++++++++++
 .../dag/app/TestTaskCommunicatorManager.java    |   4 +-
 .../tez/dag/app/dag/impl/TestTaskAttempt.java   |  12 +-
 .../launcher/TestContainerLauncherRouter.java   |   6 +-
 .../tez/dag/app/rm/TestContainerReuse.java      | 136 +++++++++++++------
 .../app/rm/TestLocalTaskSchedulerService.java   |   4 +-
 .../tez/dag/app/rm/TestTaskScheduler.java       |  18 +--
 .../app/rm/TestTaskSchedulerEventHandler.java   |  13 +-
 .../dag/app/rm/container/TestAMContainer.java   | 127 +++++++++++------
 .../org/apache/tez/examples/JoinValidate.java   |   8 ++
 tez-ext-service-tests/pom.xml                   |   5 -
 .../rm/TezTestServiceTaskSchedulerService.java  |   5 +-
 .../TezTestServiceTaskCommunicatorImpl.java     |  10 +-
 .../tez/service/impl/ContainerRunnerImpl.java   |   2 +-
 .../apache/tez/runtime/task/TezTaskRunner2.java |  16 +--
 .../runtime/task/TaskExecutionTestHelpers.java  |   1 +
 .../runtime/task/TestContainerExecution.java    |   1 +
 47 files changed, 663 insertions(+), 231 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/fee059b9/TEZ-2003-CHANGES.txt
----------------------------------------------------------------------
diff --git a/TEZ-2003-CHANGES.txt b/TEZ-2003-CHANGES.txt
index 75fac88..fd3374e 100644
--- a/TEZ-2003-CHANGES.txt
+++ b/TEZ-2003-CHANGES.txt
@@ -45,5 +45,6 @@ ALL CHANGES:
   TEZ-2126. Add unit tests for verifying multiple schedulers, launchers, communicators.
   TEZ-2698. rebase 08/05
   TEZ-2675. Add javadocs for new pluggable components, fix problems reported by jenkins
+  TEZ-2678. Fix comments from reviews - part 1.
 
 INCOMPATIBLE CHANGES:

http://git-wip-us.apache.org/repos/asf/tez/blob/fee059b9/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 373be81..036b5e8 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
@@ -471,7 +471,7 @@ public class TezClient {
 
     Map<String, LocalResource> tezJarResources = getTezJarResources(sessionCredentials);
     DAGPlan dagPlan = TezClientUtils.prepareAndCreateDAGPlan(dag, amConfig, tezJarResources,
-        usingTezArchiveDeploy, sessionCredentials, aclConfigs);
+        usingTezArchiveDeploy, sessionCredentials, aclConfigs, servicePluginsDescriptor);
 
     SubmitDAGRequestProto.Builder requestBuilder = SubmitDAGRequestProto.newBuilder();
     requestBuilder.setDAGPlan(dagPlan).build();

http://git-wip-us.apache.org/repos/asf/tez/blob/fee059b9/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java b/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java
index 6086fa1..ecf5c07 100644
--- a/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java
+++ b/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java
@@ -609,7 +609,7 @@ public class TezClientUtils {
     if(dag != null) {
       
       DAGPlan dagPB = prepareAndCreateDAGPlan(dag, amConfig, tezJarResources, tezLrsAsArchive,
-          sessionCreds);
+          sessionCreds, servicePluginsDescriptor);
 
       // emit protobuf DAG file style
       Path binaryPath = TezCommonUtils.getTezBinPlanStagingPath(tezSysStagingPath);
@@ -685,18 +685,19 @@ public class TezClientUtils {
   
   static DAGPlan prepareAndCreateDAGPlan(DAG dag, AMConfiguration amConfig,
       Map<String, LocalResource> tezJarResources, boolean tezLrsAsArchive,
-      Credentials credentials) throws IOException {
+      Credentials credentials, ServicePluginsDescriptor servicePluginsDescriptor) throws IOException {
     return prepareAndCreateDAGPlan(dag, amConfig, tezJarResources, tezLrsAsArchive, credentials,
-        null);
+        null, servicePluginsDescriptor);
   }
 
   static DAGPlan prepareAndCreateDAGPlan(DAG dag, AMConfiguration amConfig,
       Map<String, LocalResource> tezJarResources, boolean tezLrsAsArchive,
-      Credentials credentials, Map<String, String> additionalDAGConfigs) throws IOException {
+      Credentials credentials, Map<String, String> additionalDAGConfigs,
+      ServicePluginsDescriptor servicePluginsDescriptor) throws IOException {
     Credentials dagCredentials = setupDAGCredentials(dag, credentials,
         amConfig.getTezConfiguration());
     return dag.createDag(amConfig.getTezConfiguration(), dagCredentials, tezJarResources,
-        amConfig.getBinaryConfLR(), tezLrsAsArchive, additionalDAGConfigs);
+        amConfig.getBinaryConfLR(), tezLrsAsArchive, additionalDAGConfigs, servicePluginsDescriptor);
   }
   
   static void maybeAddDefaultLoggingJavaOpts(String logLevel, List<String> vargs) {
@@ -776,7 +777,7 @@ public class TezClientUtils {
     }
 
     AMPluginDescriptorProto pluginDescriptorProto =
-        DagTypeConverters.convertServicePluginDescriptoToProto(servicePluginsDescriptor);
+        DagTypeConverters.convertServicePluginDescriptorToProto(servicePluginsDescriptor);
     builder.setAmPluginDescriptor(pluginDescriptorProto);
 
     return builder.build();
@@ -1035,4 +1036,5 @@ public class TezClientUtils {
       }
     }
   }
+
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/fee059b9/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 927039a..78bb660 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
@@ -35,6 +35,8 @@ import org.apache.commons.collections4.bidimap.DualLinkedHashBidiMap;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.tez.dag.api.Vertex.VertexExecutionContext;
 import org.apache.tez.dag.api.records.DAGProtos;
+import org.apache.tez.serviceplugins.api.ServicePluginsDescriptor;
+import org.apache.tez.serviceplugins.api.TaskSchedulerDescriptor;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
@@ -714,14 +716,15 @@ public class DAG {
                            Map<String, LocalResource> tezJarResources, LocalResource binaryConfig,
                            boolean tezLrsAsArchive) {
     return createDag(tezConf, extraCredentials, tezJarResources, binaryConfig, tezLrsAsArchive,
-        null);
+        null, null);
   }
 
   // create protobuf message describing DAG
   @Private
   public synchronized DAGPlan createDag(Configuration tezConf, Credentials extraCredentials,
       Map<String, LocalResource> tezJarResources, LocalResource binaryConfig,
-      boolean tezLrsAsArchive, Map<String, String> additionalConfigs) {
+      boolean tezLrsAsArchive, Map<String, String> additionalConfigs,
+                                        ServicePluginsDescriptor servicePluginsDescriptor) {
     verify(true);
 
     DAGPlan.Builder dagBuilder = DAGPlan.newBuilder();
@@ -732,6 +735,7 @@ public class DAG {
 
     // Setup default execution context.
     VertexExecutionContext defaultContext = getDefaultExecutionContext();
+    verifyExecutionContext(defaultContext, servicePluginsDescriptor, "DAGDefault");
     if (defaultContext != null) {
       DAGProtos.VertexExecutionContextProto contextProto = DagTypeConverters.convertToProto(
           defaultContext);
@@ -834,6 +838,7 @@ public class DAG {
 
       // Vertex ExecutionContext setup
       VertexExecutionContext execContext = vertex.getVertexExecutionContext();
+      verifyExecutionContext(execContext, servicePluginsDescriptor, vertex.getName());
       if (execContext != null) {
         DAGProtos.VertexExecutionContextProto contextProto =
             DagTypeConverters.convertToProto(execContext);
@@ -986,4 +991,71 @@ public class DAG {
     
     return dagBuilder.build();
   }
+
+  private void verifyExecutionContext(VertexExecutionContext executionContext,
+                                      ServicePluginsDescriptor servicePluginsDescriptor,
+                                      String context) {
+    if (executionContext != null) {
+      if (executionContext.shouldExecuteInContainers()) {
+        if (servicePluginsDescriptor == null || !servicePluginsDescriptor.areContainersEnabled()) {
+          throw new IllegalStateException("Invalid configuration. ExecutionContext for " + context +
+              " specifies container execution but this is disabled in the ServicePluginDescriptor");
+        }
+      }
+      if (executionContext.shouldExecuteInAm()) {
+        if (servicePluginsDescriptor == null || !servicePluginsDescriptor.isUberEnabled()) {
+          throw new IllegalStateException("Invalid configuration. ExecutionContext for " + context +
+              " specifies AM execution but this is disabled in the ServicePluginDescriptor");
+        }
+      }
+      if (executionContext.getTaskSchedulerName() != null) {
+        boolean found = false;
+        if (servicePluginsDescriptor != null) {
+          found = checkNamedEntityExists(executionContext.getTaskSchedulerName(),
+              servicePluginsDescriptor.getTaskSchedulerDescriptors());
+        }
+        if (!found) {
+          throw new IllegalStateException("Invalid configuration. ExecutionContext for " + context +
+              " specifies task scheduler as " + executionContext.getTaskSchedulerName() +
+              " which is not part of the ServicePluginDescriptor");
+        }
+      }
+      if (executionContext.getContainerLauncherName() != null) {
+        boolean found = false;
+        if (servicePluginsDescriptor != null) {
+          found = checkNamedEntityExists(executionContext.getContainerLauncherName(),
+              servicePluginsDescriptor.getContainerLauncherDescriptors());
+        }
+        if (!found) {
+          throw new IllegalStateException("Invalid configuration. ExecutionContext for " + context +
+              " specifies container launcher as " + executionContext.getContainerLauncherName() +
+              " which is not part of the ServicePluginDescriptor");
+        }
+      }
+      if (executionContext.getTaskCommName() != null) {
+        boolean found = false;
+        if (servicePluginsDescriptor != null) {
+          found = checkNamedEntityExists(executionContext.getTaskCommName(),
+              servicePluginsDescriptor.getTaskCommunicatorDescriptors());
+        }
+        if (!found) {
+          throw new IllegalStateException("Invalid configuration. ExecutionContext for " + context +
+              " specifies task communicator as " + executionContext.getTaskCommName() +
+              " which is not part of the ServicePluginDescriptor");
+        }
+      }
+    }
+  }
+
+  private boolean checkNamedEntityExists(String expected, NamedEntityDescriptor[] namedEntities) {
+    if (namedEntities == null) {
+      return false;
+    }
+    for (NamedEntityDescriptor named : namedEntities) {
+      if (named.getEntityName().equals(expected)) {
+        return true;
+      }
+    }
+    return false;
+  }
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/fee059b9/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 61e4d33..2823a86 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
@@ -801,7 +801,7 @@ public class DagTypeConverters {
     return builder.build();
   }
 
-  public static AMPluginDescriptorProto convertServicePluginDescriptoToProto(
+  public static AMPluginDescriptorProto convertServicePluginDescriptorToProto(
       ServicePluginsDescriptor servicePluginsDescriptor) {
     AMPluginDescriptorProto.Builder pluginDescriptorBuilder =
         AMPluginDescriptorProto.newBuilder();

http://git-wip-us.apache.org/repos/asf/tez/blob/fee059b9/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ContainerLauncherOperationBase.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ContainerLauncherOperationBase.java b/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ContainerLauncherOperationBase.java
index 260b681..98806fa 100644
--- a/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ContainerLauncherOperationBase.java
+++ b/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ContainerLauncherOperationBase.java
@@ -42,8 +42,8 @@ public class ContainerLauncherOperationBase {
   }
 
   /**
-   * Get the node on whcih this container is to be launched
-   * @return
+   * Get the node on which this container is to be launched
+   * @return the node id for the container
    */
   public NodeId getNodeId() {
     return nodeId;
@@ -51,7 +51,7 @@ public class ContainerLauncherOperationBase {
 
   /**
    * Get the containerId for the container
-   * @return
+   * @return the container id for the container opeartion
    */
   public ContainerId getContainerId() {
     return containerId;
@@ -59,7 +59,7 @@ public class ContainerLauncherOperationBase {
 
   /**
    * Get the security token for the container. Primarily for YARN
-   * @return
+   * @return the token for the container launch.
    */
   public Token getContainerToken() {
     return containerToken;

http://git-wip-us.apache.org/repos/asf/tez/blob/fee059b9/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ServicePluginsDescriptor.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ServicePluginsDescriptor.java b/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ServicePluginsDescriptor.java
index ce35350..113b7db 100644
--- a/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ServicePluginsDescriptor.java
+++ b/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ServicePluginsDescriptor.java
@@ -14,6 +14,8 @@
 
 package org.apache.tez.serviceplugins.api;
 
+import java.util.Arrays;
+
 import com.google.common.base.Preconditions;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
@@ -138,4 +140,15 @@ public class ServicePluginsDescriptor {
   public TaskCommunicatorDescriptor[] getTaskCommunicatorDescriptors() {
     return taskCommunicatorDescriptors;
   }
+
+  @Override
+  public String toString() {
+    return "ServicePluginsDescriptor{" +
+        "enableContainers=" + enableContainers +
+        ", enableUber=" + enableUber +
+        ", taskSchedulerDescriptors=" + Arrays.toString(taskSchedulerDescriptors) +
+        ", containerLauncherDescriptors=" + Arrays.toString(containerLauncherDescriptors) +
+        ", taskCommunicatorDescriptors=" + Arrays.toString(taskCommunicatorDescriptors) +
+        '}';
+  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tez/blob/fee059b9/tez-api/src/main/java/org/apache/tez/serviceplugins/api/TaskAttemptEndReason.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/serviceplugins/api/TaskAttemptEndReason.java b/tez-api/src/main/java/org/apache/tez/serviceplugins/api/TaskAttemptEndReason.java
index 4255c28..bff36cd 100644
--- a/tez-api/src/main/java/org/apache/tez/serviceplugins/api/TaskAttemptEndReason.java
+++ b/tez-api/src/main/java/org/apache/tez/serviceplugins/api/TaskAttemptEndReason.java
@@ -22,7 +22,7 @@ import org.apache.hadoop.classification.InterfaceStability;
 public enum TaskAttemptEndReason {
   NODE_FAILED, // Completed because the node running the container was marked as dead
   COMMUNICATION_ERROR, // Communication error with the task
-  SERVICE_BUSY, // External service busy
+  EXECUTOR_BUSY, // External service busy
   INTERNAL_PREEMPTION, // Preempted by the AM, due to an internal decision
   EXTERNAL_PREEMPTION, // Preempted due to cluster contention
   APPLICATION_ERROR, // An error in the AM caused by user code

http://git-wip-us.apache.org/repos/asf/tez/blob/fee059b9/tez-api/src/main/java/org/apache/tez/serviceplugins/api/TaskScheduler.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/serviceplugins/api/TaskScheduler.java b/tez-api/src/main/java/org/apache/tez/serviceplugins/api/TaskScheduler.java
index f05bddc..9a864c5 100644
--- a/tez-api/src/main/java/org/apache/tez/serviceplugins/api/TaskScheduler.java
+++ b/tez-api/src/main/java/org/apache/tez/serviceplugins/api/TaskScheduler.java
@@ -14,6 +14,8 @@
 
 package org.apache.tez.serviceplugins.api;
 
+import javax.annotation.Nullable;
+
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.yarn.api.records.Container;
@@ -178,11 +180,13 @@ public abstract class TaskScheduler implements ServicePluginLifecycle {
    * @param task          the task being de-allocated.
    * @param taskSucceeded whether the task succeeded or not
    * @param endReason     the reason for the task failure
+   * @param diagnostics   additional diagnostics information which may be relevant
    * @return true if the task was associated with a container, false if the task was not associated
    * with a container
    */
   public abstract boolean deallocateTask(Object task, boolean taskSucceeded,
-                                         TaskAttemptEndReason endReason);
+                                         TaskAttemptEndReason endReason,
+                                         @Nullable String diagnostics);
 
   /**
    * A request to de-allocate a previously allocated container.

http://git-wip-us.apache.org/repos/asf/tez/blob/fee059b9/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 3fe17df..268267b 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
@@ -86,7 +86,7 @@ public class TestDAG {
         dummyTaskCount, dummyTaskResource);
 
     DAG dag = DAG.create("testDAG");
-    dag.createVertexGroup("group_1", v1,v2);
+    dag.createVertexGroup("group_1", v1, v2);
 
     try {
       dag.createVertexGroup("group_1", v2, v3);

http://git-wip-us.apache.org/repos/asf/tez/blob/fee059b9/tez-api/src/test/java/org/apache/tez/dag/api/TestDAGPlan.java
----------------------------------------------------------------------
diff --git a/tez-api/src/test/java/org/apache/tez/dag/api/TestDAGPlan.java b/tez-api/src/test/java/org/apache/tez/dag/api/TestDAGPlan.java
index cd42109..7edea2f 100644
--- a/tez-api/src/test/java/org/apache/tez/dag/api/TestDAGPlan.java
+++ b/tez-api/src/test/java/org/apache/tez/dag/api/TestDAGPlan.java
@@ -38,7 +38,6 @@ import org.apache.tez.dag.api.EdgeProperty.DataMovementType;
 import org.apache.tez.dag.api.EdgeProperty.DataSourceType;
 import org.apache.tez.dag.api.EdgeProperty.SchedulingType;
 import org.apache.tez.dag.api.Vertex.VertexExecutionContext;
-import org.apache.tez.dag.api.records.DAGProtos;
 import org.apache.tez.dag.api.records.DAGProtos.DAGPlan;
 import org.apache.tez.dag.api.records.DAGProtos.EdgePlan;
 import org.apache.tez.dag.api.records.DAGProtos.PlanTaskConfiguration;
@@ -46,6 +45,10 @@ import org.apache.tez.dag.api.records.DAGProtos.PlanTaskLocationHint;
 import org.apache.tez.dag.api.records.DAGProtos.PlanVertexType;
 import org.apache.tez.dag.api.records.DAGProtos.VertexExecutionContextProto;
 import org.apache.tez.dag.api.records.DAGProtos.VertexPlan;
+import org.apache.tez.serviceplugins.api.ContainerLauncherDescriptor;
+import org.apache.tez.serviceplugins.api.ServicePluginsDescriptor;
+import org.apache.tez.serviceplugins.api.TaskCommunicatorDescriptor;
+import org.apache.tez.serviceplugins.api.TaskSchedulerDescriptor;
 import org.junit.Assert;
 import org.junit.Rule;
 import org.junit.Test;
@@ -317,6 +320,108 @@ public class TestDAGPlan {
   }
 
   @Test(timeout = 5000)
+  public void testInvalidExecContext_1() {
+    DAG dag = DAG.create("dag1");
+    dag.setExecutionContext(VertexExecutionContext.createExecuteInAm(true));
+    Vertex v1 = Vertex.create("testvertex", ProcessorDescriptor.create("processor1"), 1);
+    dag.addVertex(v1);
+
+    try {
+      dag.createDag(new TezConfiguration(false), null, null, null, true, null, null);
+      fail("Expecting dag create to fail due to invalid ServicePluginDescriptor");
+    } catch (IllegalStateException e) {
+      assertTrue(e.getMessage().contains("AM execution"));
+    }
+
+    dag.setExecutionContext(VertexExecutionContext.createExecuteInContainers(true));
+
+    try {
+      dag.createDag(new TezConfiguration(false), null, null, null, true, null, null);
+      fail("Expecting dag create to fail due to invalid ServicePluginDescriptor");
+    } catch (IllegalStateException e) {
+      assertTrue(e.getMessage().contains("container execution"));
+    }
+
+  }
+
+  @Test(timeout = 5000)
+  public void testInvalidExecContext_2() {
+
+    ServicePluginsDescriptor servicePluginsDescriptor = ServicePluginsDescriptor
+        .create(false,
+            new TaskSchedulerDescriptor[]{TaskSchedulerDescriptor.create("plugin", null)},
+            new ContainerLauncherDescriptor[]{ContainerLauncherDescriptor.create("plugin", null)},
+            new TaskCommunicatorDescriptor[]{TaskCommunicatorDescriptor.create("plugin", null)});
+
+    VertexExecutionContext validExecContext = VertexExecutionContext.create("plugin", "plugin",
+        "plugin");
+    VertexExecutionContext invalidExecContext1 =
+        VertexExecutionContext.create("invalidplugin", "plugin", "plugin");
+    VertexExecutionContext invalidExecContext2 =
+        VertexExecutionContext.create("plugin", "invalidplugin", "plugin");
+    VertexExecutionContext invalidExecContext3 =
+        VertexExecutionContext.create("plugin", "plugin", "invalidplugin");
+
+
+    DAG dag = DAG.create("dag1");
+    dag.setExecutionContext(VertexExecutionContext.createExecuteInContainers(true));
+    Vertex v1 = Vertex.create("testvertex", ProcessorDescriptor.create("processor1"), 1);
+    dag.addVertex(v1);
+
+    // Should succeed. Default context is containers.
+    dag.createDag(new TezConfiguration(false), null, null, null, true, null,
+        servicePluginsDescriptor);
+
+
+    // Set execute in AM should fail
+    v1.setExecutionContext(VertexExecutionContext.createExecuteInAm(true));
+    try {
+      dag.createDag(new TezConfiguration(false), null, null, null, true, null, servicePluginsDescriptor);
+      fail("Expecting dag create to fail due to invalid ServicePluginDescriptor");
+    } catch (IllegalStateException e) {
+      assertTrue(e.getMessage().contains("AM execution"));
+    }
+
+    // Valid context
+    v1.setExecutionContext(validExecContext);
+    dag.createDag(new TezConfiguration(false), null, null, null, true, null, servicePluginsDescriptor);
+
+    // Invalid task scheduler
+    v1.setExecutionContext(invalidExecContext1);
+    try {
+      dag.createDag(new TezConfiguration(false), null, null, null, true, null, servicePluginsDescriptor);
+      fail("Expecting dag create to fail due to invalid ServicePluginDescriptor");
+    } catch (IllegalStateException e) {
+      assertTrue(e.getMessage().contains("testvertex"));
+      assertTrue(e.getMessage().contains("task scheduler"));
+      assertTrue(e.getMessage().contains("invalidplugin"));
+    }
+
+    // Invalid ContainerLauncher
+    v1.setExecutionContext(invalidExecContext2);
+    try {
+      dag.createDag(new TezConfiguration(false), null, null, null, true, null, servicePluginsDescriptor);
+      fail("Expecting dag create to fail due to invalid ServicePluginDescriptor");
+    } catch (IllegalStateException e) {
+      assertTrue(e.getMessage().contains("testvertex"));
+      assertTrue(e.getMessage().contains("container launcher"));
+      assertTrue(e.getMessage().contains("invalidplugin"));
+    }
+
+    // Invalid task comm
+    v1.setExecutionContext(invalidExecContext3);
+    try {
+      dag.createDag(new TezConfiguration(false), null, null, null, true, null, servicePluginsDescriptor);
+      fail("Expecting dag create to fail due to invalid ServicePluginDescriptor");
+    } catch (IllegalStateException e) {
+      assertTrue(e.getMessage().contains("testvertex"));
+      assertTrue(e.getMessage().contains("task communicator"));
+      assertTrue(e.getMessage().contains("invalidplugin"));
+    }
+
+  }
+
+  @Test(timeout = 5000)
   public void testServiceDescriptorPropagation() {
     DAG dag = DAG.create("testDag");
     ProcessorDescriptor pd1 = ProcessorDescriptor.create("processor1").
@@ -328,6 +433,10 @@ public class TestDAGPlan {
         VertexExecutionContext.create("plugin", "plugin", "plugin");
     VertexExecutionContext v1Context = VertexExecutionContext.createExecuteInAm(true);
 
+    ServicePluginsDescriptor servicePluginsDescriptor = ServicePluginsDescriptor
+        .create(true, new TaskSchedulerDescriptor[]{TaskSchedulerDescriptor.create("plugin", null)},
+            new ContainerLauncherDescriptor[]{ContainerLauncherDescriptor.create("plugin", null)},
+            new TaskCommunicatorDescriptor[]{TaskCommunicatorDescriptor.create("plugin", null)});
 
     Vertex v1 = Vertex.create("v1", pd1, 10, Resource.newInstance(1024, 1)).setExecutionContext(v1Context);
     Vertex v2 = Vertex.create("v2", pd2, 1, Resource.newInstance(1024, 1));
@@ -347,7 +456,7 @@ public class TestDAGPlan {
     dag.addVertex(v1).addVertex(v2).addEdge(edge);
     dag.setExecutionContext(defaultExecutionContext);
 
-    DAGPlan dagProto = dag.createDag(new TezConfiguration(), null, null, null, true);
+    DAGPlan dagProto = dag.createDag(new TezConfiguration(), null, null, null, true, null, servicePluginsDescriptor);
 
     assertEquals(2, dagProto.getVertexCount());
     assertEquals(1, dagProto.getEdgeCount());

http://git-wip-us.apache.org/repos/asf/tez/blob/fee059b9/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 e37f849..6f795fc 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
@@ -33,16 +33,13 @@ 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.Vertex.VertexExecutionContext;
-import org.apache.tez.dag.api.records.DAGProtos;
 import org.apache.tez.dag.api.records.DAGProtos.AMPluginDescriptorProto;
 import org.apache.tez.dag.api.records.DAGProtos.TezEntityDescriptorProto;
 import org.apache.tez.dag.api.records.DAGProtos.TezNamedEntityDescriptorProto;
 import org.apache.tez.dag.api.records.DAGProtos.VertexExecutionContextProto;
-import org.apache.tez.serviceplugins.api.ContainerLauncher;
 import org.apache.tez.serviceplugins.api.ContainerLauncherDescriptor;
 import org.apache.tez.serviceplugins.api.ServicePluginsDescriptor;
 import org.apache.tez.serviceplugins.api.TaskCommunicatorDescriptor;
-import org.apache.tez.serviceplugins.api.TaskScheduler;
 import org.apache.tez.serviceplugins.api.TaskSchedulerDescriptor;
 import org.junit.Assert;
 import org.junit.Test;
@@ -152,7 +149,7 @@ public class TestDagTypeConverters {
 
     // Uber-execution
     servicePluginsDescriptor = ServicePluginsDescriptor.create(true);
-    proto = DagTypeConverters.convertServicePluginDescriptoToProto(servicePluginsDescriptor);
+    proto = DagTypeConverters.convertServicePluginDescriptorToProto(servicePluginsDescriptor);
     assertTrue(proto.hasUberEnabled());
     assertTrue(proto.hasContainersEnabled());
     assertTrue(proto.getUberEnabled());
@@ -168,7 +165,7 @@ public class TestDagTypeConverters {
 
     servicePluginsDescriptor = ServicePluginsDescriptor.create(taskSchedulers, containerLaunchers,
         taskComms);
-    proto = DagTypeConverters.convertServicePluginDescriptoToProto(servicePluginsDescriptor);
+    proto = DagTypeConverters.convertServicePluginDescriptorToProto(servicePluginsDescriptor);
     assertTrue(proto.hasUberEnabled());
     assertTrue(proto.hasContainersEnabled());
     assertFalse(proto.getUberEnabled());
@@ -185,7 +182,7 @@ public class TestDagTypeConverters {
 
     servicePluginsDescriptor = ServicePluginsDescriptor.create(taskSchedulers, containerLaunchers,
         taskComms);
-    proto = DagTypeConverters.convertServicePluginDescriptoToProto(servicePluginsDescriptor);
+    proto = DagTypeConverters.convertServicePluginDescriptorToProto(servicePluginsDescriptor);
     assertTrue(proto.hasUberEnabled());
     assertTrue(proto.hasContainersEnabled());
     assertFalse(proto.getUberEnabled());
@@ -201,7 +198,7 @@ public class TestDagTypeConverters {
 
     servicePluginsDescriptor = ServicePluginsDescriptor.create(false, true, taskSchedulers, containerLaunchers,
         taskComms);
-    proto = DagTypeConverters.convertServicePluginDescriptoToProto(servicePluginsDescriptor);
+    proto = DagTypeConverters.convertServicePluginDescriptorToProto(servicePluginsDescriptor);
     assertTrue(proto.hasUberEnabled());
     assertTrue(proto.hasContainersEnabled());
     assertTrue(proto.getUberEnabled());

http://git-wip-us.apache.org/repos/asf/tez/blob/fee059b9/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 1fb7ff9..d6ef901 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
@@ -45,7 +45,6 @@ import org.apache.tez.dag.api.TezConstants;
 import org.apache.tez.dag.api.records.DAGProtos;
 import org.apache.tez.dag.api.records.DAGProtos.ConfigurationProto;
 import org.apache.tez.dag.api.records.DAGProtos.PlanKeyValuePair;
-import org.apache.tez.serviceplugins.api.TaskSchedulerContext;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -271,7 +270,7 @@ public class TezUtilsInternal {
     switch (taskAttemptEndReason) {
       case COMMUNICATION_ERROR:
         return TaskAttemptTerminationCause.COMMUNICATION_ERROR;
-      case SERVICE_BUSY:
+      case EXECUTOR_BUSY:
         return TaskAttemptTerminationCause.SERVICE_BUSY;
       case INTERNAL_PREEMPTION:
         return TaskAttemptTerminationCause.INTERNAL_PREEMPTION;
@@ -301,7 +300,7 @@ public class TezUtilsInternal {
       case COMMUNICATION_ERROR:
         return TaskAttemptEndReason.COMMUNICATION_ERROR;
       case SERVICE_BUSY:
-        return TaskAttemptEndReason.SERVICE_BUSY;
+        return TaskAttemptEndReason.EXECUTOR_BUSY;
       case INTERNAL_PREEMPTION:
         return TaskAttemptEndReason.INTERNAL_PREEMPTION;
       case EXTERNAL_PREEMPTION:

http://git-wip-us.apache.org/repos/asf/tez/blob/fee059b9/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 4fc541c..f1f683b 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 javax.annotation.Nullable;
 import java.net.InetSocketAddress;
 import java.util.Map;
 
@@ -115,8 +116,10 @@ public abstract class TaskCommunicator implements ServicePluginLifecycle {
    *
    * @param containerId the associated containerId
    * @param endReason   the end reason for the container completing
+   * @param diagnostics diagnostics associated with the container end
    */
-  public abstract void registerContainerEnd(ContainerId containerId, ContainerEndReason endReason);
+  public abstract void registerContainerEnd(ContainerId containerId, ContainerEndReason endReason,
+                                            @Nullable String diagnostics);
 
   /**
    * Register a task attempt to execute on a container
@@ -138,14 +141,15 @@ public abstract class TaskCommunicator implements ServicePluginLifecycle {
 
   /**
    * Register the completion of a task. This may be a result of preemption, the container dying,
-   * the
-   * node dying, the task completing to success
+   * the node dying, the task completing to success
    *
    * @param taskAttemptID the task attempt which has completed / needs to be completed
    * @param endReason     the endReason for the task attempt.
+   * @param diagnostics   diagnostics associated with the task end
    */
   public abstract void unregisterRunningTaskAttempt(TezTaskAttemptID taskAttemptID,
-                                                    TaskAttemptEndReason endReason);
+                                                    TaskAttemptEndReason endReason,
+                                                    @Nullable String diagnostics);
 
   /**
    * Return the address, if any, that the service listens on

http://git-wip-us.apache.org/repos/asf/tez/blob/fee059b9/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 0a684e7..e81ba2b 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
@@ -160,7 +160,7 @@ public interface TaskCommunicatorContext {
    *
    * @return the name of the currently executing dag
    */
-  String getCurretnDagName();
+  String getCurrentDagName();
 
   /**
    * Get the name of the Input vertices for the specified vertex.

http://git-wip-us.apache.org/repos/asf/tez/blob/fee059b9/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 2eec2fb..761bdb0 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
@@ -34,9 +34,9 @@ public interface TaskAttemptListener {
 
   void registerTaskAttempt(AMContainerTask amContainerTask, ContainerId containerId, int taskCommId);
   
-  void unregisterRunningContainer(ContainerId containerId, int taskCommId, ContainerEndReason endReason);
+  void unregisterRunningContainer(ContainerId containerId, int taskCommId, ContainerEndReason endReason, String diagnostics);
   
-  void unregisterTaskAttempt(TezTaskAttemptID attemptID, int taskCommId, TaskAttemptEndReason endReason);
+  void unregisterTaskAttempt(TezTaskAttemptID attemptID, int taskCommId, TaskAttemptEndReason endReason, String diagnostics);
 
   void dagComplete(DAG dag);
 

http://git-wip-us.apache.org/repos/asf/tez/blob/fee059b9/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 ad6f2c4..2f6e93c 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
@@ -176,7 +176,6 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
     try {
       Constructor<? extends TaskCommunicator> ctor =
           taskCommClazz.getConstructor(TaskCommunicatorContext.class);
-      ctor.setAccessible(true);
       return ctor.newInstance(taskCommunicatorContext);
     } catch (NoSuchMethodException | InvocationTargetException | InstantiationException | IllegalAccessException e) {
       throw new TezUncheckedException(e);
@@ -366,7 +365,7 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
   }
 
   @Override
-  public void unregisterRunningContainer(ContainerId containerId, int taskCommId, ContainerEndReason endReason) {
+  public void unregisterRunningContainer(ContainerId containerId, int taskCommId, ContainerEndReason endReason, String diagnostics) {
     if (LOG.isDebugEnabled()) {
       LOG.debug("Unregistering Container from TaskAttemptListener: " + containerId);
     }
@@ -374,7 +373,7 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
     if (containerInfo.taskAttemptId != null) {
       registeredAttempts.remove(containerInfo.taskAttemptId);
     }
-    taskCommunicators[taskCommId].registerContainerEnd(containerId, endReason);
+    taskCommunicators[taskCommId].registerContainerEnd(containerId, endReason, diagnostics);
   }
 
   @Override
@@ -408,7 +407,7 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
   }
 
   @Override
-  public void unregisterTaskAttempt(TezTaskAttemptID attemptId, int taskCommId, TaskAttemptEndReason endReason) {
+  public void unregisterTaskAttempt(TezTaskAttemptID attemptId, int taskCommId, TaskAttemptEndReason endReason, String diagnostics) {
     ContainerId containerId = registeredAttempts.remove(attemptId);
     if (containerId == null) {
       LOG.warn("Unregister task attempt: " + attemptId + " from unknown container");
@@ -422,7 +421,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);
-    taskCommunicators[taskCommId].unregisterRunningTaskAttempt(attemptId, endReason);
+    taskCommunicators[taskCommId].unregisterRunningTaskAttempt(attemptId, endReason, diagnostics);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/tez/blob/fee059b9/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 0f10305..c56311c 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
@@ -27,6 +27,7 @@ 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.UserPayload;
+import org.apache.tez.dag.app.rm.container.AMContainer;
 import org.apache.tez.serviceplugins.api.TaskAttemptEndReason;
 import org.apache.tez.dag.api.TaskCommunicatorContext;
 import org.apache.tez.dag.api.TaskHeartbeatRequest;
@@ -96,7 +97,13 @@ public class TaskCommunicatorContextImpl implements TaskCommunicatorContext, Ver
 
   @Override
   public boolean isKnownContainer(ContainerId containerId) {
-    return context.getAllContainers().get(containerId) != null;
+    AMContainer amContainer = context.getAllContainers().get(containerId);
+    if (amContainer == null ||
+        amContainer.getTaskCommunicatorIdentifier() != taskCommunicatorIndex) {
+      return false;
+    } else {
+      return true;
+    }
   }
 
   @Override
@@ -106,7 +113,9 @@ public class TaskCommunicatorContextImpl implements TaskCommunicatorContext, Ver
 
   @Override
   public void containerAlive(ContainerId containerId) {
-    taskAttemptListener.containerAlive(containerId);
+    if (isKnownContainer(containerId)) {
+      taskAttemptListener.containerAlive(containerId);
+    }
   }
 
   @Override
@@ -136,7 +145,7 @@ public class TaskCommunicatorContextImpl implements TaskCommunicatorContext, Ver
   }
 
   @Override
-  public String getCurretnDagName() {
+  public String getCurrentDagName() {
     return getDag().getName();
   }
 

http://git-wip-us.apache.org/repos/asf/tez/blob/fee059b9/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 d3f1c44..9ecee5b 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
@@ -36,8 +36,12 @@ 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.ContainerTask;
+import org.apache.tez.common.TezConverterUtils;
+import org.apache.tez.common.TezLocalResource;
+import org.apache.tez.common.TezTaskUmbilicalProtocol;
+import org.apache.tez.common.TezUtils;
 import org.apache.tez.common.security.JobTokenIdentifier;
 import org.apache.tez.common.security.JobTokenSecretManager;
 import org.apache.tez.common.security.TokenCache;
@@ -199,7 +203,7 @@ public class TezTaskCommunicatorImpl extends TaskCommunicator {
   }
 
   @Override
-  public void registerContainerEnd(ContainerId containerId, ContainerEndReason endReason) {
+  public void registerContainerEnd(ContainerId containerId, ContainerEndReason endReason, String diagnostics) {
     ContainerInfo containerInfo = registeredContainers.remove(containerId);
     if (containerInfo != null) {
       synchronized(containerInfo) {
@@ -245,7 +249,7 @@ public class TezTaskCommunicatorImpl extends TaskCommunicator {
 
 
   @Override
-  public void unregisterRunningTaskAttempt(TezTaskAttemptID taskAttemptID, TaskAttemptEndReason endReason) {
+  public void unregisterRunningTaskAttempt(TezTaskAttemptID taskAttemptID, TaskAttemptEndReason endReason, String diagnostics) {
     TaskAttempt taskAttempt = new TaskAttempt(taskAttemptID);
     ContainerId containerId = attemptToContainerMap.remove(taskAttempt);
     if(containerId == null) {

http://git-wip-us.apache.org/repos/asf/tez/blob/fee059b9/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 17f5675..6b474ff 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
@@ -82,7 +82,6 @@ import org.apache.tez.dag.api.records.DAGProtos.PlanKeyValuePair;
 import org.apache.tez.dag.api.records.DAGProtos.PlanVertexGroupInfo;
 import org.apache.tez.dag.api.records.DAGProtos.VertexPlan;
 import org.apache.tez.dag.app.AppContext;
-import org.apache.tez.dag.app.DAGAppMasterState;
 import org.apache.tez.dag.app.TaskAttemptListener;
 import org.apache.tez.dag.app.TaskHeartbeatHandler;
 import org.apache.tez.dag.app.dag.DAG;
@@ -180,6 +179,7 @@ public class DAGImpl implements org.apache.tez.dag.app.dag.DAG,
   private final AppContext appContext;
   private final UserGroupInformation dagUGI;
   private final ACLManager aclManager;
+  private final org.apache.tez.dag.api.Vertex.VertexExecutionContext defaultExecutionContext;
   @VisibleForTesting
   StateChangeNotifier entityUpdateTracker;
 
@@ -538,6 +538,11 @@ public class DAGImpl implements org.apache.tez.dag.app.dag.DAG,
     // this is only for recovery in case it does not call the init transition
     this.startDAGCpuTime = appContext.getCumulativeCPUTime();
     this.startDAGGCTime = appContext.getCumulativeGCTime();
+    if (jobPlan.hasDefaultExecutionContext()) {
+      defaultExecutionContext = DagTypeConverters.convertFromProto(jobPlan.getDefaultExecutionContext());
+    } else {
+      defaultExecutionContext = null;
+    }
     
     this.taskSpecificLaunchCmdOption = new TaskSpecificLaunchCmdOption(dagConf);
     // This "this leak" is okay because the retained pointer is in an
@@ -718,11 +723,7 @@ public class DAGImpl implements org.apache.tez.dag.app.dag.DAG,
 
   @Override
   public org.apache.tez.dag.api.Vertex.VertexExecutionContext getDefaultExecutionContext() {
-    if (jobPlan.hasDefaultExecutionContext()) {
-      return DagTypeConverters.convertFromProto(jobPlan.getDefaultExecutionContext());
-    } else {
-      return null;
-    }
+    return defaultExecutionContext;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/tez/blob/fee059b9/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 65ea3fb..c6d8a7e 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
@@ -227,7 +227,7 @@ public class TaskAttemptImpl implements TaskAttempt,
           TaskAttemptEventType.TA_KILL_REQUEST,
           new TerminatedBeforeRunningTransition(KILLED_HELPER))
       .addTransition(TaskAttemptStateInternal.START_WAIT,
-          TaskAttemptStateInternal.KILL_IN_PROGRESS,
+          TaskAttemptStateInternal.KILLED,
           TaskAttemptEventType.TA_KILLED,
           new TerminatedBeforeRunningTransition(KILLED_HELPER))
       .addTransition(TaskAttemptStateInternal.START_WAIT,
@@ -267,7 +267,7 @@ public class TaskAttemptImpl implements TaskAttempt,
           TaskAttemptEventType.TA_KILL_REQUEST,
           new TerminatedWhileRunningTransition(KILLED_HELPER))
       .addTransition(TaskAttemptStateInternal.RUNNING,
-          TaskAttemptStateInternal.KILL_IN_PROGRESS,
+          TaskAttemptStateInternal.KILLED,
           TaskAttemptEventType.TA_KILLED,
           new TerminatedWhileRunningTransition(KILLED_HELPER))
       .addTransition(TaskAttemptStateInternal.RUNNING,
@@ -1095,7 +1095,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.
+      // TODO Post TEZ-2003. Allow for a policy in the VMPlugin to define locality for different attempts.
       TaskLocationHint locationHint = ta.getTaskLocationHint();
       if (locationHint != null) {
         if (locationHint.getRacks() != null) {
@@ -1266,6 +1266,7 @@ public class TaskAttemptImpl implements TaskAttempt,
       if (sendSchedulerEvent()) {
         ta.sendEvent(new AMSchedulerEventTAEnded(ta, ta.containerId, helper
             .getTaskAttemptState(), TezUtilsInternal.toTaskAttemptEndReason(ta.terminationCause),
+            ta instanceof DiagnosableEvent ? ((DiagnosableEvent)ta).getDiagnosticInfo() : null,
             ta.getVertex().getTaskSchedulerIdentifier()));
       }
     }
@@ -1348,7 +1349,7 @@ public class TaskAttemptImpl implements TaskAttempt,
 
       // Inform the Scheduler.
       ta.sendEvent(new AMSchedulerEventTAEnded(ta, ta.containerId,
-          TaskAttemptState.SUCCEEDED, null, ta.getVertex().getTaskSchedulerIdentifier()));
+          TaskAttemptState.SUCCEEDED, null, null, ta.getVertex().getTaskSchedulerIdentifier()));
 
       // Inform the task.
       ta.sendEvent(new TaskEventTAUpdate(ta.attemptId,

http://git-wip-us.apache.org/repos/asf/tez/blob/fee059b9/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 d0cee21..b56bd5b 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
@@ -70,7 +70,7 @@ public class ContainerLauncherRouter extends AbstractService
                                  TaskAttemptListener taskAttemptListener,
                                  String workingDirectory,
                                  List<NamedEntityDescriptor> containerLauncherDescriptors,
-                                 boolean isPureLocalMode) throws UnknownHostException {
+                                 boolean isPureLocalMode) {
     super(ContainerLauncherRouter.class.getName());
 
     this.appContext = context;
@@ -101,8 +101,7 @@ public class ContainerLauncherRouter extends AbstractService
       TaskAttemptListener taskAttemptListener,
       String workingDirectory,
       int containerLauncherIndex,
-      boolean isPureLocalMode) throws
-      UnknownHostException {
+      boolean isPureLocalMode) {
     if (containerLauncherDescriptor.getEntityName().equals(
         TezConstants.getTezYarnServicePluginName())) {
       return createYarnContainerLauncher(containerLauncherContext);
@@ -126,15 +125,18 @@ public class ContainerLauncherRouter extends AbstractService
                                                 AppContext context,
                                                 TaskAttemptListener taskAttemptListener,
                                                 String workingDirectory,
-                                                boolean isPureLocalMode) throws
-      UnknownHostException {
+                                                boolean isPureLocalMode) {
     LOG.info("Creating LocalContainerLauncher");
     // TODO Post TEZ-2003. LocalContainerLauncher is special cased, since it makes use of
     // extensive internals which are only available at runtime. Will likely require
     // some kind of runtime binding of parameters in the payload to work correctly.
-    return
-        new LocalContainerLauncher(containerLauncherContext, context, taskAttemptListener,
-            workingDirectory, isPureLocalMode);
+    try {
+      return
+          new LocalContainerLauncher(containerLauncherContext, context, taskAttemptListener,
+              workingDirectory, isPureLocalMode);
+    } catch (UnknownHostException e) {
+      throw new TezUncheckedException(e);
+    }
   }
 
   @VisibleForTesting
@@ -149,7 +151,6 @@ public class ContainerLauncherRouter extends AbstractService
     try {
       Constructor<? extends ContainerLauncher> ctor = containerLauncherClazz
           .getConstructor(ContainerLauncherContext.class);
-      ctor.setAccessible(true);
       return ctor.newInstance(containerLauncherContext);
     } catch (NoSuchMethodException | InvocationTargetException | InstantiationException | IllegalAccessException e) {
       throw new TezUncheckedException(e);

http://git-wip-us.apache.org/repos/asf/tez/blob/fee059b9/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 33763e7..ccc5465 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
@@ -29,15 +29,17 @@ public class AMSchedulerEventTAEnded extends AMSchedulerEvent {
   private final ContainerId containerId;
   private final TaskAttemptState state;
   private final TaskAttemptEndReason taskAttemptEndReason;
+  private final String diagnostics;
   private final int schedulerId;
 
   public AMSchedulerEventTAEnded(TaskAttempt attempt, ContainerId containerId,
-      TaskAttemptState state, TaskAttemptEndReason taskAttemptEndReason, int schedulerId) {
+      TaskAttemptState state, TaskAttemptEndReason taskAttemptEndReason, String diagnostics, int schedulerId) {
     super(AMSchedulerEventType.S_TA_ENDED);
     this.attempt = attempt;
     this.containerId = containerId;
     this.state = state;
     this.taskAttemptEndReason = taskAttemptEndReason;
+    this.diagnostics = diagnostics;
     this.schedulerId = schedulerId;
   }
 
@@ -64,4 +66,8 @@ public class AMSchedulerEventTAEnded extends AMSchedulerEvent {
   public TaskAttemptEndReason getTaskAttemptEndReason() {
     return taskAttemptEndReason;
   }
+
+  public String getDiagnostics() {
+    return diagnostics;
+  }
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/fee059b9/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 befde94..f77a9a9 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
@@ -131,7 +131,7 @@ public class LocalTaskSchedulerService extends TaskScheduler {
   }
   
   @Override
-  public boolean deallocateTask(Object task, boolean taskSucceeded, TaskAttemptEndReason endReason) {
+  public boolean deallocateTask(Object task, boolean taskSucceeded, TaskAttemptEndReason endReason, String diagnostics) {
     return taskRequestHandler.addDeallocateTaskRequest(task);
   }
 

http://git-wip-us.apache.org/repos/asf/tez/blob/fee059b9/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 f001909..7d2e768 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
@@ -286,7 +286,7 @@ public class TaskSchedulerEventHandler extends AbstractService implements
     TaskAttempt attempt = event.getAttempt();
     // 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());
+        .deallocateTask(attempt, false, event.getTaskAttemptEndReason(), event.getDiagnostics());
     // use stored value of container id in case the scheduler has removed this
     // assignment because the task has been deallocated earlier.
     // retroactive case
@@ -331,7 +331,7 @@ public class TaskSchedulerEventHandler extends AbstractService implements
     }
 
     boolean wasContainerAllocated = taskSchedulers[event.getSchedulerId()].deallocateTask(attempt,
-        true, null);
+        true, null, event.getDiagnostics());
     if (!wasContainerAllocated) {
       LOG.error("De-allocated successful task: " + attempt.getID()
           + ", but TaskScheduler reported no container assigned to task");
@@ -436,7 +436,6 @@ public class TaskSchedulerEventHandler extends AbstractService implements
     try {
       Constructor<? extends TaskScheduler> ctor = taskSchedulerClazz
           .getConstructor(TaskSchedulerContext.class);
-      ctor.setAccessible(true);
       return ctor.newInstance(taskSchedulerContext);
     } catch (NoSuchMethodException | InvocationTargetException | InstantiationException | IllegalAccessException e) {
       throw new TezUncheckedException(e);

http://git-wip-us.apache.org/repos/asf/tez/blob/fee059b9/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 940c5b0..64d0fd2 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
@@ -980,7 +980,8 @@ public class YarnTaskSchedulerService extends TaskScheduler
    */
   @Override
   public boolean deallocateTask(Object task, boolean taskSucceeded,
-                                TaskAttemptEndReason endReason) {
+                                TaskAttemptEndReason endReason,
+                                String diagnostics) {
     Map<CookieContainerRequest, Container> assignedContainers = null;
 
     synchronized (this) {
@@ -1170,7 +1171,7 @@ public class YarnTaskSchedulerService extends TaskScheduler
             CookieContainerRequest request = entry.getValue();
             if (request.getPriority().equals(lowestPriNewContainer.getPriority())) {
               LOG.info("Resending request for task again: " + task);
-              deallocateTask(task, true, null);
+              deallocateTask(task, true, null, 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/fee059b9/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 aeacf84..99cec2b 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
@@ -631,14 +631,14 @@ public class AMContainerImpl implements AMContainer {
       SingleArcTransition<AMContainerImpl, AMContainerEvent> {
     @Override
     public void transition(AMContainerImpl container, AMContainerEvent cEvent) {
+      AMContainerEventLaunchFailed event = (AMContainerEventLaunchFailed) cEvent;
       if (container.currentAttempt != null) {
-        AMContainerEventLaunchFailed event = (AMContainerEventLaunchFailed) cEvent;
         // for a properly setup cluster this should almost always be an app error
         // need to differentiate between launch failed due to framework/cluster or app
         container.sendTerminatingToTaskAttempt(container.currentAttempt,
             event.getMessage(), TaskAttemptTerminationCause.CONTAINER_LAUNCH_FAILED);
       }
-      container.unregisterFromTAListener(ContainerEndReason.LAUNCH_FAILED);
+      container.unregisterFromTAListener(ContainerEndReason.LAUNCH_FAILED, event.getMessage());
       container.deAllocate();
     }
   }
@@ -668,7 +668,7 @@ public class AMContainerImpl implements AMContainer {
       }
       container.containerLocalResources = null;
       container.additionalLocalResources = null;
-      container.unregisterFromTAListener(event.getContainerEndReason());
+      container.unregisterFromTAListener(event.getContainerEndReason(), event.getDiagnostics());
       String diag = event.getDiagnostics();
       if (!(diag == null || diag.equals(""))) {
         LOG.info("Container " + container.getContainerId()
@@ -694,7 +694,7 @@ public class AMContainerImpl implements AMContainer {
         container.sendTerminatingToTaskAttempt(container.currentAttempt,
             getMessage(container, cEvent), TaskAttemptTerminationCause.CONTAINER_STOPPED);
       }
-      container.unregisterFromTAListener(ContainerEndReason.OTHER);
+      container.unregisterFromTAListener(ContainerEndReason.OTHER, getMessage(container, cEvent));
       container.logStopped(container.currentAttempt == null ?
           ContainerExitStatus.SUCCESS 
           : ContainerExitStatus.INVALID);
@@ -746,7 +746,11 @@ public class AMContainerImpl implements AMContainer {
     @Override
     public void transition(AMContainerImpl container, AMContainerEvent cEvent) {
       super.transition(container, cEvent);
-      container.unregisterFromTAListener(ContainerEndReason.NODE_FAILED);
+      String errorMessage = "Node " + container.getContainer().getNodeId() + " failed. ";
+      if (cEvent instanceof DiagnosableEvent) {
+        errorMessage += ((DiagnosableEvent) cEvent).getDiagnosticInfo();
+      }
+      container.unregisterFromTAListener(ContainerEndReason.NODE_FAILED, errorMessage);
       container.deAllocate();
     }
   }
@@ -756,14 +760,15 @@ public class AMContainerImpl implements AMContainer {
     @Override
     public void transition(AMContainerImpl container, AMContainerEvent cEvent) {
       super.transition(container, cEvent);
+      String errorMessage = "Container " + container.getContainerId() +
+          " hit an invalid transition - " + cEvent.getType() + " at " +
+          container.getState();
       if (container.currentAttempt != null) {
         container.sendTerminatingToTaskAttempt(container.currentAttempt,
-            "Container " + container.getContainerId() +
-                " hit an invalid transition - " + cEvent.getType() + " at " +
-                container.getState(), TaskAttemptTerminationCause.FRAMEWORK_ERROR);
+            errorMessage, TaskAttemptTerminationCause.FRAMEWORK_ERROR);
       }
       container.logStopped(ContainerExitStatus.ABORTED);
-      container.unregisterFromTAListener(ContainerEndReason.FRAMEWORK_ERROR);
+      container.unregisterFromTAListener(ContainerEndReason.FRAMEWORK_ERROR, errorMessage);
       container.sendStopRequestToNM();
     }
   }
@@ -835,7 +840,12 @@ public class AMContainerImpl implements AMContainer {
     public void transition(AMContainerImpl container, AMContainerEvent cEvent) {
 
       AMContainerEventAssignTA event = (AMContainerEventAssignTA) cEvent;
-      container.unregisterAttemptFromListener(container.currentAttempt, TaskAttemptEndReason.FRAMEWORK_ERROR);
+      String errorMessage = "AMScheduler Error: Multiple simultaneous " +
+          "taskAttempt allocations to: " + container.getContainerId() +
+          ". Attempts: " + container.getCurrentTaskAttempt() + ", " + event.getTaskAttemptId() +
+          ". Current state: " + container.getState();
+      container.unregisterAttemptFromListener(container.currentAttempt,
+          TaskAttemptEndReason.FRAMEWORK_ERROR, errorMessage);
       container.handleExtraTAAssign(event, container.currentAttempt);
     }
   }
@@ -846,7 +856,7 @@ public class AMContainerImpl implements AMContainer {
     public void transition(AMContainerImpl container, AMContainerEvent cEvent) {
       container.lastTaskFinishTime = System.currentTimeMillis();
       container.completedAttempts.add(container.currentAttempt);
-      container.unregisterAttemptFromListener(container.currentAttempt, TaskAttemptEndReason.OTHER);
+      container.unregisterAttemptFromListener(container.currentAttempt, TaskAttemptEndReason.OTHER, null);
       container.currentAttempt = null;
     }
   }
@@ -863,7 +873,9 @@ public class AMContainerImpl implements AMContainer {
         container.sendTerminatedToTaskAttempt(container.currentAttempt,
             getMessage(container, event), event.getTerminationCause());
       }
-      container.unregisterAttemptFromListener(container.currentAttempt, TezUtilsInternal.toTaskAttemptEndReason(event.getTerminationCause()));
+      container.unregisterAttemptFromListener(container.currentAttempt,
+          TezUtilsInternal.toTaskAttemptEndReason(event.getTerminationCause()),
+          getMessage(container, event));
       container.registerFailedAttempt(container.currentAttempt);
       container.currentAttempt= null;
       super.transition(container, cEvent);
@@ -873,7 +885,8 @@ public class AMContainerImpl implements AMContainer {
   protected static class StopRequestAtRunningTransition
       extends StopRequestAtIdleTransition {
     public void transition(AMContainerImpl container, AMContainerEvent cEvent) {
-      container.unregisterAttemptFromListener(container.currentAttempt, TaskAttemptEndReason.OTHER);
+      container.unregisterAttemptFromListener(container.currentAttempt, TaskAttemptEndReason.OTHER,
+          getMessage(container, cEvent));
       super.transition(container, cEvent);
     }
   }
@@ -894,7 +907,8 @@ public class AMContainerImpl implements AMContainer {
     @Override
     public void transition(AMContainerImpl container, AMContainerEvent cEvent) {
       super.transition(container, cEvent);
-      container.unregisterAttemptFromListener(container.currentAttempt, TaskAttemptEndReason.NODE_FAILED);
+      String errorMessage = "Node " + container.getContainer().getNodeId() + " failed. ";
+      container.unregisterAttemptFromListener(container.currentAttempt, TaskAttemptEndReason.NODE_FAILED, errorMessage);
     }
   }
 
@@ -903,11 +917,13 @@ public class AMContainerImpl implements AMContainer {
     @Override
     public void transition(AMContainerImpl container, AMContainerEvent cEvent) {
       super.transition(container, cEvent);
-      container.unregisterAttemptFromListener(container.currentAttempt, TaskAttemptEndReason.FRAMEWORK_ERROR);
+      String errorMessage = "Container " + container.getContainerId() +
+          " hit an invalid transition - " + cEvent.getType() + " at " +
+          container.getState();
+      container.unregisterAttemptFromListener(container.currentAttempt,
+          TaskAttemptEndReason.FRAMEWORK_ERROR, errorMessage);
       container.sendTerminatingToTaskAttempt(container.currentAttempt,
-          "Container " + container.getContainerId() +
-              " hit an invalid transition - " + cEvent.getType() + " at " +
-              container.getState(), TaskAttemptTerminationCause.FRAMEWORK_ERROR);
+          errorMessage, TaskAttemptTerminationCause.FRAMEWORK_ERROR);
     }
   }
 
@@ -1029,7 +1045,7 @@ public class AMContainerImpl implements AMContainer {
     LOG.warn(errorMessage);
     this.logStopped(ContainerExitStatus.INVALID);
     this.sendStopRequestToNM();
-    this.unregisterFromTAListener(ContainerEndReason.FRAMEWORK_ERROR);
+    this.unregisterFromTAListener(ContainerEndReason.FRAMEWORK_ERROR, errorMessage);
     this.unregisterFromContainerListener();
   }
 
@@ -1087,8 +1103,8 @@ public class AMContainerImpl implements AMContainer {
         container.getNodeId(), container.getContainerToken(), launcherId, schedulerId, taskCommId));
   }
 
-  protected void unregisterAttemptFromListener(TezTaskAttemptID attemptId, TaskAttemptEndReason endReason) {
-    taskAttemptListener.unregisterTaskAttempt(attemptId, taskCommId, endReason);
+  protected void unregisterAttemptFromListener(TezTaskAttemptID attemptId, TaskAttemptEndReason endReason, String diagnostics) {
+    taskAttemptListener.unregisterTaskAttempt(attemptId, taskCommId, endReason, diagnostics);
   }
 
   protected void registerAttemptWithListener(AMContainerTask amContainerTask) {
@@ -1099,8 +1115,8 @@ public class AMContainerImpl implements AMContainer {
     taskAttemptListener.registerRunningContainer(containerId, taskCommId);
   }
 
-  protected void unregisterFromTAListener(ContainerEndReason endReason) {
-    this.taskAttemptListener.unregisterRunningContainer(containerId, taskCommId, endReason);
+  protected void unregisterFromTAListener(ContainerEndReason endReason, String diagnostics) {
+    this.taskAttemptListener.unregisterRunningContainer(containerId, taskCommId, endReason, diagnostics);
   }
 
   protected void registerWithContainerListener() {

http://git-wip-us.apache.org/repos/asf/tez/blob/fee059b9/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeTracker.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeTracker.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeTracker.java
index 0668ff2..32e515b 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeTracker.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeTracker.java
@@ -53,7 +53,7 @@ public class AMNodeTracker extends AbstractService implements
 
   @SuppressWarnings("rawtypes")
   public AMNodeTracker(EventHandler eventHandler, AppContext appContext) {
-    super("AMNodeMap");
+    super("AMNodeTracker");
     this.perSourceNodeTrackers = new ConcurrentHashMap<>();
     this.eventHandler = eventHandler;
     this.appContext = appContext;

http://git-wip-us.apache.org/repos/asf/tez/blob/fee059b9/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 4d404b9..5159aff 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
@@ -184,12 +184,12 @@ public class TestTaskAttemptListenerImplTezDag {
     assertEquals(taskSpec, containerTask.getTaskSpec());
 
     // Task unregistered. Should respond to heartbeats
-    taskAttemptListener.unregisterTaskAttempt(taskAttemptID, 0, TaskAttemptEndReason.OTHER);
+    taskAttemptListener.unregisterTaskAttempt(taskAttemptID, 0, TaskAttemptEndReason.OTHER, null);
     containerTask = tezUmbilical.getTask(containerContext2);
     assertNull(containerTask);
 
     // Container unregistered. Should send a shouldDie = true
-    taskAttemptListener.unregisterRunningContainer(containerId2, 0, ContainerEndReason.OTHER);
+    taskAttemptListener.unregisterRunningContainer(containerId2, 0, ContainerEndReason.OTHER, null);
     containerTask = tezUmbilical.getTask(containerContext2);
     assertTrue(containerTask.shouldDie());
 
@@ -203,7 +203,7 @@ public class TestTaskAttemptListenerImplTezDag {
     doReturn(taskAttemptId2).when(taskSpec2).getTaskAttemptID();
     AMContainerTask amContainerTask2 = new AMContainerTask(taskSpec, null, null, false, 0);
     taskAttemptListener.registerTaskAttempt(amContainerTask2, containerId3, 0);
-    taskAttemptListener.unregisterRunningContainer(containerId3, 0, ContainerEndReason.OTHER);
+    taskAttemptListener.unregisterRunningContainer(containerId3, 0, ContainerEndReason.OTHER, null);
     containerTask = tezUmbilical.getTask(containerContext3);
     assertTrue(containerTask.shouldDie());
   }

http://git-wip-us.apache.org/repos/asf/tez/blob/fee059b9/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
index abb5e42..74468f2 100644
--- 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
@@ -41,7 +41,6 @@ import org.apache.tez.common.TezUtils;
 import org.apache.tez.dag.api.NamedEntityDescriptor;
 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.UserPayload;
 import org.apache.tez.serviceplugins.api.TaskAttemptEndReason;
 import org.apache.tez.dag.app.dag.DAG;
@@ -110,7 +109,7 @@ public class TestTaskAttemptListenerImplTezDag2 {
     taskAttemptListener
         .taskFailed(taskAttemptId1, TaskAttemptEndReason.COMMUNICATION_ERROR, "Diagnostics1");
     taskAttemptListener
-        .taskKilled(taskAttemptId2, TaskAttemptEndReason.SERVICE_BUSY, "Diagnostics2");
+        .taskKilled(taskAttemptId2, TaskAttemptEndReason.EXECUTOR_BUSY, "Diagnostics2");
 
     ArgumentCaptor<Event> argumentCaptor = ArgumentCaptor.forClass(Event.class);
     verify(eventHandler, times(2)).handle(argumentCaptor.capture());

http://git-wip-us.apache.org/repos/asf/tez/blob/fee059b9/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskCommunicatorContextImpl.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskCommunicatorContextImpl.java b/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskCommunicatorContextImpl.java
new file mode 100644
index 0000000..1545eb4
--- /dev/null
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskCommunicatorContextImpl.java
@@ -0,0 +1,85 @@
+/*
+ * 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.assertFalse;
+import static org.junit.Assert.assertTrue;
+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.verify;
+
+import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.tez.common.ContainerSignatureMatcher;
+import org.apache.tez.dag.api.TaskCommunicatorContext;
+import org.apache.tez.dag.app.rm.container.AMContainerMap;
+import org.junit.Test;
+
+public class TestTaskCommunicatorContextImpl {
+
+  @Test(timeout = 5000)
+  public void testIsKnownContainer() {
+    AppContext appContext = mock(AppContext.class);
+    TaskAttemptListenerImpTezDag tal = mock(TaskAttemptListenerImpTezDag.class);
+
+    AMContainerMap amContainerMap = new AMContainerMap(mock(ContainerHeartbeatHandler.class), tal, mock(
+        ContainerSignatureMatcher.class), appContext);
+
+    doReturn(amContainerMap).when(appContext).getAllContainers();
+
+    ContainerId containerId01 = mock(ContainerId.class);
+    Container container01 = mock(Container.class);
+    doReturn(containerId01).when(container01).getId();
+
+    ContainerId containerId11 = mock(ContainerId.class);
+    Container container11 = mock(Container.class);
+    doReturn(containerId11).when(container11).getId();
+
+    amContainerMap.addContainerIfNew(container01, 0, 0, 0);
+    amContainerMap.addContainerIfNew(container11, 1, 1, 1);
+
+    TaskCommunicatorContext taskCommContext0 = new TaskCommunicatorContextImpl(appContext, tal, null, 0);
+    TaskCommunicatorContext taskCommContext1 = new TaskCommunicatorContextImpl(appContext, tal, null, 1);
+
+    assertTrue(taskCommContext0.isKnownContainer(containerId01));
+    assertFalse(taskCommContext0.isKnownContainer(containerId11));
+
+    assertFalse(taskCommContext1.isKnownContainer(containerId01));
+    assertTrue(taskCommContext1.isKnownContainer(containerId11));
+
+    taskCommContext0.containerAlive(containerId01);
+    verify(tal).containerAlive(containerId01);
+    reset(tal);
+
+    taskCommContext0.containerAlive(containerId11);
+    verify(tal, never()).containerAlive(containerId11);
+    reset(tal);
+
+    taskCommContext1.containerAlive(containerId01);
+    verify(tal, never()).containerAlive(containerId01);
+    reset(tal);
+
+    taskCommContext1.containerAlive(containerId11);
+    verify(tal).containerAlive(containerId11);
+    reset(tal);
+
+    taskCommContext1.containerAlive(containerId01);
+    verify(tal, never()).containerAlive(containerId01);
+    reset(tal);
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/tez/blob/fee059b9/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskCommunicatorManager.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskCommunicatorManager.java b/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskCommunicatorManager.java
index c76aa50..4f68fab 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskCommunicatorManager.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskCommunicatorManager.java
@@ -328,7 +328,7 @@ public class TestTaskCommunicatorManager {
     }
 
     @Override
-    public void registerContainerEnd(ContainerId containerId, ContainerEndReason endReason) {
+    public void registerContainerEnd(ContainerId containerId, ContainerEndReason endReason, String diagnostics) {
 
     }
 
@@ -342,7 +342,7 @@ public class TestTaskCommunicatorManager {
 
     @Override
     public void unregisterRunningTaskAttempt(TezTaskAttemptID taskAttemptID,
-                                             TaskAttemptEndReason endReason) {
+                                             TaskAttemptEndReason endReason, String diagnostics) {
 
     }
 

http://git-wip-us.apache.org/repos/asf/tez/blob/fee059b9/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 2bf1c85..947ea93 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
@@ -308,6 +308,7 @@ public class TestTaskAttempt {
         resource, createFakeContainerContext(), false);
 
     NodeId nid = NodeId.newInstance("127.0.0.1", 0);
+    @SuppressWarnings("deprecation")
     ContainerId contId = ContainerId.newInstance(appAttemptId, 3);
     Container container = mock(Container.class);
     when(container.getId()).thenReturn(contId);
@@ -351,6 +352,7 @@ public class TestTaskAttempt {
     Resource resource = Resource.newInstance(1024, 1);
 
     NodeId nid = NodeId.newInstance("127.0.0.1", 0);
+    @SuppressWarnings("deprecation")
     ContainerId contId = ContainerId.newInstance(appAttemptId, 3);
     Container container = mock(Container.class);
     when(container.getId()).thenReturn(contId);
@@ -453,6 +455,7 @@ public class TestTaskAttempt {
     Resource resource = Resource.newInstance(1024, 1);
 
     NodeId nid = NodeId.newInstance("127.0.0.1", 0);
+    @SuppressWarnings("deprecation")
     ContainerId contId = ContainerId.newInstance(appAttemptId, 3);
     Container container = mock(Container.class);
     when(container.getId()).thenReturn(contId);
@@ -519,6 +522,7 @@ public class TestTaskAttempt {
     Resource resource = Resource.newInstance(1024, 1);
 
     NodeId nid = NodeId.newInstance("127.0.0.1", 0);
+    @SuppressWarnings("deprecation")
     ContainerId contId = ContainerId.newInstance(appAttemptId, 3);
     Container container = mock(Container.class);
     when(container.getId()).thenReturn(contId);
@@ -613,6 +617,7 @@ public class TestTaskAttempt {
     Resource resource = Resource.newInstance(1024, 1);
 
     NodeId nid = NodeId.newInstance("127.0.0.1", 0);
+    @SuppressWarnings("deprecation")
     ContainerId contId = ContainerId.newInstance(appAttemptId, 3);
     Container container = mock(Container.class);
     when(container.getId()).thenReturn(contId);
@@ -745,6 +750,7 @@ public class TestTaskAttempt {
     Resource resource = Resource.newInstance(1024, 1);
 
     NodeId nid = NodeId.newInstance("127.0.0.1", 0);
+    @SuppressWarnings("deprecation")
     ContainerId contId = ContainerId.newInstance(appAttemptId, 3);
     Container container = mock(Container.class);
     when(container.getId()).thenReturn(contId);
@@ -837,6 +843,7 @@ public class TestTaskAttempt {
     Resource resource = Resource.newInstance(1024, 1);
 
     NodeId nid = NodeId.newInstance("127.0.0.1", 0);
+    @SuppressWarnings("deprecation")
     ContainerId contId = ContainerId.newInstance(appAttemptId, 3);
     Container container = mock(Container.class);
     when(container.getId()).thenReturn(contId);
@@ -933,6 +940,7 @@ public class TestTaskAttempt {
     Resource resource = Resource.newInstance(1024, 1);
 
     NodeId nid = NodeId.newInstance("127.0.0.1", 0);
+    @SuppressWarnings("deprecation")
     ContainerId contId = ContainerId.newInstance(appAttemptId, 3);
     Container container = mock(Container.class);
     when(container.getId()).thenReturn(contId);
@@ -1037,6 +1045,7 @@ public class TestTaskAttempt {
     Resource resource = Resource.newInstance(1024, 1);
 
     NodeId nid = NodeId.newInstance("127.0.0.1", 0);
+    @SuppressWarnings("deprecation")
     ContainerId contId = ContainerId.newInstance(appAttemptId, 3);
     Container container = mock(Container.class);
     when(container.getId()).thenReturn(contId);
@@ -1138,6 +1147,7 @@ public class TestTaskAttempt {
     Resource resource = Resource.newInstance(1024, 1);
 
     NodeId nid = NodeId.newInstance("127.0.0.1", 0);
+    @SuppressWarnings("deprecation")
     ContainerId contId = ContainerId.newInstance(appAttemptId, 3);
     Container container = mock(Container.class);
     when(container.getId()).thenReturn(contId);
@@ -1342,7 +1352,7 @@ public class TestTaskAttempt {
         }
       }
     }
-  };
+  }
 
   private class MockTaskAttemptImpl extends TaskAttemptImpl {
     

http://git-wip-us.apache.org/repos/asf/tez/blob/fee059b9/tez-dag/src/test/java/org/apache/tez/dag/app/launcher/TestContainerLauncherRouter.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/launcher/TestContainerLauncherRouter.java b/tez-dag/src/test/java/org/apache/tez/dag/app/launcher/TestContainerLauncherRouter.java
index 62a5f19..d0caf8c 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/launcher/TestContainerLauncherRouter.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/launcher/TestContainerLauncherRouter.java
@@ -273,8 +273,7 @@ public class TestContainerLauncherRouter {
                                               TaskAttemptListener taskAttemptListener,
                                               String workingDirectory,
                                               int containerLauncherIndex,
-                                              boolean isPureLocalMode) throws
-        UnknownHostException {
+                                              boolean isPureLocalMode) {
       numContainerLaunchers.incrementAndGet();
       boolean added = containerLauncherIndices.add(containerLauncherIndex);
       assertTrue("Cannot add multiple launchers with the same index", added);
@@ -298,8 +297,7 @@ public class TestContainerLauncherRouter {
                                                   AppContext context,
                                                   TaskAttemptListener taskAttemptListener,
                                                   String workingDirectory,
-                                                  boolean isPureLocalMode) throws
-        UnknownHostException {
+                                                  boolean isPureLocalMode) {
       uberContainerLauncherCreated.set(true);
       testContainerLaunchers.add(uberContainerlauncher);
       return uberContainerlauncher;


[24/50] [abbrv] tez git commit: TEZ-2502. Fix for TezTaskRunner2 not killing tasks properly in all situations. (sseth)

Posted by ss...@apache.org.
TEZ-2502. Fix for TezTaskRunner2 not killing tasks properly in all situations. (sseth)


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

Branch: refs/heads/TEZ-2003
Commit: 2a3e2b3acf2feb994d382efe246cae9b29474ae2
Parents: 2c19f3c
Author: Siddharth Seth <ss...@apache.org>
Authored: Thu May 28 18:29:12 2015 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Fri Aug 14 13:46:44 2015 -0700

----------------------------------------------------------------------
 TEZ-2003-CHANGES.txt                            |  1 +
 .../apache/tez/runtime/task/TezTaskRunner2.java | 83 ++++++++++++--------
 2 files changed, 53 insertions(+), 31 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/2a3e2b3a/TEZ-2003-CHANGES.txt
----------------------------------------------------------------------
diff --git a/TEZ-2003-CHANGES.txt b/TEZ-2003-CHANGES.txt
index e333832..42c2e1e 100644
--- a/TEZ-2003-CHANGES.txt
+++ b/TEZ-2003-CHANGES.txt
@@ -29,5 +29,6 @@ ALL CHANGES:
   TEZ-2465. Retrun the status of a kill request in TaskRunner2.
   TEZ-2471. NPE in LogicalIOProcessorRuntimeTask while printing thread info.
   TEZ-2495. Inform TaskCommunicaor about Task and Container termination reasons.
+  TEZ-2502. Fix for TezTaskRunner2 not killing tasks properly in all situations.
 
 INCOMPATIBLE CHANGES:

http://git-wip-us.apache.org/repos/asf/tez/blob/2a3e2b3a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezTaskRunner2.java
----------------------------------------------------------------------
diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezTaskRunner2.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezTaskRunner2.java
index 15629fd..a5fabb5 100644
--- a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezTaskRunner2.java
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezTaskRunner2.java
@@ -124,6 +124,8 @@ public class TezTaskRunner2 {
     try {
       ListenableFuture<TaskRunner2CallableResult> future = null;
       synchronized (this) {
+        // All running state changes must be made within a synchronized block to ensure
+        // kills are issued or the task is not setup.
         if (isRunningState()) {
           // Safe to do this within a synchronized block because we're providing
           // the handler on which the Reporter will communicate back. Assuming
@@ -252,27 +254,34 @@ public class TezTaskRunner2 {
    * @return true if the task kill was honored, false otherwise
    */
   public boolean killTask() {
+    boolean isFirstError = false;
     synchronized (this) {
       if (isRunningState()) {
         if (trySettingEndReason(EndReason.KILL_REQUESTED)) {
+          isFirstError = true;
           killTaskRequested.set(true);
-          if (taskRunnerCallable != null) {
-            taskKillStartTime = System.currentTimeMillis();
-            taskRunnerCallable.interruptTask();
-          }
-          return true;
         } else {
-          LOG.info("Ignoring killTask request for {} since end reason is already set to {}",
-              task.getTaskAttemptID(), firstEndReason);
+          logErrorIngored("killTask", null);
         }
       } else {
-        LOG.info("Ignoring killTask request for {} since it is not in a running state",
-            task.getTaskAttemptID());
+        logErrorIngored("killTask", null);
       }
     }
-    return false;
+    if (isFirstError) {
+      logAborting("killTask");
+      killTaskInternal();
+      return true;
+    } else {
+      return false;
+    }
   }
 
+  private void killTaskInternal() {
+    if (taskRunnerCallable != null) {
+      taskKillStartTime = System.currentTimeMillis();
+      taskRunnerCallable.interruptTask();
+    }
+  }
 
   // Checks and changes on these states should happen within a synchronized block,
   // to ensure the first event is the one that is captured and causes specific behaviour.
@@ -310,17 +319,18 @@ public class TezTaskRunner2 {
             errorReporterToAm.set(true);
             oobSignalErrorInProgress = true;
           } else {
-            LOG.info(
-                "Ignoring fatal error since the task has ended for reason: {}. IgnoredError: {} ",
-                firstEndReason, (t == null ? message : t.getMessage()));
+            logErrorIngored("signalFatalError", message);
           }
+        } else {
+          logErrorIngored("signalFatalError", message);
         }
       }
 
       // Informing the TaskReporter here because the running task may not be interruptable.
       // Has to be outside the lock.
       if (isFirstError) {
-        killTask();
+        logAborting("signalFatalError");
+        killTaskInternal();
         try {
           taskReporter.taskFailed(taskAttemptID, t, getTaskDiagnosticsString(t, message), sourceInfo);
         } catch (IOException e) {
@@ -371,19 +381,22 @@ public class TezTaskRunner2 {
           if (trySettingEndReason(EndReason.COMMUNICATION_FAILURE)) {
             registerFirstException(t, null);
             isFirstError = true;
+          } else {
+            logErrorIngored("umbilicalFatalError", null);
           }
           // A race is possible between a task succeeding, and a subsequent timed heartbeat failing.
           // These errors can be ignored, since a task can only succeed if the synchronous taskSucceeded
           // method does not throw an exception, in which case task success is registered with the AM.
           // Leave subsequent heartbeat errors to the next entity to communicate using the TaskReporter
         } else {
-          LOG.info("Ignoring Communication failure since task with id=" + task.getTaskAttemptID()
-              + " is already complete, is failing or has been asked to terminate");
+          logErrorIngored("umbilicalFatalError", null);
         }
+        // Since this error came from the taskReporter - there's no point attempting to report a failure back to it.
+        // However, the task does need to be cleaned up
       }
-      // Since this error came from the taskReporter - there's no point attempting to report a failure back to it.
       if (isFirstError) {
-        killTask();
+        logAborting("umbilicalFatalError");
+        killTaskInternal();
       }
     }
 
@@ -395,18 +408,12 @@ public class TezTaskRunner2 {
         isFirstTerminate = trySettingEndReason(EndReason.CONTAINER_STOP_REQUESTED);
         // Respect stopContainerRequested since it can come in at any point, despite a previous failure.
         stopContainerRequested.set(true);
-
-        if (isFirstTerminate) {
-          LOG.info("Attempting to abort {} since a shutdown request was received",
-              task.getTaskAttemptID());
-          if (taskRunnerCallable != null) {
-            taskKillStartTime = System.currentTimeMillis();
-            taskRunnerCallable.interruptTask();
-          }
-        } else {
-          LOG.info("Not acting on shutdown request for {} since the task is not in running state",
-              task.getTaskAttemptID());
-        }
+      }
+      if (isFirstTerminate) {
+        logAborting("shutdownRequested");
+        killTaskInternal();
+      } else {
+        logErrorIngored("shutdownRequested", null);
       }
     }
   }
@@ -451,6 +458,20 @@ public class TezTaskRunner2 {
 
   private void handleFinalStatusUpdateFailure(Throwable t, boolean successReportAttempted) {
     // TODO Ideally differentiate between FAILED/KILLED
-    LOG.warn("Failure while reporting state= {} to AM", (successReportAttempted ? "success" : "failure/killed"), t);
+    LOG.warn("Failure while reporting state= {} to AM",
+        (successReportAttempted ? "success" : "failure/killed"), t);
+  }
+
+  private void logErrorIngored(String ignoredEndReason, String errorMessage) {
+    LOG.info(
+        "Ignoring {} request since the task with id {} has ended for reason: {}. IgnoredError: {} ",
+        ignoredEndReason, task.getTaskAttemptID(),
+        firstEndReason, (firstException == null ? (errorMessage == null ? "" : errorMessage) :
+            firstException.getMessage()));
+  }
+
+  private void logAborting(String abortReason) {
+    LOG.info("Attempting to abort {} due to an invocation of {}", task.getTaskAttemptID(),
+        abortReason);
   }
 }
\ No newline at end of file


[50/50] [abbrv] tez git commit: TEZ-2721. rebase 08/14 (sseth)

Posted by ss...@apache.org.
TEZ-2721. rebase 08/14 (sseth)


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

Branch: refs/heads/TEZ-2003
Commit: ded95e59ecfc0a5931bf6946f2b9b4641dbfe534
Parents: 4b0ffdd
Author: Siddharth Seth <ss...@apache.org>
Authored: Fri Aug 14 13:57:26 2015 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Fri Aug 14 13:57:26 2015 -0700

----------------------------------------------------------------------
 TEZ-2003-CHANGES.txt                                               | 1 +
 .../main/java/org/apache/tez/dag/app/dag/impl/TaskAttemptImpl.java | 2 +-
 2 files changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/ded95e59/TEZ-2003-CHANGES.txt
----------------------------------------------------------------------
diff --git a/TEZ-2003-CHANGES.txt b/TEZ-2003-CHANGES.txt
index 154fb11..8a8e257 100644
--- a/TEZ-2003-CHANGES.txt
+++ b/TEZ-2003-CHANGES.txt
@@ -48,5 +48,6 @@ ALL CHANGES:
   TEZ-2678. Fix comments from reviews - part 1.
   TEZ-2707. Fix comments from reviews - part 2.
   TEZ-2713. Add tests for node handling when there's multiple schedulers.
+  TEZ-2721. rebase 08/14
 
 INCOMPATIBLE CHANGES:

http://git-wip-us.apache.org/repos/asf/tez/blob/ded95e59/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 1c4102d..6957b1d 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
@@ -175,6 +175,7 @@ public class TaskAttemptImpl implements TaskAttempt,
   
   private TezTaskAttemptID creationCausalTA;
   private long creationTime;
+  private long scheduledTime;
 
   protected static final FailedTransitionHelper FAILED_HELPER =
       new FailedTransitionHelper();
@@ -439,7 +440,6 @@ public class TaskAttemptImpl implements TaskAttempt,
     this.vertex = this.task.getVertex();
     this.creationCausalTA = schedulingCausalTA;
     this.creationTime = clock.getTime();
-    this.schedulingCausalTA = schedulingCausalTA;
 
     this.reportedStatus = new TaskAttemptStatus(this.attemptId);
     initTaskAttemptStatus(reportedStatus);


[15/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/a5f872e9
Tree: http://git-wip-us.apache.org/repos/asf/tez/tree/a5f872e9
Diff: http://git-wip-us.apache.org/repos/asf/tez/diff/a5f872e9

Branch: refs/heads/TEZ-2003
Commit: a5f872e908835a152466b670164bf9f46568fa05
Parents: c44b337
Author: Siddharth Seth <ss...@apache.org>
Authored: Thu Apr 23 17:26:25 2015 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Fri Aug 14 13:46:43 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/a5f872e9/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/a5f872e9/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/a5f872e9/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 6ab0f8e..04e72db 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
@@ -766,7 +766,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());
@@ -780,7 +780,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/a5f872e9/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 386e4af..7cdf292 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
@@ -80,6 +80,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;
@@ -122,7 +123,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
@@ -147,10 +150,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
@@ -158,7 +161,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) {
@@ -317,6 +320,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/a5f872e9/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/a5f872e9/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 fa2749a..6200a5b 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
@@ -260,6 +260,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/a5f872e9/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/a5f872e9/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/a5f872e9/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/a5f872e9/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 7dbf937..fe23409 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;
@@ -170,14 +169,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/a5f872e9/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 b39eee2..9882954 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;
@@ -150,14 +147,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/a5f872e9/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/a5f872e9/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/a5f872e9/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,


[46/50] [abbrv] tez git commit: TEZ-2657. Add tests for client side changes - specifying plugins, etc. (sseth)

Posted by ss...@apache.org.
TEZ-2657. Add tests for client side changes - specifying plugins, etc. (sseth)


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

Branch: refs/heads/TEZ-2003
Commit: e445576ef97d6ec9226065f502173d33bfaa2e57
Parents: 25dd309
Author: Siddharth Seth <ss...@apache.org>
Authored: Wed Jul 29 18:26:01 2015 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Fri Aug 14 13:47:10 2015 -0700

----------------------------------------------------------------------
 TEZ-2003-CHANGES.txt                            |   1 +
 .../java/org/apache/tez/client/TezClient.java   |  15 +-
 .../org/apache/tez/client/TezClientUtils.java   |  38 +---
 .../apache/tez/dag/api/DagTypeConverters.java   |  67 +++++--
 .../java/org/apache/tez/dag/api/Vertex.java     |  41 ++++
 .../api/ServicePluginsDescriptor.java           |  36 ++++
 tez-api/src/main/proto/DAGApiRecords.proto      |   2 +-
 .../org/apache/tez/client/TestTezClient.java    | 113 +++++++++--
 .../apache/tez/client/TestTezClientUtils.java   |  16 +-
 .../org/apache/tez/dag/api/TestDAGPlan.java     |  63 +++++-
 .../tez/dag/api/TestDagTypeConverters.java      | 196 ++++++++++++++++++-
 .../org/apache/tez/dag/app/DAGAppMaster.java    |   4 +-
 12 files changed, 508 insertions(+), 84 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/e445576e/TEZ-2003-CHANGES.txt
----------------------------------------------------------------------
diff --git a/TEZ-2003-CHANGES.txt b/TEZ-2003-CHANGES.txt
index 9d72d92..9b3967a 100644
--- a/TEZ-2003-CHANGES.txt
+++ b/TEZ-2003-CHANGES.txt
@@ -40,5 +40,6 @@ ALL CHANGES:
   TEZ-2652. Cleanup the way services are specified for an AM and vertices.
   TEZ-2653. Change service contexts to expose a user specified payload instead of the AM configuration.
   TEZ-2441. Add tests for TezTaskRunner2.
+  TEZ-2657. Add tests for client side changes - specifying plugins, etc.
 
 INCOMPATIBLE CHANGES:

http://git-wip-us.apache.org/repos/asf/tez/blob/e445576e/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 8759fdc..e8b5386 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
@@ -95,13 +95,16 @@ public class TezClient {
   @VisibleForTesting
   static final String NO_CLUSTER_DIAGNOSTICS_MSG = "No cluster diagnostics found.";
 
-  private final String clientName;
+  @VisibleForTesting
+  final String clientName;
   private ApplicationId sessionAppId;
   private ApplicationId lastSubmittedAppId;
-  private AMConfiguration amConfig;
+  @VisibleForTesting
+  final AMConfiguration amConfig;
   private FrameworkClient frameworkClient;
   private String diagnostics;
-  private boolean isSession;
+  @VisibleForTesting
+  final boolean isSession;
   private boolean sessionStarted = false;
   private boolean sessionStopped = false;
   /** Tokens which will be required for all DAGs submitted to this session. */
@@ -113,8 +116,10 @@ public class TezClient {
   private JobTokenSecretManager jobTokenSecretManager =
       new JobTokenSecretManager();
   private final Map<String, LocalResource> additionalLocalResources = Maps.newHashMap();
-  private final TezApiVersionInfo apiVersionInfo;
-  private final ServicePluginsDescriptor servicePluginsDescriptor;
+  @VisibleForTesting
+  final TezApiVersionInfo apiVersionInfo;
+  @VisibleForTesting
+  final ServicePluginsDescriptor servicePluginsDescriptor;
   private HistoryACLPolicyManager historyACLPolicyManager;
 
   private int preWarmDAGCounter = 0;

http://git-wip-us.apache.org/repos/asf/tez/blob/e445576e/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java b/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java
index 9cf1f3f..6086fa1 100644
--- a/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java
+++ b/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java
@@ -39,9 +39,7 @@ import java.util.Map.Entry;
 
 import com.google.common.base.Strings;
 import org.apache.commons.lang.StringUtils;
-import org.apache.tez.dag.api.NamedEntityDescriptor;
 import org.apache.tez.dag.api.records.DAGProtos.AMPluginDescriptorProto;
-import org.apache.tez.dag.api.records.DAGProtos.TezNamedEntityDescriptorProto;
 import org.apache.tez.serviceplugins.api.ServicePluginsDescriptor;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -778,47 +776,13 @@ public class TezClientUtils {
     }
 
     AMPluginDescriptorProto pluginDescriptorProto =
-        createAMServicePluginDescriptorProto(servicePluginsDescriptor);
+        DagTypeConverters.convertServicePluginDescriptoToProto(servicePluginsDescriptor);
     builder.setAmPluginDescriptor(pluginDescriptorProto);
 
     return builder.build();
   }
 
-  static AMPluginDescriptorProto createAMServicePluginDescriptorProto(
-      ServicePluginsDescriptor servicePluginsDescriptor) {
-    AMPluginDescriptorProto.Builder pluginDescriptorBuilder =
-        AMPluginDescriptorProto.newBuilder();
-    if (servicePluginsDescriptor != null) {
 
-      pluginDescriptorBuilder.setContainersEnabled(servicePluginsDescriptor.areContainersEnabled());
-      pluginDescriptorBuilder.setUberEnabled(servicePluginsDescriptor.isUberEnabled());
-
-      if (servicePluginsDescriptor.getTaskSchedulerDescriptors() != null &&
-          servicePluginsDescriptor.getTaskSchedulerDescriptors().length > 0) {
-        List<TezNamedEntityDescriptorProto> namedEntityProtos = DagTypeConverters.convertNamedEntityCollectionToProto(
-            servicePluginsDescriptor.getTaskSchedulerDescriptors());
-        pluginDescriptorBuilder.addAllTaskScedulers(namedEntityProtos);
-      }
-
-      if (servicePluginsDescriptor.getContainerLauncherDescriptors() != null &&
-          servicePluginsDescriptor.getContainerLauncherDescriptors().length > 0) {
-        List<TezNamedEntityDescriptorProto> namedEntityProtos = DagTypeConverters.convertNamedEntityCollectionToProto(
-            servicePluginsDescriptor.getContainerLauncherDescriptors());
-        pluginDescriptorBuilder.addAllContainerLaunchers(namedEntityProtos);
-      }
-
-      if (servicePluginsDescriptor.getTaskCommunicatorDescriptors() != null &&
-          servicePluginsDescriptor.getTaskCommunicatorDescriptors().length > 0) {
-        List<TezNamedEntityDescriptorProto> namedEntityProtos = DagTypeConverters.convertNamedEntityCollectionToProto(
-            servicePluginsDescriptor.getTaskCommunicatorDescriptors());
-        pluginDescriptorBuilder.addAllTaskCommunicators(namedEntityProtos);
-      }
-
-    } else {
-      pluginDescriptorBuilder.setContainersEnabled(true).setUberEnabled(false);
-    }
-    return pluginDescriptorBuilder.build();
-  }
 
   /**
    * Helper function to create a YARN LocalResource

http://git-wip-us.apache.org/repos/asf/tez/blob/e445576e/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 2e0d417..61e4d33 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
@@ -52,9 +52,11 @@ import org.apache.tez.common.counters.TezCounters;
 import org.apache.tez.dag.api.EdgeProperty.DataMovementType;
 import org.apache.tez.dag.api.EdgeProperty.DataSourceType;
 import org.apache.tez.dag.api.EdgeProperty.SchedulingType;
+import org.apache.tez.dag.api.Vertex.VertexExecutionContext;
 import org.apache.tez.dag.api.client.StatusGetOpts;
 import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolRPC.TezAppMasterStatusProto;
 import org.apache.tez.dag.api.records.DAGProtos;
+import org.apache.tez.dag.api.records.DAGProtos.AMPluginDescriptorProto;
 import org.apache.tez.dag.api.records.DAGProtos.ConfigurationProto;
 import org.apache.tez.dag.api.records.DAGProtos.EdgePlan;
 import org.apache.tez.dag.api.records.DAGProtos.PlanEdgeDataMovementType;
@@ -74,14 +76,13 @@ import org.apache.tez.dag.api.records.DAGProtos.TezCounterProto;
 import org.apache.tez.dag.api.records.DAGProtos.TezCountersProto;
 import org.apache.tez.dag.api.records.DAGProtos.TezEntityDescriptorProto;
 import org.apache.tez.dag.api.records.DAGProtos.TezNamedEntityDescriptorProto;
+import org.apache.tez.dag.api.records.DAGProtos.VertexExecutionContextProto;
 import org.apache.tez.dag.api.records.DAGProtos.VertexLocationHintProto;
 
 import com.google.common.collect.Lists;
 import com.google.protobuf.ByteString;
 import com.google.protobuf.ByteString.Output;
-import org.apache.tez.serviceplugins.api.ContainerLauncherDescriptor;
-import org.apache.tez.serviceplugins.api.TaskCommunicatorDescriptor;
-import org.apache.tez.serviceplugins.api.TaskSchedulerDescriptor;
+import org.apache.tez.serviceplugins.api.ServicePluginsDescriptor;
 
 @Private
 public class DagTypeConverters {
@@ -732,13 +733,13 @@ public class DagTypeConverters {
     return payload.getPayload();
   }
 
-  public static DAGProtos.VertexExecutionContextProto convertToProto(
-      Vertex.VertexExecutionContext context) {
+  public static VertexExecutionContextProto convertToProto(
+      VertexExecutionContext context) {
     if (context == null) {
       return null;
     } else {
-      DAGProtos.VertexExecutionContextProto.Builder builder =
-          DAGProtos.VertexExecutionContextProto.newBuilder();
+      VertexExecutionContextProto.Builder builder =
+          VertexExecutionContextProto.newBuilder();
       builder.setExecuteInAm(context.shouldExecuteInAm());
       builder.setExecuteInContainers(context.shouldExecuteInContainers());
       if (context.getTaskSchedulerName() != null) {
@@ -754,26 +755,26 @@ public class DagTypeConverters {
     }
   }
 
-  public static Vertex.VertexExecutionContext convertFromProto(
-      DAGProtos.VertexExecutionContextProto proto) {
+  public static VertexExecutionContext convertFromProto(
+      VertexExecutionContextProto proto) {
     if (proto == null) {
       return null;
     } else {
       if (proto.getExecuteInAm()) {
-        Vertex.VertexExecutionContext context =
-            Vertex.VertexExecutionContext.createExecuteInAm(proto.getExecuteInAm());
+        VertexExecutionContext context =
+            VertexExecutionContext.createExecuteInAm(proto.getExecuteInAm());
         return context;
       } else if (proto.getExecuteInContainers()) {
-        Vertex.VertexExecutionContext context =
-            Vertex.VertexExecutionContext.createExecuteInContainers(proto.getExecuteInContainers());
+        VertexExecutionContext context =
+            VertexExecutionContext.createExecuteInContainers(proto.getExecuteInContainers());
         return context;
       } else {
         String taskScheduler = proto.hasTaskSchedulerName() ? proto.getTaskSchedulerName() : null;
         String containerLauncher =
             proto.hasContainerLauncherName() ? proto.getContainerLauncherName() : null;
         String taskComm = proto.hasTaskCommName() ? proto.getTaskCommName() : null;
-        Vertex.VertexExecutionContext context =
-            Vertex.VertexExecutionContext.create(taskScheduler, containerLauncher, taskComm);
+        VertexExecutionContext context =
+            VertexExecutionContext.create(taskScheduler, containerLauncher, taskComm);
         return context;
       }
     }
@@ -800,4 +801,40 @@ public class DagTypeConverters {
     return builder.build();
   }
 
+  public static AMPluginDescriptorProto convertServicePluginDescriptoToProto(
+      ServicePluginsDescriptor servicePluginsDescriptor) {
+    AMPluginDescriptorProto.Builder pluginDescriptorBuilder =
+        AMPluginDescriptorProto.newBuilder();
+    if (servicePluginsDescriptor != null) {
+
+      pluginDescriptorBuilder.setContainersEnabled(servicePluginsDescriptor.areContainersEnabled());
+      pluginDescriptorBuilder.setUberEnabled(servicePluginsDescriptor.isUberEnabled());
+
+      if (servicePluginsDescriptor.getTaskSchedulerDescriptors() != null &&
+          servicePluginsDescriptor.getTaskSchedulerDescriptors().length > 0) {
+        List<TezNamedEntityDescriptorProto> namedEntityProtos = DagTypeConverters.convertNamedEntityCollectionToProto(
+            servicePluginsDescriptor.getTaskSchedulerDescriptors());
+        pluginDescriptorBuilder.addAllTaskSchedulers(namedEntityProtos);
+      }
+
+      if (servicePluginsDescriptor.getContainerLauncherDescriptors() != null &&
+          servicePluginsDescriptor.getContainerLauncherDescriptors().length > 0) {
+        List<TezNamedEntityDescriptorProto> namedEntityProtos = DagTypeConverters.convertNamedEntityCollectionToProto(
+            servicePluginsDescriptor.getContainerLauncherDescriptors());
+        pluginDescriptorBuilder.addAllContainerLaunchers(namedEntityProtos);
+      }
+
+      if (servicePluginsDescriptor.getTaskCommunicatorDescriptors() != null &&
+          servicePluginsDescriptor.getTaskCommunicatorDescriptors().length > 0) {
+        List<TezNamedEntityDescriptorProto> namedEntityProtos = DagTypeConverters.convertNamedEntityCollectionToProto(
+            servicePluginsDescriptor.getTaskCommunicatorDescriptors());
+        pluginDescriptorBuilder.addAllTaskCommunicators(namedEntityProtos);
+      }
+
+    } else {
+      pluginDescriptorBuilder.setContainersEnabled(true).setUberEnabled(false);
+    }
+    return pluginDescriptorBuilder.build();
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/e445576e/tez-api/src/main/java/org/apache/tez/dag/api/Vertex.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/Vertex.java b/tez-api/src/main/java/org/apache/tez/dag/api/Vertex.java
index 34124b2..8953ae1 100644
--- a/tez-api/src/main/java/org/apache/tez/dag/api/Vertex.java
+++ b/tez-api/src/main/java/org/apache/tez/dag/api/Vertex.java
@@ -511,6 +511,47 @@ public class Vertex {
           ", taskCommName='" + taskCommName + '\'' +
           '}';
     }
+
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) {
+        return true;
+      }
+      if (o == null || getClass() != o.getClass()) {
+        return false;
+      }
+
+      VertexExecutionContext that = (VertexExecutionContext) o;
+
+      if (executeInAm != that.executeInAm) {
+        return false;
+      }
+      if (executeInContainers != that.executeInContainers) {
+        return false;
+      }
+      if (taskSchedulerName != null ? !taskSchedulerName.equals(that.taskSchedulerName) :
+          that.taskSchedulerName != null) {
+        return false;
+      }
+      if (containerLauncherName != null ?
+          !containerLauncherName.equals(that.containerLauncherName) :
+          that.containerLauncherName != null) {
+        return false;
+      }
+      return !(taskCommName != null ? !taskCommName.equals(that.taskCommName) :
+          that.taskCommName != null);
+
+    }
+
+    @Override
+    public int hashCode() {
+      int result = (executeInAm ? 1 : 0);
+      result = 31 * result + (executeInContainers ? 1 : 0);
+      result = 31 * result + (taskSchedulerName != null ? taskSchedulerName.hashCode() : 0);
+      result = 31 * result + (containerLauncherName != null ? containerLauncherName.hashCode() : 0);
+      result = 31 * result + (taskCommName != null ? taskCommName.hashCode() : 0);
+      return result;
+    }
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/tez/blob/e445576e/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ServicePluginsDescriptor.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ServicePluginsDescriptor.java b/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ServicePluginsDescriptor.java
index 8df102a..2e4fc46 100644
--- a/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ServicePluginsDescriptor.java
+++ b/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ServicePluginsDescriptor.java
@@ -46,6 +46,15 @@ public class ServicePluginsDescriptor {
     this.taskCommunicatorDescriptors = taskCommunicatorDescriptors;
   }
 
+  /**
+   * Create a service plugin descriptor with the provided plugins. Regular containers will also be enabled
+   * when using this method.
+   *
+   * @param taskSchedulerDescriptor the task scheduler plugin descriptors
+   * @param containerLauncherDescriptors the container launcher plugin descriptors
+   * @param taskCommunicatorDescriptors the task communicator plugin descriptors
+   * @return
+   */
   public static ServicePluginsDescriptor create(TaskSchedulerDescriptor[] taskSchedulerDescriptor,
                                                 ContainerLauncherDescriptor[] containerLauncherDescriptors,
                                                 TaskCommunicatorDescriptor[] taskCommunicatorDescriptors) {
@@ -53,6 +62,15 @@ public class ServicePluginsDescriptor {
         containerLauncherDescriptors, taskCommunicatorDescriptors);
   }
 
+  /**
+   * Create a service plugin descriptor with the provided plugins. Also allows specification of whether
+   * in-AM execution is enabled. Container execution is enabled by default.
+   * @param enableUber whether to enable execution in the AM or not
+   * @param taskSchedulerDescriptor the task scheduler plugin descriptors
+   * @param containerLauncherDescriptors the container launcher plugin descriptors
+   * @param taskCommunicatorDescriptors the task communicator plugin descriptors
+   * @return
+   */
   public static ServicePluginsDescriptor create(boolean enableUber,
                                                 TaskSchedulerDescriptor[] taskSchedulerDescriptor,
                                                 ContainerLauncherDescriptor[] containerLauncherDescriptors,
@@ -61,6 +79,17 @@ public class ServicePluginsDescriptor {
         containerLauncherDescriptors, taskCommunicatorDescriptors);
   }
 
+  /**
+   * Create a service plugin descriptor with the provided plugins. Also allows specification of whether
+   * container execution and in-AM execution will be enabled.
+   *
+   * @param enableContainers whether to enable execution in containers
+   * @param enableUber whether to enable execution in the AM or not
+   * @param taskSchedulerDescriptor the task scheduler plugin descriptors
+   * @param containerLauncherDescriptors the container launcher plugin descriptors
+   * @param taskCommunicatorDescriptors the task communicator plugin descriptors
+   * @return
+   */
   public static ServicePluginsDescriptor create(boolean enableContainers, boolean enableUber,
                                                 TaskSchedulerDescriptor[] taskSchedulerDescriptor,
                                                 ContainerLauncherDescriptor[] containerLauncherDescriptors,
@@ -69,6 +98,13 @@ public class ServicePluginsDescriptor {
         containerLauncherDescriptors, taskCommunicatorDescriptors);
   }
 
+  /**
+   * Create a service plugin descriptor which may have in-AM execution of tasks enabled. Container
+   * execution is enabled by default
+   *
+   * @param enableUber whether to enable execution in the AM or not
+   * @return
+   */
   public static ServicePluginsDescriptor create(boolean enableUber) {
     return new ServicePluginsDescriptor(true, enableUber, null, null, null);
   }

http://git-wip-us.apache.org/repos/asf/tez/blob/e445576e/tez-api/src/main/proto/DAGApiRecords.proto
----------------------------------------------------------------------
diff --git a/tez-api/src/main/proto/DAGApiRecords.proto b/tez-api/src/main/proto/DAGApiRecords.proto
index ebe3259..193f7b8 100644
--- a/tez-api/src/main/proto/DAGApiRecords.proto
+++ b/tez-api/src/main/proto/DAGApiRecords.proto
@@ -180,7 +180,7 @@ message TezNamedEntityDescriptorProto {
 message AMPluginDescriptorProto {
   optional bool containers_enabled = 1 [default = true];
   optional bool uber_enabled = 2 [default = false];
-  repeated TezNamedEntityDescriptorProto task_scedulers = 3;
+  repeated TezNamedEntityDescriptorProto task_schedulers = 3;
   repeated TezNamedEntityDescriptorProto container_launchers = 4;
   repeated TezNamedEntityDescriptorProto task_communicators = 5;
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/e445576e/tez-api/src/test/java/org/apache/tez/client/TestTezClient.java
----------------------------------------------------------------------
diff --git a/tez-api/src/test/java/org/apache/tez/client/TestTezClient.java b/tez-api/src/test/java/org/apache/tez/client/TestTezClient.java
index dc0fbb1..7a642e6 100644
--- a/tez-api/src/test/java/org/apache/tez/client/TestTezClient.java
+++ b/tez-api/src/test/java/org/apache/tez/client/TestTezClient.java
@@ -26,6 +26,11 @@ import java.util.concurrent.atomic.AtomicReference;
 
 import javax.annotation.Nullable;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
 import static org.mockito.Matchers.any;
 import static org.mockito.Mockito.RETURNS_DEEP_STUBS;
 import static org.mockito.Mockito.mock;
@@ -61,6 +66,7 @@ import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolRPC.GetAMStatusRespo
 import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolRPC.ShutdownSessionRequestProto;
 import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolRPC.SubmitDAGRequestProto;
 import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolRPC.TezAppMasterStatusProto;
+import org.apache.tez.serviceplugins.api.ServicePluginsDescriptor;
 import org.hamcrest.CoreMatchers;
 import org.junit.Assert;
 import org.junit.Test;
@@ -153,11 +159,11 @@ public class TestTezClient {
       verify(client.mockYarnClient, times(1)).submitApplication(captor.capture());
       ApplicationSubmissionContext context = captor.getValue();
       Assert.assertEquals(3, context.getAMContainerSpec().getLocalResources().size());
-      Assert.assertTrue(context.getAMContainerSpec().getLocalResources().containsKey(
+      assertTrue(context.getAMContainerSpec().getLocalResources().containsKey(
           TezConstants.TEZ_AM_LOCAL_RESOURCES_PB_FILE_NAME));
-      Assert.assertTrue(context.getAMContainerSpec().getLocalResources().containsKey(
+      assertTrue(context.getAMContainerSpec().getLocalResources().containsKey(
           TezConstants.TEZ_PB_BINARY_CONF_NAME));
-      Assert.assertTrue(context.getAMContainerSpec().getLocalResources().containsKey(
+      assertTrue(context.getAMContainerSpec().getLocalResources().containsKey(
           lrName1));
     } else {
       verify(client.mockYarnClient, times(0)).submitApplication(captor.capture());
@@ -172,7 +178,7 @@ public class TestTezClient {
     DAG dag = DAG.create("DAG").addVertex(vertex).addTaskLocalFiles(lrDAG);
     DAGClient dagClient = client.submitDAG(dag);
         
-    Assert.assertTrue(dagClient.getExecutionContext().contains(client.mockAppId.toString()));
+    assertTrue(dagClient.getExecutionContext().contains(client.mockAppId.toString()));
     
     if (isSession) {
       verify(client.mockYarnClient, times(1)).submitApplication(captor.capture());
@@ -181,13 +187,13 @@ public class TestTezClient {
       verify(client.mockYarnClient, times(1)).submitApplication(captor.capture());
       ApplicationSubmissionContext context = captor.getValue();
       Assert.assertEquals(4, context.getAMContainerSpec().getLocalResources().size());
-      Assert.assertTrue(context.getAMContainerSpec().getLocalResources().containsKey(
+      assertTrue(context.getAMContainerSpec().getLocalResources().containsKey(
           TezConstants.TEZ_AM_LOCAL_RESOURCES_PB_FILE_NAME));
-      Assert.assertTrue(context.getAMContainerSpec().getLocalResources().containsKey(
+      assertTrue(context.getAMContainerSpec().getLocalResources().containsKey(
           TezConstants.TEZ_PB_BINARY_CONF_NAME));
-      Assert.assertTrue(context.getAMContainerSpec().getLocalResources().containsKey(
+      assertTrue(context.getAMContainerSpec().getLocalResources().containsKey(
           TezConstants.TEZ_PB_PLAN_BINARY_NAME));
-      Assert.assertTrue(context.getAMContainerSpec().getLocalResources().containsKey(
+      assertTrue(context.getAMContainerSpec().getLocalResources().containsKey(
           lrName1));
     }
     
@@ -211,7 +217,7 @@ public class TestTezClient {
     if (isSession) {
       // same app master
       verify(client.mockYarnClient, times(1)).submitApplication(captor.capture());
-      Assert.assertTrue(dagClient.getExecutionContext().contains(client.mockAppId.toString()));
+      assertTrue(dagClient.getExecutionContext().contains(client.mockAppId.toString()));
       // additional resource is sent
       ArgumentCaptor<SubmitDAGRequestProto> captor1 = ArgumentCaptor.forClass(SubmitDAGRequestProto.class);
       verify(client.sessionAmProxy, times(2)).submitDAG((RpcController)any(), captor1.capture());
@@ -220,20 +226,20 @@ public class TestTezClient {
       Assert.assertEquals(lrName2, proto.getAdditionalAmResources().getLocalResources(0).getName());
     } else {
       // new app master
-      Assert.assertTrue(dagClient.getExecutionContext().contains(appId2.toString()));
+      assertTrue(dagClient.getExecutionContext().contains(appId2.toString()));
       verify(client.mockYarnClient, times(2)).submitApplication(captor.capture());
       // additional resource is added
       ApplicationSubmissionContext context = captor.getValue();
       Assert.assertEquals(5, context.getAMContainerSpec().getLocalResources().size());
-      Assert.assertTrue(context.getAMContainerSpec().getLocalResources().containsKey(
+      assertTrue(context.getAMContainerSpec().getLocalResources().containsKey(
           TezConstants.TEZ_AM_LOCAL_RESOURCES_PB_FILE_NAME));
-      Assert.assertTrue(context.getAMContainerSpec().getLocalResources().containsKey(
+      assertTrue(context.getAMContainerSpec().getLocalResources().containsKey(
           TezConstants.TEZ_PB_BINARY_CONF_NAME));
-      Assert.assertTrue(context.getAMContainerSpec().getLocalResources().containsKey(
+      assertTrue(context.getAMContainerSpec().getLocalResources().containsKey(
           TezConstants.TEZ_PB_PLAN_BINARY_NAME));
-      Assert.assertTrue(context.getAMContainerSpec().getLocalResources().containsKey(
+      assertTrue(context.getAMContainerSpec().getLocalResources().containsKey(
           lrName1));
-      Assert.assertTrue(context.getAMContainerSpec().getLocalResources().containsKey(
+      assertTrue(context.getAMContainerSpec().getLocalResources().containsKey(
           lrName2));
     }
     
@@ -263,7 +269,7 @@ public class TestTezClient {
     ArgumentCaptor<SubmitDAGRequestProto> captor1 = ArgumentCaptor.forClass(SubmitDAGRequestProto.class);
     verify(client.sessionAmProxy, times(1)).submitDAG((RpcController)any(), captor1.capture());
     SubmitDAGRequestProto proto = captor1.getValue();
-    Assert.assertTrue(proto.getDAGPlan().getName().startsWith(TezConstants.TEZ_PREWARM_DAG_NAME_PREFIX));
+    assertTrue(proto.getDAGPlan().getName().startsWith(TezConstants.TEZ_PREWARM_DAG_NAME_PREFIX));
 
     client.stop();
   }
@@ -330,7 +336,7 @@ public class TestTezClient {
     thread.join(250);
     thread.interrupt();
     thread.join();
-    Assert.assertThat(exceptionReference.get(),CoreMatchers. instanceOf(InterruptedException.class));
+    Assert.assertThat(exceptionReference.get(), CoreMatchers.instanceOf(InterruptedException.class));
     client.stop();
   }
   
@@ -347,7 +353,7 @@ public class TestTezClient {
       client.waitTillReady();
       Assert.fail();
     } catch (SessionNotRunning e) {
-      Assert.assertTrue(e.getMessage().contains(msg));
+      assertTrue(e.getMessage().contains(msg));
     }
     client.stop();
   }
@@ -362,7 +368,7 @@ public class TestTezClient {
       client.waitTillReady();
       Assert.fail();
     } catch (SessionNotRunning e) {
-      Assert.assertTrue(e.getMessage().contains(TezClient.NO_CLUSTER_DIAGNOSTICS_MSG));
+      assertTrue(e.getMessage().contains(TezClient.NO_CLUSTER_DIAGNOSTICS_MSG));
     }
     client.stop();
   }
@@ -387,9 +393,76 @@ public class TestTezClient {
       client.submitDAG(dag);
       Assert.fail();
     } catch (SessionNotRunning e) {
-      Assert.assertTrue(e.getMessage().contains(msg));
+      assertTrue(e.getMessage().contains(msg));
     }
     client.stop();
   }
 
+  @Test(timeout = 5000)
+  public void testClientBuilder() {
+    TezConfiguration tezConfWitSession = new TezConfiguration();
+    tezConfWitSession.setBoolean(TezConfiguration.TEZ_AM_SESSION_MODE, true);
+
+    TezConfiguration tezConfNoSession = new TezConfiguration();
+    tezConfNoSession.setBoolean(TezConfiguration.TEZ_AM_SESSION_MODE, false);
+
+    AMConfiguration amConf;
+    TezClient tezClient;
+    Credentials credentials = new Credentials();
+    Map<String, LocalResource> localResourceMap = new HashMap<>();
+    localResourceMap.put("testResource", mock(LocalResource.class));
+    ServicePluginsDescriptor servicePluginsDescriptor = ServicePluginsDescriptor.create(true);
+
+    // Session mode via conf
+    tezClient = TezClient.newBuilder("client", tezConfWitSession).build();
+    assertTrue(tezClient.isSession);
+    assertNull(tezClient.servicePluginsDescriptor);
+    assertNotNull(tezClient.apiVersionInfo);
+    amConf = tezClient.amConfig;
+    assertNotNull(amConf);
+    assertEquals(0, amConf.getAMLocalResources().size());
+    assertNull(amConf.getCredentials());
+    assertTrue(
+        amConf.getTezConfiguration().getBoolean(TezConfiguration.TEZ_AM_SESSION_MODE, false));
+
+    // Non-Session mode via conf
+    tezClient = TezClient.newBuilder("client", tezConfNoSession).build();
+    assertFalse(tezClient.isSession);
+    assertNull(tezClient.servicePluginsDescriptor);
+    assertNotNull(tezClient.apiVersionInfo);
+    amConf = tezClient.amConfig;
+    assertNotNull(amConf);
+    assertEquals(0, amConf.getAMLocalResources().size());
+    assertNull(amConf.getCredentials());
+    assertFalse(
+        amConf.getTezConfiguration().getBoolean(TezConfiguration.TEZ_AM_SESSION_MODE, true));
+
+    // no-session via config. API explicit session.
+    tezClient = TezClient.newBuilder("client", tezConfNoSession).setIsSession(true).build();
+    assertTrue(tezClient.isSession);
+    assertNull(tezClient.servicePluginsDescriptor);
+    assertNotNull(tezClient.apiVersionInfo);
+    amConf = tezClient.amConfig;
+    assertNotNull(amConf);
+    assertEquals(0, amConf.getAMLocalResources().size());
+    assertNull(amConf.getCredentials());
+    assertTrue(
+        amConf.getTezConfiguration().getBoolean(TezConfiguration.TEZ_AM_SESSION_MODE, false));
+
+    // Plugins, credentials, local resources
+    tezClient = TezClient.newBuilder("client", tezConfWitSession).setCredentials(credentials)
+        .setLocalResources(localResourceMap).setServicePluginDescriptor(servicePluginsDescriptor)
+        .build();
+    assertTrue(tezClient.isSession);
+    assertEquals(servicePluginsDescriptor, tezClient.servicePluginsDescriptor);
+    assertNotNull(tezClient.apiVersionInfo);
+    amConf = tezClient.amConfig;
+    assertNotNull(amConf);
+    assertEquals(1, amConf.getAMLocalResources().size());
+    assertEquals(localResourceMap, amConf.getAMLocalResources());
+    assertEquals(credentials, amConf.getCredentials());
+    assertTrue(
+        amConf.getTezConfiguration().getBoolean(TezConfiguration.TEZ_AM_SESSION_MODE, false));
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/e445576e/tez-api/src/test/java/org/apache/tez/client/TestTezClientUtils.java
----------------------------------------------------------------------
diff --git a/tez-api/src/test/java/org/apache/tez/client/TestTezClientUtils.java b/tez-api/src/test/java/org/apache/tez/client/TestTezClientUtils.java
index 8946ef0..8f40bbd 100644
--- a/tez-api/src/test/java/org/apache/tez/client/TestTezClientUtils.java
+++ b/tez-api/src/test/java/org/apache/tez/client/TestTezClientUtils.java
@@ -70,6 +70,7 @@ import org.apache.tez.dag.api.TezUncheckedException;
 import org.apache.tez.dag.api.Vertex;
 import org.apache.tez.dag.api.records.DAGProtos.ConfigurationProto;
 import org.apache.tez.dag.api.records.DAGProtos.PlanKeyValuePair;
+import org.apache.tez.serviceplugins.api.ServicePluginsDescriptor;
 import org.junit.Assert;
 import org.junit.Test;
 /**
@@ -500,7 +501,8 @@ public class TestTezClientUtils {
     Assert.assertNotNull(javaOpts);
     Assert.assertTrue(javaOpts.contains("-D" + TezConstants.TEZ_ROOT_LOGGER_NAME + "=FOOBAR")
         && javaOpts.contains(TezConstants.TEZ_CONTAINER_LOG4J_PROPERTIES_FILE)
-        && javaOpts.contains("-Dlog4j.configuratorClass=org.apache.tez.common.TezLog4jConfigurator"));
+        &&
+        javaOpts.contains("-Dlog4j.configuratorClass=org.apache.tez.common.TezLog4jConfigurator"));
   }
 
   @Test (timeout = 5000)
@@ -677,6 +679,16 @@ public class TestTezClientUtils {
     Assert.assertTrue(resourceNames.contains("dir2-f.txt"));
   }
 
-  // TODO TEZ-2003 Add test to validate ServicePluginDescriptor propagation
+  @Test(timeout = 5000)
+  public void testServiceDescriptorSerializationForAM() {
+    Configuration conf = new Configuration(false);
+    ServicePluginsDescriptor servicePluginsDescriptor = ServicePluginsDescriptor.create(true);
+
+    ConfigurationProto confProto = TezClientUtils.createFinalConfProtoForApp(conf, null,
+        servicePluginsDescriptor);
+
+    assertTrue(confProto.hasAmPluginDescriptor());
+    assertTrue(confProto.getAmPluginDescriptor().getUberEnabled());
+  }
 
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/e445576e/tez-api/src/test/java/org/apache/tez/dag/api/TestDAGPlan.java
----------------------------------------------------------------------
diff --git a/tez-api/src/test/java/org/apache/tez/dag/api/TestDAGPlan.java b/tez-api/src/test/java/org/apache/tez/dag/api/TestDAGPlan.java
index fccbb08..cd42109 100644
--- a/tez-api/src/test/java/org/apache/tez/dag/api/TestDAGPlan.java
+++ b/tez-api/src/test/java/org/apache/tez/dag/api/TestDAGPlan.java
@@ -37,11 +37,14 @@ import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.tez.dag.api.EdgeProperty.DataMovementType;
 import org.apache.tez.dag.api.EdgeProperty.DataSourceType;
 import org.apache.tez.dag.api.EdgeProperty.SchedulingType;
+import org.apache.tez.dag.api.Vertex.VertexExecutionContext;
+import org.apache.tez.dag.api.records.DAGProtos;
 import org.apache.tez.dag.api.records.DAGProtos.DAGPlan;
 import org.apache.tez.dag.api.records.DAGProtos.EdgePlan;
 import org.apache.tez.dag.api.records.DAGProtos.PlanTaskConfiguration;
 import org.apache.tez.dag.api.records.DAGProtos.PlanTaskLocationHint;
 import org.apache.tez.dag.api.records.DAGProtos.PlanVertexType;
+import org.apache.tez.dag.api.records.DAGProtos.VertexExecutionContextProto;
 import org.apache.tez.dag.api.records.DAGProtos.VertexPlan;
 import org.junit.Assert;
 import org.junit.Rule;
@@ -131,7 +134,8 @@ public class TestDAGPlan {
     EdgeManagerPluginDescriptor emDesc = edgeProperty.getEdgeManagerDescriptor();
     Assert.assertNotNull(emDesc);
     Assert.assertEquals("emClass", emDesc.getClassName());
-    Assert.assertTrue(Arrays.equals("emPayload".getBytes(), emDesc.getUserPayload().deepCopyAsArray()));
+    Assert.assertTrue(
+        Arrays.equals("emPayload".getBytes(), emDesc.getUserPayload().deepCopyAsArray()));
   }
 
   @Test(timeout = 5000)
@@ -311,4 +315,61 @@ public class TestDAGPlan {
     assertNotNull(fetchedCredentials.getToken(new Text("Token1")));
     assertNotNull(fetchedCredentials.getToken(new Text("Token2")));
   }
+
+  @Test(timeout = 5000)
+  public void testServiceDescriptorPropagation() {
+    DAG dag = DAG.create("testDag");
+    ProcessorDescriptor pd1 = ProcessorDescriptor.create("processor1").
+        setUserPayload(UserPayload.create(ByteBuffer.wrap("processor1Bytes".getBytes())));
+    ProcessorDescriptor pd2 = ProcessorDescriptor.create("processor2").
+        setUserPayload(UserPayload.create(ByteBuffer.wrap("processor2Bytes".getBytes())));
+
+    VertexExecutionContext defaultExecutionContext =
+        VertexExecutionContext.create("plugin", "plugin", "plugin");
+    VertexExecutionContext v1Context = VertexExecutionContext.createExecuteInAm(true);
+
+
+    Vertex v1 = Vertex.create("v1", pd1, 10, Resource.newInstance(1024, 1)).setExecutionContext(v1Context);
+    Vertex v2 = Vertex.create("v2", pd2, 1, Resource.newInstance(1024, 1));
+    v1.setTaskLaunchCmdOpts("").setTaskEnvironment(new HashMap<String, String>())
+        .addTaskLocalFiles(new HashMap<String, LocalResource>());
+    v2.setTaskLaunchCmdOpts("").setTaskEnvironment(new HashMap<String, String>())
+        .addTaskLocalFiles(new HashMap<String, LocalResource>());
+
+    InputDescriptor inputDescriptor = InputDescriptor.create("input").
+        setUserPayload(UserPayload.create(ByteBuffer.wrap("inputBytes".getBytes())));
+    OutputDescriptor outputDescriptor = OutputDescriptor.create("output").
+        setUserPayload(UserPayload.create(ByteBuffer.wrap("outputBytes".getBytes())));
+    Edge edge = Edge.create(v1, v2, EdgeProperty.create(
+        DataMovementType.SCATTER_GATHER, DataSourceType.PERSISTED,
+        SchedulingType.SEQUENTIAL, outputDescriptor, inputDescriptor));
+
+    dag.addVertex(v1).addVertex(v2).addEdge(edge);
+    dag.setExecutionContext(defaultExecutionContext);
+
+    DAGPlan dagProto = dag.createDag(new TezConfiguration(), null, null, null, true);
+
+    assertEquals(2, dagProto.getVertexCount());
+    assertEquals(1, dagProto.getEdgeCount());
+
+    assertTrue(dagProto.hasDefaultExecutionContext());
+    VertexExecutionContextProto defaultContextProto = dagProto.getDefaultExecutionContext();
+    assertFalse(defaultContextProto.getExecuteInContainers());
+    assertFalse(defaultContextProto.getExecuteInAm());
+    assertEquals("plugin", defaultContextProto.getTaskSchedulerName());
+    assertEquals("plugin", defaultContextProto.getContainerLauncherName());
+    assertEquals("plugin", defaultContextProto.getTaskCommName());
+
+    VertexPlan v1Proto = dagProto.getVertex(0);
+    assertTrue(v1Proto.hasExecutionContext());
+    VertexExecutionContextProto v1ContextProto = v1Proto.getExecutionContext();
+    assertFalse(v1ContextProto.getExecuteInContainers());
+    assertTrue(v1ContextProto.getExecuteInAm());
+    assertFalse(v1ContextProto.hasTaskSchedulerName());
+    assertFalse(v1ContextProto.hasContainerLauncherName());
+    assertFalse(v1ContextProto.hasTaskCommName());
+
+    VertexPlan v2Proto = dagProto.getVertex(1);
+    assertFalse(v2Proto.hasExecutionContext());
+  }
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/e445576e/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 51b179a..e37f849 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
@@ -18,15 +18,32 @@
 
 package org.apache.tez.dag.api;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+
 import java.io.IOException;
 
 import java.nio.ByteBuffer;
+import java.util.List;
 
 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.Vertex.VertexExecutionContext;
+import org.apache.tez.dag.api.records.DAGProtos;
+import org.apache.tez.dag.api.records.DAGProtos.AMPluginDescriptorProto;
 import org.apache.tez.dag.api.records.DAGProtos.TezEntityDescriptorProto;
+import org.apache.tez.dag.api.records.DAGProtos.TezNamedEntityDescriptorProto;
+import org.apache.tez.dag.api.records.DAGProtos.VertexExecutionContextProto;
+import org.apache.tez.serviceplugins.api.ContainerLauncher;
+import org.apache.tez.serviceplugins.api.ContainerLauncherDescriptor;
+import org.apache.tez.serviceplugins.api.ServicePluginsDescriptor;
+import org.apache.tez.serviceplugins.api.TaskCommunicatorDescriptor;
+import org.apache.tez.serviceplugins.api.TaskScheduler;
+import org.apache.tez.serviceplugins.api.TaskSchedulerDescriptor;
 import org.junit.Assert;
 import org.junit.Test;
 
@@ -43,7 +60,7 @@ public class TestDagTypeConverters {
         DagTypeConverters.convertToDAGPlan(entityDescriptor);
     Assert.assertEquals(payload.getVersion(), proto.getTezUserPayload().getVersion());
     Assert.assertArrayEquals(payload.deepCopyAsArray(), proto.getTezUserPayload().getUserPayload().toByteArray());
-    Assert.assertTrue(proto.hasHistoryText());
+    assertTrue(proto.hasHistoryText());
     Assert.assertNotEquals(historytext, proto.getHistoryText());
     Assert.assertEquals(historytext, new String(
         TezCommonUtils.decompressByteStringToByteArray(proto.getHistoryText())));
@@ -89,4 +106,181 @@ public class TestDagTypeConverters {
     Assert.assertEquals(2311, lr2UrlDeserialized.getPort());
   }
 
+
+  @Test(timeout = 5000)
+  public void testVertexExecutionContextTranslation() {
+    VertexExecutionContext originalContext;
+    VertexExecutionContextProto contextProto;
+    VertexExecutionContext retrievedContext;
+
+
+    // Uber
+    originalContext = VertexExecutionContext.createExecuteInAm(true);
+    contextProto = DagTypeConverters.convertToProto(originalContext);
+    retrievedContext = DagTypeConverters.convertFromProto(contextProto);
+    assertEquals(originalContext, retrievedContext);
+
+    // Regular containers
+    originalContext = VertexExecutionContext.createExecuteInContainers(true);
+    contextProto = DagTypeConverters.convertToProto(originalContext);
+    retrievedContext = DagTypeConverters.convertFromProto(contextProto);
+    assertEquals(originalContext, retrievedContext);
+
+    // Custom
+    originalContext = VertexExecutionContext.create("plugin", "plugin", "plugin");
+    contextProto = DagTypeConverters.convertToProto(originalContext);
+    retrievedContext = DagTypeConverters.convertFromProto(contextProto);
+    assertEquals(originalContext, retrievedContext);
+  }
+
+
+  static final String testScheduler = "testScheduler";
+  static final String testLauncher = "testLauncher";
+  static final String testComm = "testComm";
+  static final String classSuffix = "_class";
+
+  @Test(timeout = 5000)
+  public void testServiceDescriptorTranslation() {
+
+
+    TaskSchedulerDescriptor[] taskSchedulers;
+    ContainerLauncherDescriptor[] containerLaunchers;
+    TaskCommunicatorDescriptor[] taskComms;
+
+    ServicePluginsDescriptor servicePluginsDescriptor;
+    AMPluginDescriptorProto proto;
+
+    // Uber-execution
+    servicePluginsDescriptor = ServicePluginsDescriptor.create(true);
+    proto = DagTypeConverters.convertServicePluginDescriptoToProto(servicePluginsDescriptor);
+    assertTrue(proto.hasUberEnabled());
+    assertTrue(proto.hasContainersEnabled());
+    assertTrue(proto.getUberEnabled());
+    assertTrue(proto.getContainersEnabled());
+    assertEquals(0, proto.getTaskSchedulersCount());
+    assertEquals(0, proto.getContainerLaunchersCount());
+    assertEquals(0, proto.getTaskCommunicatorsCount());
+
+    // Single plugin set specified. One with a payload.
+    taskSchedulers = createTaskScheduelrs(1, false);
+    containerLaunchers = createContainerLaunchers(1, false);
+    taskComms = createTaskCommunicators(1, true);
+
+    servicePluginsDescriptor = ServicePluginsDescriptor.create(taskSchedulers, containerLaunchers,
+        taskComms);
+    proto = DagTypeConverters.convertServicePluginDescriptoToProto(servicePluginsDescriptor);
+    assertTrue(proto.hasUberEnabled());
+    assertTrue(proto.hasContainersEnabled());
+    assertFalse(proto.getUberEnabled());
+    assertTrue(proto.getContainersEnabled());
+    verifyPlugins(proto.getTaskSchedulersList(), 1, testScheduler, false);
+    verifyPlugins(proto.getContainerLaunchersList(), 1, testLauncher, false);
+    verifyPlugins(proto.getTaskCommunicatorsList(), 1, testComm, true);
+
+
+    // Multiple plugin set specified. All with a payload
+    taskSchedulers = createTaskScheduelrs(3, true);
+    containerLaunchers = createContainerLaunchers(3, true);
+    taskComms = createTaskCommunicators(3, true);
+
+    servicePluginsDescriptor = ServicePluginsDescriptor.create(taskSchedulers, containerLaunchers,
+        taskComms);
+    proto = DagTypeConverters.convertServicePluginDescriptoToProto(servicePluginsDescriptor);
+    assertTrue(proto.hasUberEnabled());
+    assertTrue(proto.hasContainersEnabled());
+    assertFalse(proto.getUberEnabled());
+    assertTrue(proto.getContainersEnabled());
+    verifyPlugins(proto.getTaskSchedulersList(), 3, testScheduler, true);
+    verifyPlugins(proto.getContainerLaunchersList(), 3, testLauncher, true);
+    verifyPlugins(proto.getTaskCommunicatorsList(), 3, testComm, true);
+
+    // Single plugin set specified. One with a payload. No container execution. Uber enabled.
+    taskSchedulers = createTaskScheduelrs(1, false);
+    containerLaunchers = createContainerLaunchers(1, false);
+    taskComms = createTaskCommunicators(1, true);
+
+    servicePluginsDescriptor = ServicePluginsDescriptor.create(false, true, taskSchedulers, containerLaunchers,
+        taskComms);
+    proto = DagTypeConverters.convertServicePluginDescriptoToProto(servicePluginsDescriptor);
+    assertTrue(proto.hasUberEnabled());
+    assertTrue(proto.hasContainersEnabled());
+    assertTrue(proto.getUberEnabled());
+    assertFalse(proto.getContainersEnabled());
+    verifyPlugins(proto.getTaskSchedulersList(), 1, testScheduler, false);
+    verifyPlugins(proto.getContainerLaunchersList(), 1, testLauncher, false);
+    verifyPlugins(proto.getTaskCommunicatorsList(), 1, testComm, true);
+  }
+
+  private void verifyPlugins(List<TezNamedEntityDescriptorProto> entities, int expectedCount,
+                             String baseString, boolean hasPayload) {
+    assertEquals(expectedCount, entities.size());
+    for (int i = 0; i < expectedCount; i++) {
+      assertEquals(indexedEntity(baseString, i), entities.get(i).getName());
+      TezEntityDescriptorProto subEntityProto = entities.get(i).getEntityDescriptor();
+      assertEquals(append(indexedEntity(baseString, i), classSuffix),
+          subEntityProto.getClassName());
+      assertEquals(hasPayload, subEntityProto.hasTezUserPayload());
+      if (hasPayload) {
+        UserPayload userPayload =
+            UserPayload
+                .create(subEntityProto.getTezUserPayload().getUserPayload().asReadOnlyByteBuffer(),
+                    subEntityProto.getTezUserPayload().getVersion());
+        ByteBuffer bb = userPayload.getPayload();
+        assertNotNull(bb);
+        assertEquals(i, bb.getInt());
+      }
+    }
+  }
+
+  private TaskSchedulerDescriptor[] createTaskScheduelrs(int count, boolean withUserPayload) {
+    TaskSchedulerDescriptor[] descriptors = new TaskSchedulerDescriptor[count];
+    for (int i = 0; i < count; i++) {
+      descriptors[i] = TaskSchedulerDescriptor.create(indexedEntity(testScheduler, i),
+          append(indexedEntity(testScheduler, i), classSuffix));
+      if (withUserPayload) {
+        descriptors[i].setUserPayload(createPayload(i));
+      }
+    }
+    return descriptors;
+  }
+
+  private ContainerLauncherDescriptor[] createContainerLaunchers(int count,
+                                                                 boolean withUserPayload) {
+    ContainerLauncherDescriptor[] descriptors = new ContainerLauncherDescriptor[count];
+    for (int i = 0; i < count; i++) {
+      descriptors[i] = ContainerLauncherDescriptor.create(indexedEntity(testLauncher, i),
+          append(indexedEntity(testLauncher, i), classSuffix));
+      if (withUserPayload) {
+        descriptors[i].setUserPayload(createPayload(i));
+      }
+    }
+    return descriptors;
+  }
+
+  private TaskCommunicatorDescriptor[] createTaskCommunicators(int count, boolean withUserPayload) {
+    TaskCommunicatorDescriptor[] descriptors = new TaskCommunicatorDescriptor[count];
+    for (int i = 0; i < count; i++) {
+      descriptors[i] = TaskCommunicatorDescriptor.create(indexedEntity(testComm, i),
+          append(indexedEntity(testComm, i), classSuffix));
+      if (withUserPayload) {
+        descriptors[i].setUserPayload(createPayload(i));
+      }
+    }
+    return descriptors;
+  }
+
+  private static UserPayload createPayload(int i) {
+    ByteBuffer bb = ByteBuffer.allocate(4);
+    bb.putInt(0, i);
+    UserPayload payload = UserPayload.create(bb);
+    return payload;
+  }
+
+  private String indexedEntity(String name, int index) {
+    return name + index;
+  }
+
+  private String append(String s1, String s2) {
+    return s1 + s2;
+  }
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/e445576e/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 4128841..9b16a90 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
@@ -409,9 +409,9 @@ public class DAGAppMaster extends AbstractService {
     }
 
     taskSchedulerDescriptors = parsePlugin(taskSchedulers,
-        (amPluginDescriptorProto == null || amPluginDescriptorProto.getTaskScedulersCount() == 0 ?
+        (amPluginDescriptorProto == null || amPluginDescriptorProto.getTaskSchedulersCount() == 0 ?
             null :
-            amPluginDescriptorProto.getTaskScedulersList()),
+            amPluginDescriptorProto.getTaskSchedulersList()),
         tezYarnEnabled, uberEnabled);
 
     containerLauncherDescriptors = parsePlugin(containerLaunchers,